From 8df60d675cb7e12d3a14f2922366a2a38779aa72 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 14 Dec 2018 10:40:10 -0700 Subject: [PATCH 001/860] Split dbt into core and plugins --- dbt/__init__.py | 1 + dbt/adapters/__init__.py | 1 + dbt/adapters/bigquery/__init__.py | 12 + dbt/adapters/bigquery/connections.py | 290 ++++++++++ dbt/adapters/bigquery/impl.py | 525 ++++++++++++++++++ dbt/adapters/bigquery/relation.py | 144 +++++ dbt/include/__init__.py | 1 + dbt/include/bigquery/__init__.py | 2 + dbt/include/bigquery/dbt_project.yml | 5 + dbt/include/bigquery/macros/adapters.sql | 47 ++ dbt/include/bigquery/macros/etc.sql | 4 + .../macros/materializations/archive.sql | 23 + .../macros/materializations/incremental.sql | 63 +++ .../macros/materializations/merge.sql | 3 + .../bigquery/macros/materializations/seed.sql | 15 + .../macros/materializations/table.sql | 74 +++ .../bigquery/macros/materializations/view.sql | 13 + setup.py | 29 + 18 files changed, 1252 insertions(+) create mode 100644 dbt/__init__.py create mode 100644 dbt/adapters/__init__.py create mode 100644 dbt/adapters/bigquery/__init__.py create mode 100644 dbt/adapters/bigquery/connections.py create mode 100644 dbt/adapters/bigquery/impl.py create mode 100644 dbt/adapters/bigquery/relation.py create mode 100644 dbt/include/__init__.py create mode 100644 dbt/include/bigquery/__init__.py create mode 100644 dbt/include/bigquery/dbt_project.yml create mode 100644 dbt/include/bigquery/macros/adapters.sql create mode 100644 dbt/include/bigquery/macros/etc.sql create mode 100644 dbt/include/bigquery/macros/materializations/archive.sql create mode 100644 dbt/include/bigquery/macros/materializations/incremental.sql create mode 100644 dbt/include/bigquery/macros/materializations/merge.sql create mode 100644 dbt/include/bigquery/macros/materializations/seed.sql create mode 100644 dbt/include/bigquery/macros/materializations/table.sql create mode 100644 dbt/include/bigquery/macros/materializations/view.sql create mode 100644 setup.py diff --git a/dbt/__init__.py b/dbt/__init__.py new file mode 100644 index 000000000..69e3be50d --- /dev/null +++ b/dbt/__init__.py @@ -0,0 +1 @@ +__path__ = __import__('pkgutil').extend_path(__path__, __name__) diff --git a/dbt/adapters/__init__.py b/dbt/adapters/__init__.py new file mode 100644 index 000000000..69e3be50d --- /dev/null +++ b/dbt/adapters/__init__.py @@ -0,0 +1 @@ +__path__ = __import__('pkgutil').extend_path(__path__, __name__) diff --git a/dbt/adapters/bigquery/__init__.py b/dbt/adapters/bigquery/__init__.py new file mode 100644 index 000000000..5707c1690 --- /dev/null +++ b/dbt/adapters/bigquery/__init__.py @@ -0,0 +1,12 @@ +from dbt.adapters.bigquery.connections import BigQueryConnectionManager +from dbt.adapters.bigquery.connections import BigQueryCredentials +from dbt.adapters.bigquery.relation import BigQueryRelation +from dbt.adapters.bigquery.impl import BigQueryAdapter + +from dbt.adapters.base import AdapterPlugin +from dbt.include import bigquery + +Plugin = AdapterPlugin( + adapter=BigQueryAdapter, + credentials=BigQueryCredentials, + include_path=bigquery.PACKAGE_PATH) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py new file mode 100644 index 000000000..d59ffc01f --- /dev/null +++ b/dbt/adapters/bigquery/connections.py @@ -0,0 +1,290 @@ +import abc +from contextlib import contextmanager + +import google.auth +import google.api_core +import google.oauth2 +import google.cloud.exceptions +import google.cloud.bigquery + +import dbt.clients.agate_helper +import dbt.exceptions +from dbt.adapters.base import BaseConnectionManager, Credentials +from dbt.compat import abstractclassmethod +from dbt.logger import GLOBAL_LOGGER as logger + + +BIGQUERY_CREDENTIALS_CONTRACT = { + 'type': 'object', + 'additionalProperties': False, + 'properties': { + 'method': { + 'enum': ['oauth', 'service-account', 'service-account-json'], + }, + 'project': { + 'type': 'string', + }, + 'schema': { + 'type': 'string', + }, + 'keyfile': { + 'type': 'string', + }, + 'keyfile_json': { + 'type': 'object', + }, + 'timeout_seconds': { + 'type': 'integer', + }, + 'location': { + 'type': 'string', + }, + }, + 'required': ['method', 'project', 'schema'], +} + + +class BigQueryCredentials(Credentials): + SCHEMA = BIGQUERY_CREDENTIALS_CONTRACT + + @property + def type(self): + return 'bigquery' + + def _connection_keys(self): + return ('method', 'project', 'schema', 'location') + + +class BigQueryConnectionManager(BaseConnectionManager): + TYPE = 'bigquery' + + SCOPE = ('https://www.googleapis.com/auth/bigquery', + 'https://www.googleapis.com/auth/cloud-platform', + 'https://www.googleapis.com/auth/drive') + + QUERY_TIMEOUT = 300 + + @classmethod + def handle_error(cls, error, message, sql): + logger.debug(message.format(sql=sql)) + logger.debug(error) + error_msg = "\n".join( + [item['message'] for item in error.errors]) + + raise dbt.exceptions.DatabaseException(error_msg) + + @contextmanager + def exception_handler(self, sql, connection_name='master'): + try: + yield + + except google.cloud.exceptions.BadRequest as e: + message = "Bad request while running:\n{sql}" + self.handle_error(e, message, sql) + + except google.cloud.exceptions.Forbidden as e: + message = "Access denied while running:\n{sql}" + self.handle_error(e, message, sql) + + except Exception as e: + logger.debug("Unhandled error while running:\n{}".format(sql)) + logger.debug(e) + raise dbt.exceptions.RuntimeException(dbt.compat.to_string(e)) + + def cancel_open(self): + pass + + @classmethod + def close(cls, connection): + connection.state = 'closed' + + return connection + + def begin(self, name): + pass + + def commit(self, connection): + pass + + @classmethod + def get_bigquery_credentials(cls, profile_credentials): + method = profile_credentials.method + creds = google.oauth2.service_account.Credentials + + if method == 'oauth': + credentials, project_id = google.auth.default(scopes=cls.SCOPE) + return credentials + + elif method == 'service-account': + keyfile = profile_credentials.keyfile + return creds.from_service_account_file(keyfile, scopes=cls.SCOPE) + + elif method == 'service-account-json': + details = profile_credentials.keyfile_json + return creds.from_service_account_info(details, scopes=cls.SCOPE) + + error = ('Invalid `method` in profile: "{}"'.format(method)) + raise dbt.exceptions.FailedToConnectException(error) + + @classmethod + def get_bigquery_client(cls, profile_credentials): + project_name = profile_credentials.project + creds = cls.get_bigquery_credentials(profile_credentials) + location = getattr(profile_credentials, 'location', None) + return google.cloud.bigquery.Client(project_name, creds, + location=location) + + @classmethod + def open(cls, connection): + if connection.state == 'open': + logger.debug('Connection is already open, skipping open.') + return connection + + try: + handle = cls.get_bigquery_client(connection.credentials) + + except google.auth.exceptions.DefaultCredentialsError as e: + logger.info("Please log into GCP to continue") + dbt.clients.gcloud.setup_default_credentials() + + handle = cls.get_bigquery_client(connection.credentials) + + except Exception as e: + raise + logger.debug("Got an error when attempting to create a bigquery " + "client: '{}'".format(e)) + + connection.handle = None + connection.state = 'fail' + + raise dbt.exceptions.FailedToConnectException(str(e)) + + connection.handle = handle + connection.state = 'open' + return connection + + @classmethod + def get_timeout(cls, conn): + credentials = conn['credentials'] + return credentials.get('timeout_seconds', cls.QUERY_TIMEOUT) + + @classmethod + def get_table_from_response(cls, resp): + column_names = [field.name for field in resp.schema] + rows = [dict(row.items()) for row in resp] + return dbt.clients.agate_helper.table_from_data(rows, column_names) + + def raw_execute(self, sql, model_name=None, fetch=False, **kwargs): + conn = self.get(model_name) + client = conn.handle + + logger.debug('On %s: %s', model_name, sql) + + job_config = google.cloud.bigquery.QueryJobConfig() + job_config.use_legacy_sql = False + query_job = client.query(sql, job_config) + + # this blocks until the query has completed + with self.exception_handler(sql, conn.name): + iterator = query_job.result() + + return query_job, iterator + + def execute(self, sql, model_name=None, fetch=None, **kwargs): + _, iterator = self.raw_execute(sql, model_name, fetch, **kwargs) + + if fetch: + res = self.get_table_from_response(iterator) + else: + res = dbt.clients.agate_helper.empty_table() + + # If we get here, the query succeeded + status = 'OK' + return status, res + + def create_bigquery_table(self, dataset_name, table_name, conn_name, + callback, sql): + """Create a bigquery table. The caller must supply a callback + that takes one argument, a `google.cloud.bigquery.Table`, and mutates + it. + """ + conn = self.get(conn_name) + client = conn.handle + + view_ref = self.table_ref(dataset_name, table_name, conn) + view = google.cloud.bigquery.Table(view_ref) + callback(view) + + with self.exception_handler(sql, conn.name): + client.create_table(view) + + def create_view(self, dataset_name, table_name, conn_name, sql): + def callback(table): + table.view_query = sql + table.view_use_legacy_sql = False + + self.create_bigquery_table(dataset_name, table_name, conn_name, + callback, sql) + + def create_table(self, dataset_name, table_name, conn_name, sql): + conn = self.get(conn_name) + client = conn.handle + + table_ref = self.table_ref(dataset_name, table_name, conn) + job_config = google.cloud.bigquery.QueryJobConfig() + job_config.destination = table_ref + job_config.write_disposition = 'WRITE_TRUNCATE' + + query_job = client.query(sql, job_config=job_config) + + # this waits for the job to complete + with self.exception_handler(sql, conn_name): + query_job.result(timeout=self.get_timeout(conn)) + + def create_date_partitioned_table(self, dataset_name, table_name, + conn_name): + def callback(table): + table.partitioning_type = 'DAY' + + self.create_bigquery_table(dataset_name, table_name, conn_name, + callback, 'CREATE DAY PARTITIONED TABLE') + + @staticmethod + def dataset(dataset_name, conn): + dataset_ref = conn.handle.dataset(dataset_name) + return google.cloud.bigquery.Dataset(dataset_ref) + + def table_ref(self, dataset_name, table_name, conn): + dataset = self.dataset(dataset_name, conn) + return dataset.table(table_name) + + def get_bq_table(self, schema, identifier, conn_name=None): + """Get a bigquery table for a schema/model.""" + conn = self.get(conn_name) + table_ref = self.table_ref(schema, identifier, conn) + return conn.handle.get_table(table_ref) + + def drop_dataset(self, dataset_name, conn_name=None): + conn = self.get(conn_name) + dataset = self.dataset(dataset_name, conn) + client = conn.handle + + with self.exception_handler('drop dataset', conn.name): + for table in client.list_tables(dataset): + client.delete_table(table.reference) + client.delete_dataset(dataset) + + def create_dataset(self, dataset_name, conn_name=None): + conn = self.get(conn_name) + client = conn.handle + dataset = self.dataset(dataset_name, conn) + + # Emulate 'create schema if not exists ...' + try: + client.get_dataset(dataset) + return + except google.api_core.exceptions.NotFound: + pass + + with self.exception_handler('create dataset', conn.name): + client.create_dataset(dataset) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py new file mode 100644 index 000000000..d82b199b0 --- /dev/null +++ b/dbt/adapters/bigquery/impl.py @@ -0,0 +1,525 @@ +from __future__ import absolute_import + +import copy + +import dbt.compat +import dbt.deprecations +import dbt.exceptions +import dbt.schema +import dbt.flags as flags +import dbt.clients.gcloud +import dbt.clients.agate_helper + +from dbt.adapters.base import BaseAdapter, available +from dbt.adapters.bigquery import BigQueryRelation +from dbt.adapters.bigquery import BigQueryConnectionManager +from dbt.contracts.connection import Connection +from dbt.logger import GLOBAL_LOGGER as logger + +import google.auth +import google.api_core +import google.oauth2 +import google.cloud.exceptions +import google.cloud.bigquery + +import time +import agate + + +def column_to_bq_schema(col): + """Convert a column to a bigquery schema object. This is here instead of + in dbt.schema to avoid importing google libraries there. + """ + kwargs = {} + if len(col.fields) > 0: + fields = [column_to_bq_schema(field) for field in col.fields] + kwargs = {"fields": fields} + + return google.cloud.bigquery.SchemaField(col.name, col.dtype, col.mode, + **kwargs) + + +class BigQueryAdapter(BaseAdapter): + + RELATION_TYPES = { + 'TABLE': BigQueryRelation.Table, + 'VIEW': BigQueryRelation.View, + 'EXTERNAL': BigQueryRelation.External + } + + Relation = BigQueryRelation + Column = dbt.schema.BigQueryColumn + ConnectionManager = BigQueryConnectionManager + + ### + # Implementations of abstract methods + ### + + @classmethod + def date_function(cls): + return 'CURRENT_TIMESTAMP()' + + @classmethod + def is_cancelable(cls): + return False + + def drop_relation(self, relation, model_name=None): + if self._schema_is_cached(relation.schema, model_name): + self.cache.drop(relation) + + conn = self.connections.get(model_name) + client = conn.handle + + dataset = self.connections.dataset(relation.schema, conn) + relation_object = dataset.table(relation.identifier) + client.delete_table(relation_object) + + def truncate_relation(self, relation, model_name=None): + raise dbt.exceptions.NotImplementedException( + '`truncate` is not implemented for this adapter!' + ) + + def rename_relation(self, from_relation, to_relation, model_name=None): + raise dbt.exceptions.NotImplementedException( + '`rename_relation` is not implemented for this adapter!' + ) + + def list_schemas(self, model_name=None): + conn = self.connections.get(model_name) + client = conn.handle + + with self.connections.exception_handler('list dataset', conn.name): + all_datasets = client.list_datasets(include_all=True) + return [ds.dataset_id for ds in all_datasets] + + def get_columns_in_relation(self, relation, model_name=None): + try: + table = self.connections.get_bq_table( + schema=relation.schema, + identifier=relation.table_name, + conn_name=model_name + ) + return self._get_dbt_columns_from_bq_table(table) + + except (ValueError, google.cloud.exceptions.NotFound) as e: + logger.debug("get_columns_in_table error: {}".format(e)) + return [] + + def expand_column_types(self, goal, current, model_name=None): + # This is a no-op on BigQuery + pass + + def list_relations_without_caching(self, schema, model_name=None): + connection = self.connections.get(model_name) + client = connection.handle + + bigquery_dataset = self.connections.dataset(schema, connection) + + all_tables = client.list_tables( + bigquery_dataset, + # BigQuery paginates tables by alphabetizing them, and using + # the name of the last table on a page as the key for the + # next page. If that key table gets dropped before we run + # list_relations, then this will 404. So, we avoid this + # situation by making the page size sufficiently large. + # see: https://github.com/fishtown-analytics/dbt/issues/726 + # TODO: cache the list of relations up front, and then we + # won't need to do this + max_results=100000) + + # This will 404 if the dataset does not exist. This behavior mirrors + # the implementation of list_relations for other adapters + try: + return [self._bq_table_to_relation(table) for table in all_tables] + except google.api_core.exceptions.NotFound as e: + return [] + + def get_relation(self, schema, identifier, model_name=None): + if self._schema_is_cached(schema, model_name): + # if it's in the cache, use the parent's model of going through + # the relations cache and picking out the relation + return super(BigQueryAdapter, self).get_relation( + schema=schema, + identifier=identifier, + model_name=model_name + ) + + try: + table = self.connections.get_bq_table(schema, identifier) + except google.api_core.exceptions.NotFound: + table = None + return self._bq_table_to_relation(table) + + def create_schema(self, schema, model_name=None): + logger.debug('Creating schema "%s".', schema) + self.connections.create_dataset(schema, model_name) + + def drop_schema(self, schema, model_name=None): + logger.debug('Dropping schema "%s".', schema) + + if not self.check_schema_exists(schema, model_name): + return + self.connections.drop_dataset(schema, model_name) + + @classmethod + def quote(cls, identifier): + return '`{}`'.format(identifier) + + @classmethod + def convert_text_type(cls, agate_table, col_idx): + return "string" + + @classmethod + def convert_number_type(cls, agate_table, col_idx): + decimals = agate_table.aggregate(agate.MaxPrecision(col_idx)) + return "float64" if decimals else "int64" + + @classmethod + def convert_boolean_type(cls, agate_table, col_idx): + return "bool" + + @classmethod + def convert_datetime_type(cls, agate_table, col_idx): + return "datetime" + + @classmethod + def convert_date_type(cls, agate_table, col_idx): + return "date" + + @classmethod + def convert_time_type(cls, agate_table, col_idx): + return "time" + + ### + # Implementation details + ### + def _get_dbt_columns_from_bq_table(self, table): + "Translates BQ SchemaField dicts into dbt BigQueryColumn objects" + + columns = [] + for col in table.schema: + # BigQuery returns type labels that are not valid type specifiers + dtype = self.Column.translate_type(col.field_type) + column = self.Column( + col.name, dtype, col.fields, col.mode) + columns.append(column) + + return columns + + def _agate_to_schema(self, agate_table, column_override): + """Convert agate.Table with column names to a list of bigquery schemas. + """ + bq_schema = [] + for idx, col_name in enumerate(agate_table.column_names): + inferred_type = self.convert_agate_type(agate_table, idx) + type_ = column_override.get(col_name, inferred_type) + bq_schema.append( + google.cloud.bigquery.SchemaField(col_name, type_) + ) + return bq_schema + + def _materialize_as_view(self, model): + model_schema = model.get('schema') + model_name = model.get('name') + model_alias = model.get('alias') + model_sql = model.get('injected_sql') + + logger.debug("Model SQL ({}):\n{}".format(model_name, model_sql)) + self.connections.create_view( + dataset_name=model_schema, + table_name=model_alias, + conn_name=model_name, + sql=model_sql + ) + return "CREATE VIEW" + + def _materialize_as_table(self, model, model_sql, decorator=None): + model_schema = model.get('schema') + model_name = model.get('name') + model_alias = model.get('alias') + + if decorator is None: + table_name = model_alias + else: + table_name = "{}${}".format(model_alias, decorator) + + logger.debug("Model SQL ({}):\n{}".format(table_name, model_sql)) + self.connections.create_table( + dataset_name=model_schema, + conn_name=model_name, + table_name=table_name, + sql=model_sql + ) + + return "CREATE TABLE" + + @classmethod + def poll_until_job_completes(cls, job, timeout): + retry_count = timeout + + while retry_count > 0 and job.state != 'DONE': + retry_count -= 1 + time.sleep(1) + job.reload() + + if job.state != 'DONE': + raise dbt.exceptions.RuntimeException("BigQuery Timeout Exceeded") + + elif job.error_result: + message = '\n'.join( + error['message'].strip() for error in job.errors + ) + raise dbt.exceptions.RuntimeException(message) + + def _bq_table_to_relation(self, bq_table): + if bq_table is None: + return None + + return self.Relation.create( + project=bq_table.project, + schema=bq_table.dataset_id, + identifier=bq_table.table_id, + quote_policy={ + 'schema': True, + 'identifier': True + }, + type=self.RELATION_TYPES.get(bq_table.table_type)) + + @classmethod + def warning_on_hooks(hook_type): + msg = "{} is not supported in bigquery and will be ignored" + dbt.ui.printer.print_timestamped_line(msg.format(hook_type), + dbt.ui.printer.COLOR_FG_YELLOW) + + @available + def add_query(self, sql, model_name=None, auto_begin=True, + bindings=None, abridge_sql_log=False): + if model_name in ['on-run-start', 'on-run-end']: + self.warning_on_hooks(model_name) + else: + raise dbt.exceptions.NotImplementedException( + '`add_query` is not implemented for this adapter!') + + def quote_schema_and_table(self, schema, table, model_name=None): + return self.render_relation(self.quote(schema), self.quote(table)) + + def render_relation(cls, schema, table): + connection = self.connections.get(None) + project = connection.credentials.project + return '{}.{}.{}'.format(self.quote(project), schema, table) + + ### + # Special bigquery adapter methods + ### + @available + def make_date_partitioned_table(self, schema, identifier, model_name=None): + return self.connections.create_date_partitioned_table( + dataset_name=schema, + table_name=identifier, + conn_name=model_name + ) + + @available + def execute_model(self, model, materialization, sql_override=None, + decorator=None, model_name=None): + + if sql_override is None: + sql_override = model.get('injected_sql') + + if flags.STRICT_MODE: + connection = self.connections.get(model.get('name')) + Connection(**connection) + + if materialization == 'view': + res = self._materialize_as_view(model) + elif materialization == 'table': + res = self._materialize_as_table(model, sql_override, decorator) + else: + msg = "Invalid relation type: '{}'".format(materialization) + raise dbt.exceptions.RuntimeException(msg, model) + + return res + + @available + def create_temporary_table(self, sql, model_name=None, **kwargs): + + # BQ queries always return a temp table with their results + query_job, _ = self.connections.raw_execute(sql, model_name) + bq_table = query_job.destination + + return self.Relation.create( + project=bq_table.project, + schema=bq_table.dataset_id, + identifier=bq_table.table_id, + quote_policy={ + 'schema': True, + 'identifier': True + }, + type=BigQueryRelation.Table) + + @available + def alter_table_add_columns(self, relation, columns, model_name=None): + + logger.debug('Adding columns ({}) to table {}".'.format( + columns, relation)) + + conn = self.connections.get(model_name) + client = conn.handle + + table_ref = self.connections.table_ref(relation.schema, + relation.identifier, conn) + table = client.get_table(table_ref) + + new_columns = [column_to_bq_schema(col) for col in columns] + new_schema = table.schema + new_columns + + new_table = google.cloud.bigquery.Table(table_ref, schema=new_schema) + client.update_table(new_table, ['schema']) + + @available + def load_dataframe(self, schema, table_name, agate_table, column_override, + model_name=None): + bq_schema = self._agate_to_schema(agate_table, column_override) + conn = self.connections.get(model_name) + client = conn.handle + + table = self.connections.table_ref(schema, table_name, conn) + + load_config = google.cloud.bigquery.LoadJobConfig() + load_config.skip_leading_rows = 1 + load_config.schema = bq_schema + + with open(agate_table.original_abspath, "rb") as f: + job = client.load_table_from_file(f, table, rewind=True, + job_config=load_config) + + timeout = self.connections.get_timeout(conn) + with self.connections.exception_handler("LOAD TABLE", conn.name): + self.poll_until_job_completes(job, timeout) + + ### + # The get_catalog implementation for bigquery + ### + def _flat_columns_in_table(self, table): + """An iterator over the flattened columns for a given schema and table. + Resolves child columns as having the name "parent.child". + """ + for col in self._get_dbt_columns_from_bq_table(table): + flattened = col.flatten() + for subcol in flattened: + yield subcol + + @classmethod + def _get_stats_column_names(cls): + """Construct a tuple of the column names for stats. Each stat has 4 + columns of data. + """ + columns = [] + stats = ('num_bytes', 'num_rows', 'location', 'partitioning_type', + 'clustering_fields') + stat_components = ('label', 'value', 'description', 'include') + for stat_id in stats: + for stat_component in stat_components: + columns.append('stats:{}:{}'.format(stat_id, stat_component)) + return tuple(columns) + + @classmethod + def _get_stats_columns(cls, table, relation_type): + """Given a table, return an iterator of key/value pairs for stats + column names/values. + """ + column_names = cls._get_stats_column_names() + + # agate does not handle the array of column names gracefully + clustering_value = None + if table.clustering_fields is not None: + clustering_value = ','.join(table.clustering_fields) + # cast num_bytes/num_rows to str before they get to agate, or else + # agate will incorrectly decide they are booleans. + column_values = ( + 'Number of bytes', + str(table.num_bytes), + 'The number of bytes this table consumes', + relation_type == 'table', + + 'Number of rows', + str(table.num_rows), + 'The number of rows in this table', + relation_type == 'table', + + 'Location', + table.location, + 'The geographic location of this table', + True, + + 'Partitioning Type', + table.partitioning_type, + 'The partitioning type used for this table', + relation_type == 'table', + + 'Clustering Fields', + clustering_value, + 'The clustering fields for this table', + relation_type == 'table', + ) + return zip(column_names, column_values) + + def get_catalog(self, manifest): + connection = self.connections.get('catalog') + client = connection.handle + + schemas = { + node.to_dict()['schema'] + for node in manifest.nodes.values() + } + + column_names = ( + 'table_schema', + 'table_name', + 'table_type', + 'table_comment', + # does not exist in bigquery, but included for consistency + 'table_owner', + 'column_name', + 'column_index', + 'column_type', + 'column_comment', + ) + all_names = column_names + self._get_stats_column_names() + columns = [] + + for schema_name in schemas: + relations = self.list_relations(schema_name) + for relation in relations: + + # This relation contains a subset of the info we care about. + # Fetch the full table object here + table_ref = self.connections.table_ref( + relation.schema, + relation.identifier, + connection + ) + table = client.get_table(table_ref) + + flattened = self._flat_columns_in_table(table) + relation_stats = dict(self._get_stats_columns(table, + relation.type)) + + for index, column in enumerate(flattened, start=1): + column_data = ( + relation.schema, + relation.name, + relation.type, + None, + None, + column.name, + index, + column.data_type, + None, + ) + column_dict = dict(zip(column_names, column_data)) + column_dict.update(copy.deepcopy(relation_stats)) + + columns.append(column_dict) + + return dbt.clients.agate_helper.table_from_data(columns, all_names) diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py new file mode 100644 index 000000000..077444a4d --- /dev/null +++ b/dbt/adapters/bigquery/relation.py @@ -0,0 +1,144 @@ +from dbt.adapters.base.relation import BaseRelation +from dbt.utils import filter_null_values + + +class BigQueryRelation(BaseRelation): + External = "external" + + DEFAULTS = { + 'metadata': { + 'type': 'BigQueryRelation' + }, + 'quote_character': '`', + 'quote_policy': { + 'project': True, + 'schema': True, + 'identifier': True + }, + 'include_policy': { + 'project': True, + 'schema': True, + 'identifier': True + } + } + + PATH_SCHEMA = { + 'type': 'object', + 'properties': { + 'project': {'type': ['string', 'null']}, + 'schema': {'type': ['string', 'null']}, + 'identifier': {'type': 'string'}, + }, + 'required': ['project', 'schema', 'identifier'], + } + + POLICY_SCHEMA = { + 'type': 'object', + 'properties': { + 'project': {'type': 'boolean'}, + 'schema': {'type': 'boolean'}, + 'identifier': {'type': 'boolean'}, + }, + 'required': ['project', 'schema', 'identifier'], + } + + SCHEMA = { + 'type': 'object', + 'properties': { + 'metadata': { + 'type': 'object', + 'properties': { + 'type': { + 'type': 'string', + 'const': 'BigQueryRelation', + }, + }, + }, + 'type': { + 'enum': BaseRelation.RelationTypes + [External, None], + }, + 'path': PATH_SCHEMA, + 'include_policy': POLICY_SCHEMA, + 'quote_policy': POLICY_SCHEMA, + 'quote_character': {'type': 'string'}, + }, + 'required': ['metadata', 'type', 'path', 'include_policy', + 'quote_policy', 'quote_character'] + } + + PATH_ELEMENTS = ['project', 'schema', 'identifier'] + + def matches(self, project=None, schema=None, identifier=None): + search = filter_null_values({ + 'project': project, + 'schema': schema, + 'identifier': identifier + }) + + if not search: + # nothing was passed in + pass + + for k, v in search.items(): + if self.get_path_part(k) != v: + return False + + return True + + @classmethod + def _create_from_node(cls, config, node, **kwargs): + return cls.create( + project=config.credentials.project, + schema=node.get('schema'), + identifier=node.get('alias'), + **kwargs) + + @classmethod + def create(cls, project=None, schema=None, + identifier=None, table_name=None, + type=None, **kwargs): + if table_name is None: + table_name = identifier + + return cls(type=type, + path={ + 'project': project, + 'schema': schema, + 'identifier': identifier + }, + table_name=table_name, + **kwargs) + + def quote(self, project=None, schema=None, identifier=None): + policy = filter_null_values({ + 'project': project, + 'schema': schema, + 'identifier': identifier + }) + + return self.incorporate(quote_policy=policy) + + def include(self, project=None, schema=None, identifier=None): + policy = filter_null_values({ + 'project': project, + 'schema': schema, + 'identifier': identifier + }) + + return self.incorporate(include_policy=policy) + + @property + def project(self): + return self.path.get('project') + + @property + def schema(self): + return self.path.get('schema') + + @property + def dataset(self): + return self.path.get('schema') + + @property + def identifier(self): + return self.path.get('identifier') diff --git a/dbt/include/__init__.py b/dbt/include/__init__.py new file mode 100644 index 000000000..69e3be50d --- /dev/null +++ b/dbt/include/__init__.py @@ -0,0 +1 @@ +__path__ = __import__('pkgutil').extend_path(__path__, __name__) diff --git a/dbt/include/bigquery/__init__.py b/dbt/include/bigquery/__init__.py new file mode 100644 index 000000000..87098354a --- /dev/null +++ b/dbt/include/bigquery/__init__.py @@ -0,0 +1,2 @@ +import os +PACKAGE_PATH = os.path.dirname(os.path.dirname(__file__)) diff --git a/dbt/include/bigquery/dbt_project.yml b/dbt/include/bigquery/dbt_project.yml new file mode 100644 index 000000000..edae53869 --- /dev/null +++ b/dbt/include/bigquery/dbt_project.yml @@ -0,0 +1,5 @@ + +name: dbt_bigquery +version: 1.0 + +macro-paths: ["macros"] diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql new file mode 100644 index 000000000..b223b6d4e --- /dev/null +++ b/dbt/include/bigquery/macros/adapters.sql @@ -0,0 +1,47 @@ +{% macro partition_by(raw_partition_by) %} + {%- if raw_partition_by is none -%} + {{ return('') }} + {% endif %} + + {% set partition_by_clause %} + partition by {{ raw_partition_by }} + {%- endset -%} + + {{ return(partition_by_clause) }} +{%- endmacro -%} + + +{% macro cluster_by(raw_cluster_by) %} + {%- if raw_cluster_by is not none -%} + cluster by + {% if raw_cluster_by is string -%} + {% set raw_cluster_by = [raw_cluster_by] %} + {%- endif -%} + {%- for cluster in raw_cluster_by -%} + {{ cluster }} + {%- if not loop.last -%},{%- endif -%} + {%- endfor -%} + + {% endif %} + +{%- endmacro -%} + + +{% macro bigquery__create_table_as(temporary, relation, sql) -%} + {%- set raw_partition_by = config.get('partition_by', none) -%} + {%- set raw_cluster_by = config.get('cluster_by', none) -%} + + create or replace table {{ relation }} + {{ partition_by(raw_partition_by) }} + {{ cluster_by(raw_cluster_by) }} + as ( + {{ sql }} + ); +{% endmacro %} + + +{% macro bigquery__create_view_as(relation, sql) -%} + create or replace view {{ relation }} as ( + {{ sql }} + ); +{% endmacro %} diff --git a/dbt/include/bigquery/macros/etc.sql b/dbt/include/bigquery/macros/etc.sql new file mode 100644 index 000000000..7bd0cba90 --- /dev/null +++ b/dbt/include/bigquery/macros/etc.sql @@ -0,0 +1,4 @@ + +{% macro date_sharded_table(base_name) %} + {{ return(base_name ~ "[DBT__PARTITION_DATE]") }} +{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/archive.sql b/dbt/include/bigquery/macros/materializations/archive.sql new file mode 100644 index 000000000..5548b71a7 --- /dev/null +++ b/dbt/include/bigquery/macros/materializations/archive.sql @@ -0,0 +1,23 @@ +{% macro bigquery__create_temporary_table(sql, relation) %} + {% set tmp_relation = adapter.create_temporary_table(sql) %} + {{ return(tmp_relation) }} +{% endmacro %} + + +{% macro bigquery__archive_scd_hash() %} + to_hex(md5(concat(cast(`dbt_pk` as string), '|', cast(`dbt_updated_at` as string)))) +{% endmacro %} + + +{% macro bigquery__create_columns(relation, columns) %} + {{ adapter.alter_table_add_columns(relation, columns) }} +{% endmacro %} + + +{% macro bigquery__archive_update(target_relation, tmp_relation) %} + update {{ target_relation }} as dest + set dest.{{ adapter.quote('valid_to') }} = tmp.{{ adapter.quote('valid_to') }} + from {{ tmp_relation }} as tmp + where tmp.{{ adapter.quote('scd_id') }} = dest.{{ adapter.quote('scd_id') }} + and {{ adapter.quote('change_type') }} = 'update'; +{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql new file mode 100644 index 000000000..d43b5f7a6 --- /dev/null +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -0,0 +1,63 @@ + +{% materialization incremental, adapter='bigquery' -%} + + {%- set unique_key = config.get('unique_key') -%} + {%- set sql_where = config.get('sql_where') -%} + + {%- set non_destructive_mode = (flags.NON_DESTRUCTIVE == True) -%} + {%- set full_refresh_mode = (flags.FULL_REFRESH == True) -%} + + {% if non_destructive_mode %} + {{ exceptions.raise_compiler_error("--non-destructive mode is not supported on BigQuery") }} + {% endif %} + + {%- set identifier = model['alias'] -%} + + {%- set old_relation = adapter.get_relation(schema=schema, identifier=identifier) -%} + + {%- set target_relation = api.Relation.create(identifier=identifier, schema=schema, type='table') -%} + + {%- set exists_as_table = (old_relation is not none and old_relation.is_table) -%} + {%- set exists_not_as_table = (old_relation is not none and not old_relation.is_table) -%} + + {%- set should_drop = (full_refresh_mode or exists_not_as_table) -%} + {%- set force_create = (full_refresh_mode) -%} + + -- setup + {% if old_relation is none -%} + -- noop + {%- elif should_drop -%} + {{ adapter.drop_relation(old_relation) }} + {%- set old_relation = none -%} + {%- endif %} + + {% set source_sql -%} + {#-- wrap sql in parens to make it a subquery --#} + ( + select * from ( + {{ sql }} + ) + {% if sql_where %} + where ({{ sql_where }}) or ({{ sql_where }}) is null + {% endif %} + ) + {%- endset -%} + + + {{ run_hooks(pre_hooks) }} + + -- build model + {% if force_create or old_relation is none -%} + {%- call statement('main') -%} + {{ create_table_as(False, target_relation, sql) }} + {%- endcall -%} + {%- else -%} + {% set dest_columns = adapter.get_columns_in_table(schema, identifier) %} + {%- call statement('main') -%} + {{ get_merge_sql(target_relation, source_sql, unique_key, dest_columns) }} + {% endcall %} + {%- endif %} + + {{ run_hooks(post_hooks) }} + +{%- endmaterialization %} diff --git a/dbt/include/bigquery/macros/materializations/merge.sql b/dbt/include/bigquery/macros/materializations/merge.sql new file mode 100644 index 000000000..8e8f42a35 --- /dev/null +++ b/dbt/include/bigquery/macros/materializations/merge.sql @@ -0,0 +1,3 @@ +{% macro bigquery__get_merge_sql(target, source, unique_key, dest_columns) %} + {{ common_get_merge_sql(target, source, unique_key, dest_columns) }} +{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/seed.sql b/dbt/include/bigquery/macros/materializations/seed.sql new file mode 100644 index 000000000..11f67a9db --- /dev/null +++ b/dbt/include/bigquery/macros/materializations/seed.sql @@ -0,0 +1,15 @@ + +{% macro bigquery__create_csv_table(model) %} + -- no-op +{% endmacro %} + +{% macro bigquery__reset_csv_table(model, full_refresh, old_relation) %} + {{ adapter.drop_relation(old_relation) }} +{% endmacro %} + +{% macro bigquery__load_csv_rows(model) %} + + {%- set column_override = model['config'].get('column_types', {}) -%} + {{ adapter.load_dataframe(model['schema'], model['alias'], model['agate_table'], column_override) }} + +{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql new file mode 100644 index 000000000..92f54d30c --- /dev/null +++ b/dbt/include/bigquery/macros/materializations/table.sql @@ -0,0 +1,74 @@ +{% macro make_date_partitioned_table(model, relation, dates, should_create, verbose=False) %} + + {% if should_create %} + {{ adapter.make_date_partitioned_table(relation.schema, relation.identifier) }} + {% endif %} + + {% for date in dates %} + {% set date = (date | string) %} + {% if verbose %} + {% set table_start_time = modules.datetime.datetime.now().strftime("%H:%M:%S") %} + {{ log(table_start_time ~ ' | -> Running for day ' ~ date, info=True) }} + {% endif %} + + {% set fixed_sql = model['injected_sql'] | replace('[DBT__PARTITION_DATE]', date) %} + {% set _ = adapter.execute_model(model, 'table', fixed_sql, decorator=date) %} + {% endfor %} + + {% set num_days = dates | length %} + {% if num_days == 1 %} + {% set result_str = 'CREATED 1 PARTITION' %} + {% else %} + {% set result_str = 'CREATED ' ~ num_days ~ ' PARTITIONS' %} + {% endif %} + + {{ store_result('main', status=result_str) }} + +{% endmacro %} + +{% materialization table, adapter='bigquery' -%} + + {%- set identifier = model['alias'] -%} + {%- set non_destructive_mode = (flags.NON_DESTRUCTIVE == True) -%} + {%- set old_relation = adapter.get_relation(schema=schema, identifier=identifier) -%} + {%- set exists_not_as_table = (old_relation is not none and not old_relation.is_table) -%} + {%- set target_relation = api.Relation.create(schema=schema, identifier=identifier, type='table') -%} + {%- set verbose = config.get('verbose', False) -%} + + {# partitions: iterate over each partition, running a separate query in a for-loop #} + {%- set partitions = config.get('partitions') -%} + + {% if partitions %} + {% if partitions is number or partitions is string %} + {% set partitions = [(partitions | string)] %} + {% endif %} + + {% if partitions is not iterable %} + {{ exceptions.raise_compiler_error("Provided `partitions` configuration is not a list. Got: " ~ partitions, model) }} + {% endif %} + {% endif %} + + {{ run_hooks(pre_hooks) }} + + {# + Since dbt uses WRITE_TRUNCATE mode for tables, we only need to drop this thing + if it is not a table. If it _is_ already a table, then we can overwrite it without downtime + #} + {%- if exists_not_as_table -%} + {{ adapter.drop_relation(old_relation) }} + {%- endif -%} + + -- build model + {% if partitions %} + {# Create the dp-table if 1. it does not exist or 2. it existed, but we just dropped it #} + {%- set should_create = (old_relation is none or exists_not_as_table) -%} + {{ make_date_partitioned_table(model, target_relation, partitions, should_create, verbose) }} + {% else %} + {% call statement('main') -%} + {{ create_table_as(False, target_relation, sql) }} + {% endcall -%} + {% endif %} + + {{ run_hooks(post_hooks) }} + +{% endmaterialization %} diff --git a/dbt/include/bigquery/macros/materializations/view.sql b/dbt/include/bigquery/macros/materializations/view.sql new file mode 100644 index 000000000..561c38bb5 --- /dev/null +++ b/dbt/include/bigquery/macros/materializations/view.sql @@ -0,0 +1,13 @@ + +{% macro bigquery__handle_existing_table(full_refresh, non_destructive_mode, old_relation) %} + {%- if full_refresh and not non_destructive_mode -%} + {{ adapter.drop_relation(old_relation) }} + {%- else -%} + {{ exceptions.relation_wrong_type(old_relation, 'view') }} + {%- endif -%} +{% endmacro %} + + +{% materialization view, adapter='bigquery' -%} + {{ create_or_replace_view(run_outside_transaction_hooks=False) }} +{%- endmaterialization %} diff --git a/setup.py b/setup.py new file mode 100644 index 000000000..5c743575d --- /dev/null +++ b/setup.py @@ -0,0 +1,29 @@ +#!/usr/bin/env python +from setuptools import find_packages +from distutils.core import setup + +package_name = "dbt-bigquery" +package_version = "0.13.0a1" +description = """The bigquery adapter plugin for dbt (data build tool)""" + + +setup( + name=package_name, + version=package_version, + description=description, + long_description_content_type=description, + author="Fishtown Analytics", + author_email="info@fishtownanalytics.com", + url="https://github.com/fishtown-analytics/dbt", + packages=find_packages(), + package_data={ + 'dbt': [ + 'include/bigquery/macros/*.sql', + 'include/bigquery/macros/**/*.sql', + ] + }, + install_requires=[ + 'dbt-core=={}'.format(package_version), + 'google-cloud-bigquery>=1.0.0,<2', + ] +) From 22c518a17170b3bf63b22882798ab17fb1329901 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 20 Dec 2018 08:40:13 -0700 Subject: [PATCH 002/860] fix the issue on bigquery where tests executed with a connection name of "master" --- dbt/adapters/bigquery/connections.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index d59ffc01f..3c1d2e7f4 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -174,11 +174,11 @@ def get_table_from_response(cls, resp): rows = [dict(row.items()) for row in resp] return dbt.clients.agate_helper.table_from_data(rows, column_names) - def raw_execute(self, sql, model_name=None, fetch=False, **kwargs): - conn = self.get(model_name) + def raw_execute(self, sql, name=None, fetch=False): + conn = self.get(name) client = conn.handle - logger.debug('On %s: %s', model_name, sql) + logger.debug('On %s: %s', name, sql) job_config = google.cloud.bigquery.QueryJobConfig() job_config.use_legacy_sql = False @@ -190,8 +190,9 @@ def raw_execute(self, sql, model_name=None, fetch=False, **kwargs): return query_job, iterator - def execute(self, sql, model_name=None, fetch=None, **kwargs): - _, iterator = self.raw_execute(sql, model_name, fetch, **kwargs) + def execute(self, sql, name=None, auto_begin=False, fetch=None): + # auto_begin is ignored on bigquery, and only included for consistency + _, iterator = self.raw_execute(sql, name=name, fetch=fetch) if fetch: res = self.get_table_from_response(iterator) From 6ca2cdfcce8ba398bf449362797c581d408e2ab1 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 2 Jan 2019 11:35:42 -0700 Subject: [PATCH 003/860] Make config() accept database, add adapter-specifc aliasing Add concept of aliasing for credentials/relations All databases use database, schema, and identifier internally now: - Postgres/Redshift have 'dbname' aliased to database and pass to password - Bigquery has 'project' aliased to database and 'dataset' aliased to schema - Set default database include policy to True everywhere config() calls accept aliases instead of canonical names Remove unused functions and change others to accept Relations (see core/CHANGELOG.md) Add catalog, etc support for multiple databases --- dbt/adapters/bigquery/connections.py | 32 ++++---- dbt/adapters/bigquery/impl.py | 76 ++++++++++--------- dbt/adapters/bigquery/relation.py | 65 ++++------------ .../macros/materializations/incremental.sql | 6 +- .../bigquery/macros/materializations/seed.sql | 2 +- .../macros/materializations/table.sql | 4 +- setup.py | 1 + 7 files changed, 79 insertions(+), 107 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 3c1d2e7f4..95c3215ec 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -21,7 +21,7 @@ 'method': { 'enum': ['oauth', 'service-account', 'service-account-json'], }, - 'project': { + 'database': { 'type': 'string', }, 'schema': { @@ -40,19 +40,23 @@ 'type': 'string', }, }, - 'required': ['method', 'project', 'schema'], + 'required': ['method', 'database', 'schema'], } class BigQueryCredentials(Credentials): SCHEMA = BIGQUERY_CREDENTIALS_CONTRACT + ALIASES = { + 'project': 'database', + 'dataset': 'schema', + } @property def type(self): return 'bigquery' def _connection_keys(self): - return ('method', 'project', 'schema', 'location') + return ('method', 'database', 'schema', 'location') class BigQueryConnectionManager(BaseConnectionManager): @@ -128,7 +132,7 @@ def get_bigquery_credentials(cls, profile_credentials): @classmethod def get_bigquery_client(cls, profile_credentials): - project_name = profile_credentials.project + project_name = profile_credentials.database creds = cls.get_bigquery_credentials(profile_credentials) location = getattr(profile_credentials, 'location', None) return google.cloud.bigquery.Client(project_name, creds, @@ -251,23 +255,23 @@ def callback(table): callback, 'CREATE DAY PARTITIONED TABLE') @staticmethod - def dataset(dataset_name, conn): - dataset_ref = conn.handle.dataset(dataset_name) + def dataset(dataset_name, conn, project=None): + dataset_ref = conn.handle.dataset(dataset_name, project) return google.cloud.bigquery.Dataset(dataset_ref) - def table_ref(self, dataset_name, table_name, conn): - dataset = self.dataset(dataset_name, conn) + def table_ref(self, dataset_name, table_name, conn, project=None): + dataset = self.dataset(dataset_name, conn, project=project) return dataset.table(table_name) - def get_bq_table(self, schema, identifier, conn_name=None): + def get_bq_table(self, schema, identifier, conn_name=None, database=None): """Get a bigquery table for a schema/model.""" conn = self.get(conn_name) - table_ref = self.table_ref(schema, identifier, conn) + table_ref = self.table_ref(schema, identifier, conn, project=database) return conn.handle.get_table(table_ref) - def drop_dataset(self, dataset_name, conn_name=None): + def drop_dataset(self, project_name, dataset_name, conn_name=None): conn = self.get(conn_name) - dataset = self.dataset(dataset_name, conn) + dataset = self.dataset(dataset_name, conn, project=project_name) client = conn.handle with self.exception_handler('drop dataset', conn.name): @@ -275,10 +279,10 @@ def drop_dataset(self, dataset_name, conn_name=None): client.delete_table(table.reference) client.delete_dataset(dataset) - def create_dataset(self, dataset_name, conn_name=None): + def create_dataset(self, project_name, dataset_name, conn_name=None): conn = self.get(conn_name) client = conn.handle - dataset = self.dataset(dataset_name, conn) + dataset = self.dataset(dataset_name, conn, project=project_name) # Emulate 'create schema if not exists ...' try: diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index d82b199b0..8cd281ed5 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -64,7 +64,9 @@ def is_cancelable(cls): return False def drop_relation(self, relation, model_name=None): - if self._schema_is_cached(relation.schema, model_name): + is_cached = self._schema_is_cached(relation.database, relation.schema, + model_name) + if is_cached: self.cache.drop(relation) conn = self.connections.get(model_name) @@ -84,17 +86,19 @@ def rename_relation(self, from_relation, to_relation, model_name=None): '`rename_relation` is not implemented for this adapter!' ) - def list_schemas(self, model_name=None): + def list_schemas(self, database, model_name=None): conn = self.connections.get(model_name) client = conn.handle with self.connections.exception_handler('list dataset', conn.name): - all_datasets = client.list_datasets(include_all=True) + all_datasets = client.list_datasets(project=database, + include_all=True) return [ds.dataset_id for ds in all_datasets] def get_columns_in_relation(self, relation, model_name=None): try: table = self.connections.get_bq_table( + database=relation.database, schema=relation.schema, identifier=relation.table_name, conn_name=model_name @@ -102,18 +106,20 @@ def get_columns_in_relation(self, relation, model_name=None): return self._get_dbt_columns_from_bq_table(table) except (ValueError, google.cloud.exceptions.NotFound) as e: - logger.debug("get_columns_in_table error: {}".format(e)) + logger.debug("get_columns_in_relation error: {}".format(e)) return [] def expand_column_types(self, goal, current, model_name=None): # This is a no-op on BigQuery pass - def list_relations_without_caching(self, schema, model_name=None): + def list_relations_without_caching(self, database, schema, + model_name=None): connection = self.connections.get(model_name) client = connection.handle - bigquery_dataset = self.connections.dataset(schema, connection) + bigquery_dataset = self.connections.dataset(schema, connection, + project=database) all_tables = client.list_tables( bigquery_dataset, @@ -134,32 +140,34 @@ def list_relations_without_caching(self, schema, model_name=None): except google.api_core.exceptions.NotFound as e: return [] - def get_relation(self, schema, identifier, model_name=None): - if self._schema_is_cached(schema, model_name): + def get_relation(self, database, schema, identifier, model_name=None): + if self._schema_is_cached(database, schema, model_name): # if it's in the cache, use the parent's model of going through # the relations cache and picking out the relation return super(BigQueryAdapter, self).get_relation( + database=database, schema=schema, identifier=identifier, model_name=model_name ) try: - table = self.connections.get_bq_table(schema, identifier) + table = self.connections.get_bq_table(schema, identifier, + database=database) except google.api_core.exceptions.NotFound: table = None return self._bq_table_to_relation(table) - def create_schema(self, schema, model_name=None): - logger.debug('Creating schema "%s".', schema) - self.connections.create_dataset(schema, model_name) + def create_schema(self, database, schema, model_name=None): + logger.debug('Creating schema "%s.%s".', database, schema) + self.connections.create_dataset(database, schema, model_name) - def drop_schema(self, schema, model_name=None): - logger.debug('Dropping schema "%s".', schema) + def drop_schema(self, database, schema, model_name=None): + logger.debug('Dropping schema "%s.%s".', database, schema) - if not self.check_schema_exists(schema, model_name): + if not self.check_schema_exists(database, schema, model_name): return - self.connections.drop_dataset(schema, model_name) + self.connections.drop_dataset(database, schema, model_name) @classmethod def quote(cls, identifier): @@ -276,7 +284,7 @@ def _bq_table_to_relation(self, bq_table): return None return self.Relation.create( - project=bq_table.project, + database=bq_table.project, schema=bq_table.dataset_id, identifier=bq_table.table_id, quote_policy={ @@ -300,14 +308,6 @@ def add_query(self, sql, model_name=None, auto_begin=True, raise dbt.exceptions.NotImplementedException( '`add_query` is not implemented for this adapter!') - def quote_schema_and_table(self, schema, table, model_name=None): - return self.render_relation(self.quote(schema), self.quote(table)) - - def render_relation(cls, schema, table): - connection = self.connections.get(None) - project = connection.credentials.project - return '{}.{}.{}'.format(self.quote(project), schema, table) - ### # Special bigquery adapter methods ### @@ -348,7 +348,7 @@ def create_temporary_table(self, sql, model_name=None, **kwargs): bq_table = query_job.destination return self.Relation.create( - project=bq_table.project, + database=bq_table.project, schema=bq_table.dataset_id, identifier=bq_table.table_id, quote_policy={ @@ -367,7 +367,8 @@ def alter_table_add_columns(self, relation, columns, model_name=None): client = conn.handle table_ref = self.connections.table_ref(relation.schema, - relation.identifier, conn) + relation.identifier, conn, + project=relation.database) table = client.get_table(table_ref) new_columns = [column_to_bq_schema(col) for col in columns] @@ -377,13 +378,14 @@ def alter_table_add_columns(self, relation, columns, model_name=None): client.update_table(new_table, ['schema']) @available - def load_dataframe(self, schema, table_name, agate_table, column_override, - model_name=None): + def load_dataframe(self, database, schema, table_name, agate_table, + column_override, model_name=None): bq_schema = self._agate_to_schema(agate_table, column_override) conn = self.connections.get(model_name) client = conn.handle - table = self.connections.table_ref(schema, table_name, conn) + table = self.connections.table_ref(schema, table_name, conn, + project=database) load_config = google.cloud.bigquery.LoadJobConfig() load_config.skip_leading_rows = 1 @@ -468,12 +470,10 @@ def get_catalog(self, manifest): connection = self.connections.get('catalog') client = connection.handle - schemas = { - node.to_dict()['schema'] - for node in manifest.nodes.values() - } + schemas = manifest.get_used_schemas() column_names = ( + 'table_database', 'table_schema', 'table_name', 'table_type', @@ -488,8 +488,8 @@ def get_catalog(self, manifest): all_names = column_names + self._get_stats_column_names() columns = [] - for schema_name in schemas: - relations = self.list_relations(schema_name) + for database_name, schema_name in schemas: + relations = self.list_relations(database_name, schema_name) for relation in relations: # This relation contains a subset of the info we care about. @@ -497,7 +497,8 @@ def get_catalog(self, manifest): table_ref = self.connections.table_ref( relation.schema, relation.identifier, - connection + connection, + project=database_name ) table = client.get_table(table_ref) @@ -507,6 +508,7 @@ def get_catalog(self, manifest): for index, column in enumerate(flattened, start=1): column_data = ( + relation.database, relation.schema, relation.name, relation.type, diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py index 077444a4d..a489512d6 100644 --- a/dbt/adapters/bigquery/relation.py +++ b/dbt/adapters/bigquery/relation.py @@ -11,37 +11,17 @@ class BigQueryRelation(BaseRelation): }, 'quote_character': '`', 'quote_policy': { - 'project': True, + 'database': True, 'schema': True, 'identifier': True }, 'include_policy': { - 'project': True, + 'database': True, 'schema': True, 'identifier': True } } - PATH_SCHEMA = { - 'type': 'object', - 'properties': { - 'project': {'type': ['string', 'null']}, - 'schema': {'type': ['string', 'null']}, - 'identifier': {'type': 'string'}, - }, - 'required': ['project', 'schema', 'identifier'], - } - - POLICY_SCHEMA = { - 'type': 'object', - 'properties': { - 'project': {'type': 'boolean'}, - 'schema': {'type': 'boolean'}, - 'identifier': {'type': 'boolean'}, - }, - 'required': ['project', 'schema', 'identifier'], - } - SCHEMA = { 'type': 'object', 'properties': { @@ -57,20 +37,18 @@ class BigQueryRelation(BaseRelation): 'type': { 'enum': BaseRelation.RelationTypes + [External, None], }, - 'path': PATH_SCHEMA, - 'include_policy': POLICY_SCHEMA, - 'quote_policy': POLICY_SCHEMA, + 'path': BaseRelation.PATH_SCHEMA, + 'include_policy': BaseRelation.POLICY_SCHEMA, + 'quote_policy': BaseRelation.POLICY_SCHEMA, 'quote_character': {'type': 'string'}, }, 'required': ['metadata', 'type', 'path', 'include_policy', 'quote_policy', 'quote_character'] } - PATH_ELEMENTS = ['project', 'schema', 'identifier'] - - def matches(self, project=None, schema=None, identifier=None): + def matches(self, database=None, schema=None, identifier=None): search = filter_null_values({ - 'project': project, + 'database': database, 'schema': schema, 'identifier': identifier }) @@ -86,15 +64,7 @@ def matches(self, project=None, schema=None, identifier=None): return True @classmethod - def _create_from_node(cls, config, node, **kwargs): - return cls.create( - project=config.credentials.project, - schema=node.get('schema'), - identifier=node.get('alias'), - **kwargs) - - @classmethod - def create(cls, project=None, schema=None, + def create(cls, database=None, schema=None, identifier=None, table_name=None, type=None, **kwargs): if table_name is None: @@ -102,34 +72,29 @@ def create(cls, project=None, schema=None, return cls(type=type, path={ - 'project': project, + 'database': database, 'schema': schema, 'identifier': identifier }, table_name=table_name, **kwargs) - def quote(self, project=None, schema=None, identifier=None): + def quote(self, database=None, schema=None, identifier=None): policy = filter_null_values({ - 'project': project, + 'database': database, 'schema': schema, 'identifier': identifier }) return self.incorporate(quote_policy=policy) - def include(self, project=None, schema=None, identifier=None): - policy = filter_null_values({ - 'project': project, - 'schema': schema, - 'identifier': identifier - }) - - return self.incorporate(include_policy=policy) + @property + def database(self): + return self.path.get('database') @property def project(self): - return self.path.get('project') + return self.path.get('database') @property def schema(self): diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index d43b5f7a6..d561ea487 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -13,9 +13,9 @@ {%- set identifier = model['alias'] -%} - {%- set old_relation = adapter.get_relation(schema=schema, identifier=identifier) -%} + {%- set old_relation = adapter.get_relation(database=database, schema=schema, identifier=identifier) -%} - {%- set target_relation = api.Relation.create(identifier=identifier, schema=schema, type='table') -%} + {%- set target_relation = api.Relation.create(database=database, identifier=identifier, schema=schema, type='table') -%} {%- set exists_as_table = (old_relation is not none and old_relation.is_table) -%} {%- set exists_not_as_table = (old_relation is not none and not old_relation.is_table) -%} @@ -52,7 +52,7 @@ {{ create_table_as(False, target_relation, sql) }} {%- endcall -%} {%- else -%} - {% set dest_columns = adapter.get_columns_in_table(schema, identifier) %} + {% set dest_columns = adapter.get_columns_in_relation(target_relation) %} {%- call statement('main') -%} {{ get_merge_sql(target_relation, source_sql, unique_key, dest_columns) }} {% endcall %} diff --git a/dbt/include/bigquery/macros/materializations/seed.sql b/dbt/include/bigquery/macros/materializations/seed.sql index 11f67a9db..17e1bd989 100644 --- a/dbt/include/bigquery/macros/materializations/seed.sql +++ b/dbt/include/bigquery/macros/materializations/seed.sql @@ -10,6 +10,6 @@ {% macro bigquery__load_csv_rows(model) %} {%- set column_override = model['config'].get('column_types', {}) -%} - {{ adapter.load_dataframe(model['schema'], model['alias'], model['agate_table'], column_override) }} + {{ adapter.load_dataframe(model['database'], model['schema'], model['alias'], model['agate_table'], column_override) }} {% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql index 92f54d30c..87c00cf64 100644 --- a/dbt/include/bigquery/macros/materializations/table.sql +++ b/dbt/include/bigquery/macros/materializations/table.sql @@ -30,9 +30,9 @@ {%- set identifier = model['alias'] -%} {%- set non_destructive_mode = (flags.NON_DESTRUCTIVE == True) -%} - {%- set old_relation = adapter.get_relation(schema=schema, identifier=identifier) -%} + {%- set old_relation = adapter.get_relation(database=database, schema=schema, identifier=identifier) -%} {%- set exists_not_as_table = (old_relation is not none and not old_relation.is_table) -%} - {%- set target_relation = api.Relation.create(schema=schema, identifier=identifier, type='table') -%} + {%- set target_relation = api.Relation.create(database=database, schema=schema, identifier=identifier, type='table') -%} {%- set verbose = config.get('verbose', False) -%} {# partitions: iterate over each partition, running a separate query in a for-loop #} diff --git a/setup.py b/setup.py index 5c743575d..460cd9ab0 100644 --- a/setup.py +++ b/setup.py @@ -18,6 +18,7 @@ packages=find_packages(), package_data={ 'dbt': [ + 'include/bigquery/dbt_project.yml', 'include/bigquery/macros/*.sql', 'include/bigquery/macros/**/*.sql', ] From bcefbf199aa2dfb3eeaebaaf97536bb9d2ce442a Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 3 Jan 2019 10:58:04 -0700 Subject: [PATCH 004/860] PR feedback bigquery: naming/parameter sanity cleanup postgres: never allow databases that aren't the default postgres: simplify cache buliding since we know we'll only ever have one database everything: parameter name change for execute_macro everything: cache related bugfixes to casing internal only: cross db/cross schema rename support in the cache - none of the adapters support it, but unit tests expose the behavior tests: much more comprehensive cache tests --- dbt/adapters/bigquery/connections.py | 40 +++++++++---------- dbt/adapters/bigquery/impl.py | 38 ++++++++++-------- .../macros/materializations/table.sql | 2 +- 3 files changed, 42 insertions(+), 38 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 95c3215ec..fbe0448ad 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -132,10 +132,10 @@ def get_bigquery_credentials(cls, profile_credentials): @classmethod def get_bigquery_client(cls, profile_credentials): - project_name = profile_credentials.database + database = profile_credentials.database creds = cls.get_bigquery_credentials(profile_credentials) location = getattr(profile_credentials, 'location', None) - return google.cloud.bigquery.Client(project_name, creds, + return google.cloud.bigquery.Client(database, creds, location=location) @classmethod @@ -207,7 +207,7 @@ def execute(self, sql, name=None, auto_begin=False, fetch=None): status = 'OK' return status, res - def create_bigquery_table(self, dataset_name, table_name, conn_name, + def create_bigquery_table(self, database, schema, table_name, conn_name, callback, sql): """Create a bigquery table. The caller must supply a callback that takes one argument, a `google.cloud.bigquery.Table`, and mutates @@ -216,26 +216,26 @@ def create_bigquery_table(self, dataset_name, table_name, conn_name, conn = self.get(conn_name) client = conn.handle - view_ref = self.table_ref(dataset_name, table_name, conn) + view_ref = self.table_ref(database, schema, table_name, conn) view = google.cloud.bigquery.Table(view_ref) callback(view) with self.exception_handler(sql, conn.name): client.create_table(view) - def create_view(self, dataset_name, table_name, conn_name, sql): + def create_view(self, database, schema, table_name, conn_name, sql): def callback(table): table.view_query = sql table.view_use_legacy_sql = False - self.create_bigquery_table(dataset_name, table_name, conn_name, + self.create_bigquery_table(database, schema, table_name, conn_name, callback, sql) - def create_table(self, dataset_name, table_name, conn_name, sql): + def create_table(self, database, schema, table_name, conn_name, sql): conn = self.get(conn_name) client = conn.handle - table_ref = self.table_ref(dataset_name, table_name, conn) + table_ref = self.table_ref(database, schema, table_name, conn) job_config = google.cloud.bigquery.QueryJobConfig() job_config.destination = table_ref job_config.write_disposition = 'WRITE_TRUNCATE' @@ -246,32 +246,32 @@ def create_table(self, dataset_name, table_name, conn_name, sql): with self.exception_handler(sql, conn_name): query_job.result(timeout=self.get_timeout(conn)) - def create_date_partitioned_table(self, dataset_name, table_name, + def create_date_partitioned_table(self, database, schema, table_name, conn_name): def callback(table): table.partitioning_type = 'DAY' - self.create_bigquery_table(dataset_name, table_name, conn_name, + self.create_bigquery_table(database, schema, table_name, conn_name, callback, 'CREATE DAY PARTITIONED TABLE') @staticmethod - def dataset(dataset_name, conn, project=None): - dataset_ref = conn.handle.dataset(dataset_name, project) + def dataset(database, schema, conn): + dataset_ref = conn.handle.dataset(schema, database) return google.cloud.bigquery.Dataset(dataset_ref) - def table_ref(self, dataset_name, table_name, conn, project=None): - dataset = self.dataset(dataset_name, conn, project=project) + def table_ref(self, database, schema, table_name, conn): + dataset = self.dataset(database, schema, conn) return dataset.table(table_name) - def get_bq_table(self, schema, identifier, conn_name=None, database=None): + def get_bq_table(self, database, schema, identifier, conn_name=None): """Get a bigquery table for a schema/model.""" conn = self.get(conn_name) - table_ref = self.table_ref(schema, identifier, conn, project=database) + table_ref = self.table_ref(database, schema, identifier, conn) return conn.handle.get_table(table_ref) - def drop_dataset(self, project_name, dataset_name, conn_name=None): + def drop_dataset(self, database, schema, conn_name=None): conn = self.get(conn_name) - dataset = self.dataset(dataset_name, conn, project=project_name) + dataset = self.dataset(database, schema, conn) client = conn.handle with self.exception_handler('drop dataset', conn.name): @@ -279,10 +279,10 @@ def drop_dataset(self, project_name, dataset_name, conn_name=None): client.delete_table(table.reference) client.delete_dataset(dataset) - def create_dataset(self, project_name, dataset_name, conn_name=None): + def create_dataset(self, database, schema, conn_name=None): conn = self.get(conn_name) client = conn.handle - dataset = self.dataset(dataset_name, conn, project=project_name) + dataset = self.dataset(database, schema, conn) # Emulate 'create schema if not exists ...' try: diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 8cd281ed5..dab04bff3 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -72,7 +72,8 @@ def drop_relation(self, relation, model_name=None): conn = self.connections.get(model_name) client = conn.handle - dataset = self.connections.dataset(relation.schema, conn) + dataset = self.connections.dataset(relation.database, relation.schema, + conn) relation_object = dataset.table(relation.identifier) client.delete_table(relation_object) @@ -118,8 +119,8 @@ def list_relations_without_caching(self, database, schema, connection = self.connections.get(model_name) client = connection.handle - bigquery_dataset = self.connections.dataset(schema, connection, - project=database) + bigquery_dataset = self.connections.dataset(database, schema, + connection) all_tables = client.list_tables( bigquery_dataset, @@ -152,8 +153,7 @@ def get_relation(self, database, schema, identifier, model_name=None): ) try: - table = self.connections.get_bq_table(schema, identifier, - database=database) + table = self.connections.get_bq_table(database, schema, identifier) except google.api_core.exceptions.NotFound: table = None return self._bq_table_to_relation(table) @@ -227,6 +227,7 @@ def _agate_to_schema(self, agate_table, column_override): return bq_schema def _materialize_as_view(self, model): + model_database = model.get('database') model_schema = model.get('schema') model_name = model.get('name') model_alias = model.get('alias') @@ -234,7 +235,8 @@ def _materialize_as_view(self, model): logger.debug("Model SQL ({}):\n{}".format(model_name, model_sql)) self.connections.create_view( - dataset_name=model_schema, + database=model_database, + schema=model_schema, table_name=model_alias, conn_name=model_name, sql=model_sql @@ -242,6 +244,7 @@ def _materialize_as_view(self, model): return "CREATE VIEW" def _materialize_as_table(self, model, model_sql, decorator=None): + model_database = model.get('database') model_schema = model.get('schema') model_name = model.get('name') model_alias = model.get('alias') @@ -253,7 +256,8 @@ def _materialize_as_table(self, model, model_sql, decorator=None): logger.debug("Model SQL ({}):\n{}".format(table_name, model_sql)) self.connections.create_table( - dataset_name=model_schema, + database=model_database, + schema=model_schema, conn_name=model_name, table_name=table_name, sql=model_sql @@ -312,10 +316,11 @@ def add_query(self, sql, model_name=None, auto_begin=True, # Special bigquery adapter methods ### @available - def make_date_partitioned_table(self, schema, identifier, model_name=None): + def make_date_partitioned_table(self, relation, model_name=None): return self.connections.create_date_partitioned_table( - dataset_name=schema, - table_name=identifier, + database=relation.database, + schema=relation.schema, + table_name=relation.identifier, conn_name=model_name ) @@ -366,9 +371,9 @@ def alter_table_add_columns(self, relation, columns, model_name=None): conn = self.connections.get(model_name) client = conn.handle - table_ref = self.connections.table_ref(relation.schema, - relation.identifier, conn, - project=relation.database) + table_ref = self.connections.table_ref(relation.database, + relation.schema, + relation.identifier, conn) table = client.get_table(table_ref) new_columns = [column_to_bq_schema(col) for col in columns] @@ -384,8 +389,7 @@ def load_dataframe(self, database, schema, table_name, agate_table, conn = self.connections.get(model_name) client = conn.handle - table = self.connections.table_ref(schema, table_name, conn, - project=database) + table = self.connections.table_ref(database, schema, table_name, conn) load_config = google.cloud.bigquery.LoadJobConfig() load_config.skip_leading_rows = 1 @@ -495,10 +499,10 @@ def get_catalog(self, manifest): # This relation contains a subset of the info we care about. # Fetch the full table object here table_ref = self.connections.table_ref( + database_name, relation.schema, relation.identifier, - connection, - project=database_name + connection ) table = client.get_table(table_ref) diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql index 87c00cf64..1fb13779b 100644 --- a/dbt/include/bigquery/macros/materializations/table.sql +++ b/dbt/include/bigquery/macros/materializations/table.sql @@ -1,7 +1,7 @@ {% macro make_date_partitioned_table(model, relation, dates, should_create, verbose=False) %} {% if should_create %} - {{ adapter.make_date_partitioned_table(relation.schema, relation.identifier) }} + {{ adapter.make_date_partitioned_table(relation) }} {% endif %} {% for date in dates %} From 0d2e698852f4651f78e26c80dffcd77facd4a8a6 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 3 Jan 2019 13:43:08 -0700 Subject: [PATCH 005/860] Move SQL previously embedded into adapters into macros Adapters now store an internal manifest that only has the dbt internal projects Adapters use that manifest if none is provided to execute_manifest The internal manifest is lazy-loaded to avoid recursion issues Moved declared plugin paths down one level Connection management changes to accomadate calling macro -> adapter -> macro Split up precision and scale when describing number columns so agate doesn't eat commas Manifest building now happens in the RunManager instead of the compiler Now macros: create/drop schema get_columns_in_relation alter column type rename/drop/truncate list_schemas/check_schema_exists list_relations_without_caching --- dbt/include/bigquery/__init__.py | 2 +- dbt/include/bigquery/macros/adapters.sql | 19 ++++++++++++++++++- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/dbt/include/bigquery/__init__.py b/dbt/include/bigquery/__init__.py index 87098354a..564a3d1e8 100644 --- a/dbt/include/bigquery/__init__.py +++ b/dbt/include/bigquery/__init__.py @@ -1,2 +1,2 @@ import os -PACKAGE_PATH = os.path.dirname(os.path.dirname(__file__)) +PACKAGE_PATH = os.path.dirname(__file__) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index b223b6d4e..83b97f708 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -13,7 +13,7 @@ {% macro cluster_by(raw_cluster_by) %} {%- if raw_cluster_by is not none -%} - cluster by + cluster by {% if raw_cluster_by is string -%} {% set raw_cluster_by = [raw_cluster_by] %} {%- endif -%} @@ -45,3 +45,20 @@ {{ sql }} ); {% endmacro %} + +{% macro bigquery__create_schema(database_name, schema_name) -%} + {{ adapter.create_schema(database_name, schema_name) }} +{% endmacro %} + +{% macro bigquery__drop_schema(database_name, schema_name) -%} + {{ adapter.drop_schema(database_name, schema_name) }} +{% endmacro %} + +{% macro bigquery__get_columns_in_relation(relation) -%} + {{ return(adapter.get_columns_in_relation(relation)) }} +{% endmacro %} + + +{% macro bigquery__list_relations_without_caching(database, schema) -%} + {{ return(adapter.list_relations_without_caching(database, schema)) }} +{% endmacro %} From 7a2dfe1a64257198b8c44daee04efb3fbf2184ed Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 7 Feb 2019 13:18:03 -0700 Subject: [PATCH 006/860] fix bigquery Remove unnecessary is_incremental() from test Fix strict mode type check --- dbt/adapters/bigquery/impl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index dab04bff3..f42f58303 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -333,7 +333,7 @@ def execute_model(self, model, materialization, sql_override=None, if flags.STRICT_MODE: connection = self.connections.get(model.get('name')) - Connection(**connection) + assert isinstance(connection, Connection) if materialization == 'view': res = self._materialize_as_view(model) From 51601e10cc0f64df531a6c93d4296278a51c8d7b Mon Sep 17 00:00:00 2001 From: Darren Haken Date: Sun, 10 Feb 2019 13:38:40 +0000 Subject: [PATCH 007/860] #1031 Add macro to add table description from schema.yml for BQ --- dbt/include/bigquery/macros/adapters.sql | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 83b97f708..3483c9cfd 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -26,6 +26,20 @@ {%- endmacro -%} +{% macro table_options() %} + {%- set raw_persist_docs = config.get('persist_docs', {}) -%} + {%- set raw_persist_relation_docs = raw_persist_docs.get('relation', false) -%} + + {%- if raw_persist_docs is {} -%} + {{ return('') }} + {% endif %} + + OPTIONS( + {% if raw_persist_relation_docs -%} + description={{ model.description }} + {% endif %} + ) +{%- endmacro -%} {% macro bigquery__create_table_as(temporary, relation, sql) -%} {%- set raw_partition_by = config.get('partition_by', none) -%} @@ -34,6 +48,7 @@ create or replace table {{ relation }} {{ partition_by(raw_partition_by) }} {{ cluster_by(raw_cluster_by) }} + {{ table_docs() }} as ( {{ sql }} ); From 94b9a4b09847ef4a639ae0ffce93e11aa671d9cb Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Mon, 11 Feb 2019 13:04:58 -0500 Subject: [PATCH 008/860] use frozenset for adapter specific configs, add bq configs --- dbt/adapters/bigquery/impl.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index dab04bff3..e4e7778a9 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -51,6 +51,8 @@ class BigQueryAdapter(BaseAdapter): Column = dbt.schema.BigQueryColumn ConnectionManager = BigQueryConnectionManager + AdapterSpecificConfigs = frozenset({"cluster_by", "partition_by"}) + ### # Implementations of abstract methods ### From 82d508c7c7f86ab3863afc9495fee1eabacefdcb Mon Sep 17 00:00:00 2001 From: Darren Haken Date: Tue, 12 Feb 2019 15:21:29 +0000 Subject: [PATCH 009/860] Make changes as per review from @drewbanin --- dbt/include/bigquery/macros/adapters.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 3483c9cfd..ab25e3ffc 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -36,7 +36,7 @@ OPTIONS( {% if raw_persist_relation_docs -%} - description={{ model.description }} + description={{ model.description | tojson }} {% endif %} ) {%- endmacro -%} @@ -48,7 +48,7 @@ create or replace table {{ relation }} {{ partition_by(raw_partition_by) }} {{ cluster_by(raw_cluster_by) }} - {{ table_docs() }} + {{ table_options() }} as ( {{ sql }} ); From f8ccb1fe155ac00002fb26659aa9aa3c1491e773 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 1 Feb 2019 13:57:30 -0700 Subject: [PATCH 010/860] Remove RunManager Move some of RunManager into tasks Move compile_node work into compilation Move manifest work into the GraphLoader Move the rest into the runners Implement freshness calculations for sources command: 'dbt source snapshot-freshness' support for 4 adapters (no presto) Integration tests break up main.py's argument parsing Pass the manifest along to freshness calculation Results support for freshness New freshness result contracts Fix source result printing Result contract cleanup safe_run supports alternate result types Fix tests to support changes in results PR feedback: - snowflake macro changed to always return utc - no cte in collect_freshness - remove extra optional arg - fix the has_freshness check to examine if there is anything in freshness - support error_after without warn_after and vice-versa - snowflake: convert_timestamp -> convert_timezone Update sources to be Relations - update contracts - add create_from_source - add create_from_source calls - fix tests PR feedback create_from_source forces quotes default source schema/table from source/table names snowflake quoting nonsense also fix output: pass -> PASS make seeding test 017 take 1m instead of 3m by using csv instead of sql - source tweaks for the docs site --- dbt/include/bigquery/macros/adapters.sql | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 83b97f708..d9700f359 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -62,3 +62,8 @@ {% macro bigquery__list_relations_without_caching(database, schema) -%} {{ return(adapter.list_relations_without_caching(database, schema)) }} {% endmacro %} + + +{% macro bigquery__current_timestamp() -%} + CURRENT_TIMESTAMP() +{%- endmacro %} From a8053b0323422fe487b5e51c7d2f1c300f589024 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Thu, 14 Feb 2019 15:46:23 -0500 Subject: [PATCH 011/860] fix long_description field in all the setup.pys --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 460cd9ab0..d563e6b36 100644 --- a/setup.py +++ b/setup.py @@ -11,7 +11,7 @@ name=package_name, version=package_version, description=description, - long_description_content_type=description, + long_description=description, author="Fishtown Analytics", author_email="info@fishtownanalytics.com", url="https://github.com/fishtown-analytics/dbt", From 318ee33eea1919a005c57822772b8ca0466adc8e Mon Sep 17 00:00:00 2001 From: Darren Haken Date: Sat, 16 Feb 2019 15:21:43 +0000 Subject: [PATCH 012/860] Add description for BQ tables and view relations. Also make changes to table_options macro based on testing against a real project. --- dbt/include/bigquery/macros/adapters.sql | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index ab25e3ffc..c48db7a19 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -27,18 +27,16 @@ {%- endmacro -%} {% macro table_options() %} - {%- set raw_persist_docs = config.get('persist_docs', {}) -%} - {%- set raw_persist_relation_docs = raw_persist_docs.get('relation', false) -%} + {%- set raw_persist_docs = config.get('persist_docs', none) -%} - {%- if raw_persist_docs is {} -%} - {{ return('') }} - {% endif %} - - OPTIONS( - {% if raw_persist_relation_docs -%} + {%- if raw_persist_docs -%} + {%- set raw_relation = raw_persist_docs.get('relation', false) -%} + OPTIONS( + {%- if raw_relation -%} description={{ model.description | tojson }} - {% endif %} - ) + {% endif %} + ) + {% endif %} {%- endmacro -%} {% macro bigquery__create_table_as(temporary, relation, sql) -%} @@ -56,7 +54,9 @@ {% macro bigquery__create_view_as(relation, sql) -%} - create or replace view {{ relation }} as ( + create or replace view {{ relation }} + {{ table_options() }} + as ( {{ sql }} ); {% endmacro %} From 452281948227ab67e6594b2fbccb391824cc09fb Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Mon, 18 Feb 2019 16:02:37 -0500 Subject: [PATCH 013/860] =?UTF-8?q?Bump=20version:=200.13.0a1=20=E2=86=92?= =?UTF-8?q?=200.13.0a2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index d563e6b36..bc0551948 100644 --- a/setup.py +++ b/setup.py @@ -3,7 +3,7 @@ from distutils.core import setup package_name = "dbt-bigquery" -package_version = "0.13.0a1" +package_version = "0.13.0a2" description = """The bigquery adapter plugin for dbt (data build tool)""" From 2fb608deba8a75a18b22e113559d55db7a2b28ec Mon Sep 17 00:00:00 2001 From: Darren Haken Date: Tue, 19 Feb 2019 12:18:09 +0000 Subject: [PATCH 014/860] Add `persist_docs` to project level settings. Change `table_options` to have better error handling. --- dbt/include/bigquery/macros/adapters.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 0531772d8..f02dba249 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -27,15 +27,17 @@ {%- endmacro -%} {% macro table_options() %} - {%- set raw_persist_docs = config.get('persist_docs', none) -%} + {%- set raw_persist_docs = config.get('persist_docs', {}) -%} - {%- if raw_persist_docs -%} + {%- if raw_persist_docs is mapping -%} {%- set raw_relation = raw_persist_docs.get('relation', false) -%} OPTIONS( {%- if raw_relation -%} description={{ model.description | tojson }} {% endif %} ) + {%- else -%} + {{ exceptions.raise_compiler_error("Invalid value provided for 'persist_docs'. Expected dict but got value: " ~ raw_persist_docs) }} {% endif %} {%- endmacro -%} From e74d2dfa9dc4d23321b5d2caeb03dd36038d3ff9 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 5 Mar 2019 10:42:31 -0700 Subject: [PATCH 015/860] test fixes --- dbt/adapters/bigquery/impl.py | 7 ++++--- dbt/include/bigquery/macros/adapters.sql | 4 ++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index e503148be..ad5fbf724 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -116,13 +116,14 @@ def expand_column_types(self, goal, current, model_name=None): # This is a no-op on BigQuery pass - def list_relations_without_caching(self, database, schema, + def list_relations_without_caching(self, information_schema, schema, model_name=None): connection = self.connections.get(model_name) client = connection.handle - bigquery_dataset = self.connections.dataset(database, schema, - connection) + bigquery_dataset = self.connections.dataset( + information_schema.database, schema, connection + ) all_tables = client.list_tables( bigquery_dataset, diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index d9700f359..e14dd93a1 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -59,8 +59,8 @@ {% endmacro %} -{% macro bigquery__list_relations_without_caching(database, schema) -%} - {{ return(adapter.list_relations_without_caching(database, schema)) }} +{% macro bigquery__list_relations_without_caching(information_schema, schema) -%} + {{ return(adapter.list_relations_without_caching(information_schema, schema)) }} {% endmacro %} From 9e31bd65eecda8c5ca33a7647f12526907920baf Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 4 Mar 2019 21:08:38 -0700 Subject: [PATCH 016/860] per-thread connections parsing now always opens a connection, instead of waiting to need it remove model_name/available_raw/etc --- dbt/adapters/bigquery/connections.py | 65 +++++++++++---------- dbt/adapters/bigquery/impl.py | 86 +++++++++++++--------------- 2 files changed, 71 insertions(+), 80 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index fbe0448ad..84ed923d2 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -1,4 +1,3 @@ -import abc from contextlib import contextmanager import google.auth @@ -10,7 +9,6 @@ import dbt.clients.agate_helper import dbt.exceptions from dbt.adapters.base import BaseConnectionManager, Credentials -from dbt.compat import abstractclassmethod from dbt.logger import GLOBAL_LOGGER as logger @@ -77,8 +75,11 @@ def handle_error(cls, error, message, sql): raise dbt.exceptions.DatabaseException(error_msg) + def clear_transaction(self): + pass + @contextmanager - def exception_handler(self, sql, connection_name='master'): + def exception_handler(self, sql): try: yield @@ -104,10 +105,10 @@ def close(cls, connection): return connection - def begin(self, name): + def begin(self): pass - def commit(self, connection): + def commit(self): pass @classmethod @@ -178,25 +179,25 @@ def get_table_from_response(cls, resp): rows = [dict(row.items()) for row in resp] return dbt.clients.agate_helper.table_from_data(rows, column_names) - def raw_execute(self, sql, name=None, fetch=False): - conn = self.get(name) + def raw_execute(self, sql, fetch=False): + conn = self.get_thread_connection() client = conn.handle - logger.debug('On %s: %s', name, sql) + logger.debug('On %s: %s', conn.name, sql) job_config = google.cloud.bigquery.QueryJobConfig() job_config.use_legacy_sql = False query_job = client.query(sql, job_config) # this blocks until the query has completed - with self.exception_handler(sql, conn.name): + with self.exception_handler(sql): iterator = query_job.result() return query_job, iterator - def execute(self, sql, name=None, auto_begin=False, fetch=None): + def execute(self, sql, auto_begin=False, fetch=None): # auto_begin is ignored on bigquery, and only included for consistency - _, iterator = self.raw_execute(sql, name=name, fetch=fetch) + _, iterator = self.raw_execute(sql, fetch=fetch) if fetch: res = self.get_table_from_response(iterator) @@ -207,32 +208,31 @@ def execute(self, sql, name=None, auto_begin=False, fetch=None): status = 'OK' return status, res - def create_bigquery_table(self, database, schema, table_name, conn_name, - callback, sql): + def create_bigquery_table(self, database, schema, table_name, callback, + sql): """Create a bigquery table. The caller must supply a callback that takes one argument, a `google.cloud.bigquery.Table`, and mutates it. """ - conn = self.get(conn_name) + conn = self.get_thread_connection() client = conn.handle view_ref = self.table_ref(database, schema, table_name, conn) view = google.cloud.bigquery.Table(view_ref) callback(view) - with self.exception_handler(sql, conn.name): + with self.exception_handler(sql): client.create_table(view) - def create_view(self, database, schema, table_name, conn_name, sql): + def create_view(self, database, schema, table_name, sql): def callback(table): table.view_query = sql table.view_use_legacy_sql = False - self.create_bigquery_table(database, schema, table_name, conn_name, - callback, sql) + self.create_bigquery_table(database, schema, table_name, callback, sql) - def create_table(self, database, schema, table_name, conn_name, sql): - conn = self.get(conn_name) + def create_table(self, database, schema, table_name, sql): + conn = self.get_thread_connection() client = conn.handle table_ref = self.table_ref(database, schema, table_name, conn) @@ -243,16 +243,15 @@ def create_table(self, database, schema, table_name, conn_name, sql): query_job = client.query(sql, job_config=job_config) # this waits for the job to complete - with self.exception_handler(sql, conn_name): + with self.exception_handler(sql): query_job.result(timeout=self.get_timeout(conn)) - def create_date_partitioned_table(self, database, schema, table_name, - conn_name): + def create_date_partitioned_table(self, database, schema, table_name): def callback(table): table.partitioning_type = 'DAY' - self.create_bigquery_table(database, schema, table_name, conn_name, - callback, 'CREATE DAY PARTITIONED TABLE') + self.create_bigquery_table(database, schema, table_name, callback, + 'CREATE DAY PARTITIONED TABLE') @staticmethod def dataset(database, schema, conn): @@ -263,24 +262,24 @@ def table_ref(self, database, schema, table_name, conn): dataset = self.dataset(database, schema, conn) return dataset.table(table_name) - def get_bq_table(self, database, schema, identifier, conn_name=None): + def get_bq_table(self, database, schema, identifier): """Get a bigquery table for a schema/model.""" - conn = self.get(conn_name) + conn = self.get_thread_connection() table_ref = self.table_ref(database, schema, identifier, conn) return conn.handle.get_table(table_ref) - def drop_dataset(self, database, schema, conn_name=None): - conn = self.get(conn_name) + def drop_dataset(self, database, schema): + conn = self.get_thread_connection() dataset = self.dataset(database, schema, conn) client = conn.handle - with self.exception_handler('drop dataset', conn.name): + with self.exception_handler('drop dataset'): for table in client.list_tables(dataset): client.delete_table(table.reference) client.delete_dataset(dataset) - def create_dataset(self, database, schema, conn_name=None): - conn = self.get(conn_name) + def create_dataset(self, database, schema): + conn = self.get_thread_connection() client = conn.handle dataset = self.dataset(database, schema, conn) @@ -291,5 +290,5 @@ def create_dataset(self, database, schema, conn_name=None): except google.api_core.exceptions.NotFound: pass - with self.exception_handler('create dataset', conn.name): + with self.exception_handler('create dataset'): client.create_dataset(dataset) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index ad5fbf724..bb4c45f7f 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -65,13 +65,12 @@ def date_function(cls): def is_cancelable(cls): return False - def drop_relation(self, relation, model_name=None): - is_cached = self._schema_is_cached(relation.database, relation.schema, - model_name) + def drop_relation(self, relation): + is_cached = self._schema_is_cached(relation.database, relation.schema) if is_cached: self.cache.drop(relation) - conn = self.connections.get(model_name) + conn = self.connections.get_thread_connection() client = conn.handle dataset = self.connections.dataset(relation.database, relation.schema, @@ -79,32 +78,31 @@ def drop_relation(self, relation, model_name=None): relation_object = dataset.table(relation.identifier) client.delete_table(relation_object) - def truncate_relation(self, relation, model_name=None): + def truncate_relation(self, relation): raise dbt.exceptions.NotImplementedException( '`truncate` is not implemented for this adapter!' ) - def rename_relation(self, from_relation, to_relation, model_name=None): + def rename_relation(self, from_relation, to_relation): raise dbt.exceptions.NotImplementedException( '`rename_relation` is not implemented for this adapter!' ) - def list_schemas(self, database, model_name=None): - conn = self.connections.get(model_name) + def list_schemas(self, database): + conn = self.connections.get_thread_connection() client = conn.handle - with self.connections.exception_handler('list dataset', conn.name): + with self.connections.exception_handler('list dataset'): all_datasets = client.list_datasets(project=database, include_all=True) return [ds.dataset_id for ds in all_datasets] - def get_columns_in_relation(self, relation, model_name=None): + def get_columns_in_relation(self, relation): try: table = self.connections.get_bq_table( database=relation.database, schema=relation.schema, - identifier=relation.table_name, - conn_name=model_name + identifier=relation.table_name ) return self._get_dbt_columns_from_bq_table(table) @@ -112,13 +110,12 @@ def get_columns_in_relation(self, relation, model_name=None): logger.debug("get_columns_in_relation error: {}".format(e)) return [] - def expand_column_types(self, goal, current, model_name=None): + def expand_column_types(self, goal, current): # This is a no-op on BigQuery pass - def list_relations_without_caching(self, information_schema, schema, - model_name=None): - connection = self.connections.get(model_name) + def list_relations_without_caching(self, information_schema, schema): + connection = self.connections.get_thread_connection() client = connection.handle bigquery_dataset = self.connections.dataset( @@ -144,15 +141,14 @@ def list_relations_without_caching(self, information_schema, schema, except google.api_core.exceptions.NotFound as e: return [] - def get_relation(self, database, schema, identifier, model_name=None): - if self._schema_is_cached(database, schema, model_name): + def get_relation(self, database, schema, identifier): + if self._schema_is_cached(database, schema): # if it's in the cache, use the parent's model of going through # the relations cache and picking out the relation return super(BigQueryAdapter, self).get_relation( database=database, schema=schema, - identifier=identifier, - model_name=model_name + identifier=identifier ) try: @@ -161,16 +157,16 @@ def get_relation(self, database, schema, identifier, model_name=None): table = None return self._bq_table_to_relation(table) - def create_schema(self, database, schema, model_name=None): + def create_schema(self, database, schema): logger.debug('Creating schema "%s.%s".', database, schema) - self.connections.create_dataset(database, schema, model_name) + self.connections.create_dataset(database, schema) - def drop_schema(self, database, schema, model_name=None): + def drop_schema(self, database, schema): logger.debug('Dropping schema "%s.%s".', database, schema) - if not self.check_schema_exists(database, schema, model_name): + if not self.check_schema_exists(database, schema): return - self.connections.drop_dataset(database, schema, model_name) + self.connections.drop_dataset(database, schema) @classmethod def quote(cls, identifier): @@ -232,16 +228,14 @@ def _agate_to_schema(self, agate_table, column_override): def _materialize_as_view(self, model): model_database = model.get('database') model_schema = model.get('schema') - model_name = model.get('name') model_alias = model.get('alias') model_sql = model.get('injected_sql') - logger.debug("Model SQL ({}):\n{}".format(model_name, model_sql)) + logger.debug("Model SQL ({}):\n{}".format(model_alias, model_sql)) self.connections.create_view( database=model_database, schema=model_schema, table_name=model_alias, - conn_name=model_name, sql=model_sql ) return "CREATE VIEW" @@ -249,7 +243,6 @@ def _materialize_as_view(self, model): def _materialize_as_table(self, model, model_sql, decorator=None): model_database = model.get('database') model_schema = model.get('schema') - model_name = model.get('name') model_alias = model.get('alias') if decorator is None: @@ -261,7 +254,6 @@ def _materialize_as_table(self, model, model_sql, decorator=None): self.connections.create_table( database=model_database, schema=model_schema, - conn_name=model_name, table_name=table_name, sql=model_sql ) @@ -307,10 +299,10 @@ def warning_on_hooks(hook_type): dbt.ui.printer.COLOR_FG_YELLOW) @available - def add_query(self, sql, model_name=None, auto_begin=True, - bindings=None, abridge_sql_log=False): - if model_name in ['on-run-start', 'on-run-end']: - self.warning_on_hooks(model_name) + def add_query(self, sql, auto_begin=True, bindings=None, + abridge_sql_log=False): + if self.nice_connection_name() in ['on-run-start', 'on-run-end']: + self.warning_on_hooks(self.nice_connection_name()) else: raise dbt.exceptions.NotImplementedException( '`add_query` is not implemented for this adapter!') @@ -319,24 +311,24 @@ def add_query(self, sql, model_name=None, auto_begin=True, # Special bigquery adapter methods ### @available - def make_date_partitioned_table(self, relation, model_name=None): + def make_date_partitioned_table(self, relation): return self.connections.create_date_partitioned_table( database=relation.database, schema=relation.schema, - table_name=relation.identifier, - conn_name=model_name + table_name=relation.identifier ) @available def execute_model(self, model, materialization, sql_override=None, - decorator=None, model_name=None): + decorator=None): if sql_override is None: sql_override = model.get('injected_sql') if flags.STRICT_MODE: - connection = self.connections.get(model.get('name')) + connection = self.connections.get_thread_connection() assert isinstance(connection, Connection) + assert(connection.name == model.get('name')) if materialization == 'view': res = self._materialize_as_view(model) @@ -349,10 +341,10 @@ def execute_model(self, model, materialization, sql_override=None, return res @available - def create_temporary_table(self, sql, model_name=None, **kwargs): + def create_temporary_table(self, sql, **kwargs): # BQ queries always return a temp table with their results - query_job, _ = self.connections.raw_execute(sql, model_name) + query_job, _ = self.connections.raw_execute(sql) bq_table = query_job.destination return self.Relation.create( @@ -366,12 +358,12 @@ def create_temporary_table(self, sql, model_name=None, **kwargs): type=BigQueryRelation.Table) @available - def alter_table_add_columns(self, relation, columns, model_name=None): + def alter_table_add_columns(self, relation, columns): logger.debug('Adding columns ({}) to table {}".'.format( columns, relation)) - conn = self.connections.get(model_name) + conn = self.connections.get_thread_connection() client = conn.handle table_ref = self.connections.table_ref(relation.database, @@ -387,9 +379,9 @@ def alter_table_add_columns(self, relation, columns, model_name=None): @available def load_dataframe(self, database, schema, table_name, agate_table, - column_override, model_name=None): + column_override): bq_schema = self._agate_to_schema(agate_table, column_override) - conn = self.connections.get(model_name) + conn = self.connections.get_thread_connection() client = conn.handle table = self.connections.table_ref(database, schema, table_name, conn) @@ -403,7 +395,7 @@ def load_dataframe(self, database, schema, table_name, agate_table, job_config=load_config) timeout = self.connections.get_timeout(conn) - with self.connections.exception_handler("LOAD TABLE", conn.name): + with self.connections.exception_handler("LOAD TABLE"): self.poll_until_job_completes(job, timeout) ### @@ -474,7 +466,7 @@ def _get_stats_columns(cls, table, relation_type): return zip(column_names, column_values) def get_catalog(self, manifest): - connection = self.connections.get('catalog') + connection = self.connections.get_thread_connection() client = connection.handle schemas = manifest.get_used_schemas() From 3441d6f4acc4ddd1fcf1ffee88ca8dd7c71329e7 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Wed, 6 Mar 2019 17:44:57 -0500 Subject: [PATCH 017/860] =?UTF-8?q?Bump=20version:=200.13.0a2=20=E2=86=92?= =?UTF-8?q?=200.13.0rc1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index bc0551948..198aeae62 100644 --- a/setup.py +++ b/setup.py @@ -3,7 +3,7 @@ from distutils.core import setup package_name = "dbt-bigquery" -package_version = "0.13.0a2" +package_version = "0.13.0rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" From 8f4273d389f999bc8b3495ff5344bda63ff8284a Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Thu, 21 Mar 2019 11:05:28 -0400 Subject: [PATCH 018/860] =?UTF-8?q?Bump=20version:=200.13.0rc1=20=E2=86=92?= =?UTF-8?q?=200.13.0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 198aeae62..1d21eba4e 100644 --- a/setup.py +++ b/setup.py @@ -3,7 +3,7 @@ from distutils.core import setup package_name = "dbt-bigquery" -package_version = "0.13.0rc1" +package_version = "0.13.0" description = """The bigquery adapter plugin for dbt (data build tool)""" From a5fb7583773d0ab13600b69b60d3072caa520f72 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 27 Mar 2019 12:44:04 -0600 Subject: [PATCH 019/860] when a dbt RuntimeException is raised inside the exception handler, re-raise it instead of wrapping it --- dbt/adapters/bigquery/connections.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 84ed923d2..5f8a125ce 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -94,6 +94,11 @@ def exception_handler(self, sql): except Exception as e: logger.debug("Unhandled error while running:\n{}".format(sql)) logger.debug(e) + if isinstance(e, dbt.exceptions.RuntimeException): + # during a sql query, an internal to dbt exception was raised. + # this sounds a lot like a signal handler and probably has + # useful information, so raise it without modification. + raise raise dbt.exceptions.RuntimeException(dbt.compat.to_string(e)) def cancel_open(self): From 03c147c1a89f25bdd53bf7815f258fcb48397a44 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 3 Apr 2019 19:13:22 -0600 Subject: [PATCH 020/860] pass the model name along in get_relations --- dbt/adapters/bigquery/impl.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index ad5fbf724..c36337f45 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -156,7 +156,8 @@ def get_relation(self, database, schema, identifier, model_name=None): ) try: - table = self.connections.get_bq_table(database, schema, identifier) + table = self.connections.get_bq_table(database, schema, identifier, + conn_name=model_name) except google.api_core.exceptions.NotFound: table = None return self._bq_table_to_relation(table) From b167d70483ea55ad198784b6b38a55d02387bd9f Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 17 Apr 2019 14:18:42 -0600 Subject: [PATCH 021/860] Remove sql_where, removing an unnecessary subquery in the process --- .../bigquery/macros/materializations/incremental.sql | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index d561ea487..98c65a1dd 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -2,7 +2,6 @@ {% materialization incremental, adapter='bigquery' -%} {%- set unique_key = config.get('unique_key') -%} - {%- set sql_where = config.get('sql_where') -%} {%- set non_destructive_mode = (flags.NON_DESTRUCTIVE == True) -%} {%- set full_refresh_mode = (flags.FULL_REFRESH == True) -%} @@ -34,12 +33,7 @@ {% set source_sql -%} {#-- wrap sql in parens to make it a subquery --#} ( - select * from ( - {{ sql }} - ) - {% if sql_where %} - where ({{ sql_where }}) or ({{ sql_where }}) is null - {% endif %} + {{ sql }} ) {%- endset -%} From bfc6bff433f020fddcc63d8eee0e2580c8ca556b Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 21 Mar 2019 08:42:40 -0600 Subject: [PATCH 022/860] flake8, pep8, unit tests --- dbt/adapters/bigquery/__init__.py | 4 ++-- dbt/adapters/bigquery/connections.py | 2 +- dbt/adapters/bigquery/impl.py | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/__init__.py b/dbt/adapters/bigquery/__init__.py index 5707c1690..196069ec9 100644 --- a/dbt/adapters/bigquery/__init__.py +++ b/dbt/adapters/bigquery/__init__.py @@ -1,6 +1,6 @@ -from dbt.adapters.bigquery.connections import BigQueryConnectionManager +from dbt.adapters.bigquery.connections import BigQueryConnectionManager # noqa from dbt.adapters.bigquery.connections import BigQueryCredentials -from dbt.adapters.bigquery.relation import BigQueryRelation +from dbt.adapters.bigquery.relation import BigQueryRelation # noqa from dbt.adapters.bigquery.impl import BigQueryAdapter from dbt.adapters.base import AdapterPlugin diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 5f8a125ce..24731c1a7 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -153,7 +153,7 @@ def open(cls, connection): try: handle = cls.get_bigquery_client(connection.credentials) - except google.auth.exceptions.DefaultCredentialsError as e: + except google.auth.exceptions.DefaultCredentialsError: logger.info("Please log into GCP to continue") dbt.clients.gcloud.setup_default_credentials() diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index bb4c45f7f..9f64d2a41 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -138,7 +138,7 @@ def list_relations_without_caching(self, information_schema, schema): # the implementation of list_relations for other adapters try: return [self._bq_table_to_relation(table) for table in all_tables] - except google.api_core.exceptions.NotFound as e: + except google.api_core.exceptions.NotFound: return [] def get_relation(self, database, schema, identifier): From 72a0433e1ebc86767b3e105d5127b17f30dd036c Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 25 Mar 2019 08:00:11 -0600 Subject: [PATCH 023/860] get archive blocks working tests fix event tracking test Fix print statements make archives not inherit configs from models archive now uses the name/alias properly for everything instead of target_table skip non-archive blocks in archive parsing instead of raising make archives ref-able - test for archive ref, test for archive selects raise a more useful message on incorrect archive targets add "--models" and "--exclude" arguments to archives - pass them through to selection - change get_fqn to take a full node object, have archives use that so selection behaves well - added tests Improve error handling on invalid archive configs Added a special archive-only node that has extra config restrictions add tests for invalid archive config --- dbt/include/bigquery/macros/materializations/archive.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/archive.sql b/dbt/include/bigquery/macros/materializations/archive.sql index 5548b71a7..551667649 100644 --- a/dbt/include/bigquery/macros/materializations/archive.sql +++ b/dbt/include/bigquery/macros/materializations/archive.sql @@ -16,8 +16,8 @@ {% macro bigquery__archive_update(target_relation, tmp_relation) %} update {{ target_relation }} as dest - set dest.{{ adapter.quote('valid_to') }} = tmp.{{ adapter.quote('valid_to') }} + set dest.{{ adapter.quote('dbt_valid_to') }} = tmp.{{ adapter.quote('dbt_valid_to') }} from {{ tmp_relation }} as tmp - where tmp.{{ adapter.quote('scd_id') }} = dest.{{ adapter.quote('scd_id') }} + where tmp.{{ adapter.quote('dbt_scd_id') }} = dest.{{ adapter.quote('dbt_scd_id') }} and {{ adapter.quote('change_type') }} = 'update'; {% endmacro %} From c15e4c4f1c236f083229db1f531812e9eaf21df4 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 10 Apr 2019 08:21:29 -0600 Subject: [PATCH 024/860] move column-related things into adapters where they belong --- dbt/adapters/bigquery/__init__.py | 1 + dbt/adapters/bigquery/impl.py | 19 +----- dbt/adapters/bigquery/relation.py | 102 +++++++++++++++++++++++++++++- 3 files changed, 105 insertions(+), 17 deletions(-) diff --git a/dbt/adapters/bigquery/__init__.py b/dbt/adapters/bigquery/__init__.py index 196069ec9..c45656772 100644 --- a/dbt/adapters/bigquery/__init__.py +++ b/dbt/adapters/bigquery/__init__.py @@ -1,6 +1,7 @@ from dbt.adapters.bigquery.connections import BigQueryConnectionManager # noqa from dbt.adapters.bigquery.connections import BigQueryCredentials from dbt.adapters.bigquery.relation import BigQueryRelation # noqa +from dbt.adapters.bigquery.relation import BigQueryColumn # noqa from dbt.adapters.bigquery.impl import BigQueryAdapter from dbt.adapters.base import AdapterPlugin diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 9f64d2a41..b904adf19 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -5,13 +5,13 @@ import dbt.compat import dbt.deprecations import dbt.exceptions -import dbt.schema import dbt.flags as flags import dbt.clients.gcloud import dbt.clients.agate_helper from dbt.adapters.base import BaseAdapter, available from dbt.adapters.bigquery import BigQueryRelation +from dbt.adapters.bigquery import BigQueryColumn from dbt.adapters.bigquery import BigQueryConnectionManager from dbt.contracts.connection import Connection from dbt.logger import GLOBAL_LOGGER as logger @@ -26,19 +26,6 @@ import agate -def column_to_bq_schema(col): - """Convert a column to a bigquery schema object. This is here instead of - in dbt.schema to avoid importing google libraries there. - """ - kwargs = {} - if len(col.fields) > 0: - fields = [column_to_bq_schema(field) for field in col.fields] - kwargs = {"fields": fields} - - return google.cloud.bigquery.SchemaField(col.name, col.dtype, col.mode, - **kwargs) - - class BigQueryAdapter(BaseAdapter): RELATION_TYPES = { @@ -48,7 +35,7 @@ class BigQueryAdapter(BaseAdapter): } Relation = BigQueryRelation - Column = dbt.schema.BigQueryColumn + Column = BigQueryColumn ConnectionManager = BigQueryConnectionManager AdapterSpecificConfigs = frozenset({"cluster_by", "partition_by"}) @@ -371,7 +358,7 @@ def alter_table_add_columns(self, relation, columns): relation.identifier, conn) table = client.get_table(table_ref) - new_columns = [column_to_bq_schema(col) for col in columns] + new_columns = [col.column_to_bq_schema() for col in columns] new_schema = table.schema + new_columns new_table = google.cloud.bigquery.Table(table_ref, schema=new_schema) diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py index a489512d6..e4f982b2c 100644 --- a/dbt/adapters/bigquery/relation.py +++ b/dbt/adapters/bigquery/relation.py @@ -1,6 +1,8 @@ -from dbt.adapters.base.relation import BaseRelation +from dbt.adapters.base.relation import BaseRelation, Column from dbt.utils import filter_null_values +import google.cloud.bigquery + class BigQueryRelation(BaseRelation): External = "external" @@ -107,3 +109,101 @@ def dataset(self): @property def identifier(self): return self.path.get('identifier') + + +class BigQueryColumn(Column): + TYPE_LABELS = { + 'STRING': 'STRING', + 'TIMESTAMP': 'TIMESTAMP', + 'FLOAT': 'FLOAT64', + 'INTEGER': 'INT64', + 'RECORD': 'RECORD', + } + + def __init__(self, column, dtype, fields=None, mode='NULLABLE'): + super(BigQueryColumn, self).__init__(column, dtype) + + if fields is None: + fields = [] + + self.fields = self.wrap_subfields(fields) + self.mode = mode + + @classmethod + def wrap_subfields(cls, fields): + return [BigQueryColumn.create_from_field(field) for field in fields] + + @classmethod + def create_from_field(cls, field): + return BigQueryColumn(field.name, cls.translate_type(field.field_type), + field.fields, field.mode) + + @classmethod + def _flatten_recursive(cls, col, prefix=None): + if prefix is None: + prefix = [] + + if len(col.fields) == 0: + prefixed_name = ".".join(prefix + [col.column]) + new_col = BigQueryColumn(prefixed_name, col.dtype, col.fields, + col.mode) + return [new_col] + + new_fields = [] + for field in col.fields: + new_prefix = prefix + [col.column] + new_fields.extend(cls._flatten_recursive(field, new_prefix)) + + return new_fields + + def flatten(self): + return self._flatten_recursive(self) + + @property + def quoted(self): + return '`{}`'.format(self.column) + + def literal(self, value): + return "cast({} as {})".format(value, self.dtype) + + @property + def data_type(self): + if self.dtype.upper() == 'RECORD': + subcols = [ + "{} {}".format(col.name, col.data_type) for col in self.fields + ] + field_type = 'STRUCT<{}>'.format(", ".join(subcols)) + + else: + field_type = self.dtype + + if self.mode.upper() == 'REPEATED': + return 'ARRAY<{}>'.format(field_type) + + else: + return field_type + + def is_string(self): + return self.dtype.lower() == 'string' + + def is_numeric(self): + return False + + def can_expand_to(self, other_column): + """returns True if both columns are strings""" + return self.is_string() and other_column.is_string() + + def __repr__(self): + return "".format(self.name, self.data_type, + self.mode) + + def column_to_bq_schema(self): + """Convert a column to a bigquery schema object. + """ + kwargs = {} + if len(self.fields) > 0: + fields = [field.column_to_bq_schema() for field in self.fields] + kwargs = {"fields": fields} + + return google.cloud.bigquery.SchemaField(self.name, self.dtype, + self.mode, **kwargs) From 87c36f87d54412054e775f2791ecb33330ccf87e Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 10 Apr 2019 10:07:32 -0600 Subject: [PATCH 025/860] Implmement check_cols Contracts: some anyOf shenanigans to add support for check_cols Macros: split apart archive selection, probably too much copy+paste Legacy: Archive configs now include a "timestamp" strategy when parsed from dbt_project.yml Add integration tests fix aliases test Unquote columns in archives handle null columns attr -> use_profile --- .../bigquery/macros/materializations/archive.sql | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/archive.sql b/dbt/include/bigquery/macros/materializations/archive.sql index 551667649..7a95f440f 100644 --- a/dbt/include/bigquery/macros/materializations/archive.sql +++ b/dbt/include/bigquery/macros/materializations/archive.sql @@ -4,11 +4,10 @@ {% endmacro %} -{% macro bigquery__archive_scd_hash() %} - to_hex(md5(concat(cast(`dbt_pk` as string), '|', cast(`dbt_updated_at` as string)))) +{% macro bigquery__archive_hash_arguments(args) %} + to_hex(md5(concat({% for arg in args %}coalesce(cast({{ arg }} as string), ''){% if not loop.last %}, '|',{% endif %}{% endfor %}))) {% endmacro %} - {% macro bigquery__create_columns(relation, columns) %} {{ adapter.alter_table_add_columns(relation, columns) }} {% endmacro %} @@ -16,8 +15,8 @@ {% macro bigquery__archive_update(target_relation, tmp_relation) %} update {{ target_relation }} as dest - set dest.{{ adapter.quote('dbt_valid_to') }} = tmp.{{ adapter.quote('dbt_valid_to') }} + set dest.dbt_valid_to = tmp.dbt_valid_to from {{ tmp_relation }} as tmp - where tmp.{{ adapter.quote('dbt_scd_id') }} = dest.{{ adapter.quote('dbt_scd_id') }} - and {{ adapter.quote('change_type') }} = 'update'; + where tmp.dbt_scd_id = dest.dbt_scd_id + and change_type = 'update'; {% endmacro %} From 581d72be97edb5fcc56650bd747269e3143ffdd9 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 12 Apr 2019 11:10:17 -0600 Subject: [PATCH 026/860] give bigquery list_schemas/check_schema_exist macros --- dbt/adapters/bigquery/impl.py | 6 ++++++ dbt/include/bigquery/macros/adapters.sql | 10 ++++++++++ 2 files changed, 16 insertions(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index b904adf19..012affa13 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -75,6 +75,7 @@ def rename_relation(self, from_relation, to_relation): '`rename_relation` is not implemented for this adapter!' ) + @available def list_schemas(self, database): conn = self.connections.get_thread_connection() client = conn.handle @@ -84,6 +85,11 @@ def list_schemas(self, database): include_all=True) return [ds.dataset_id for ds in all_datasets] + @available + def check_schema_exists(self, database, schema): + superself = super(BigQueryAdapter, self) + return superself.check_schema_exists(database, schema) + def get_columns_in_relation(self, relation): try: table = self.connections.get_bq_table( diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index e14dd93a1..71bb13778 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -67,3 +67,13 @@ {% macro bigquery__current_timestamp() -%} CURRENT_TIMESTAMP() {%- endmacro %} + + +{% macro bigquery__list_schemas(database) %} + {{ return(adapter.list_schemas()) }} +{% endmacro %} + + +{% macro bigquery__check_schema_exists(database, schema) %} + {{ return(adapter.check_schema_exists(database, schema)) }} +{% endmacro %} From c21e5d88ca8666578f3716c9b761a8b1c22fffe7 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 12 Apr 2019 11:10:28 -0600 Subject: [PATCH 027/860] if the schema exists, do not try to create it --- dbt/include/bigquery/macros/adapters.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 71bb13778..1c87ce4dc 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -74,6 +74,6 @@ {% endmacro %} -{% macro bigquery__check_schema_exists(database, schema) %} - {{ return(adapter.check_schema_exists(database, schema)) }} +{% macro bigquery__check_schema_exists(information_schema, schema) %} + {{ return(adapter.check_schema_exists(information_schema.database, schema)) }} {% endmacro %} From 9d30c9596a420adcea256c9b3717bd5fad8f00d5 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 24 Apr 2019 10:44:44 -0600 Subject: [PATCH 028/860] create a decorator for stubbing out methods at parse time Includes some unit tests Update integration tests to handle the fact that sometimes we now fail at runtime --- dbt/adapters/bigquery/impl.py | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index bb4c45f7f..610a3da4b 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -39,6 +39,16 @@ def column_to_bq_schema(col): **kwargs) +def _stub_relation(*args, **kwargs): + return BigQueryRelation.create( + database='', + schema='', + identifier='', + quote_policy={}, + type=BigQueryRelation.Table + ) + + class BigQueryAdapter(BaseAdapter): RELATION_TYPES = { @@ -310,7 +320,7 @@ def add_query(self, sql, auto_begin=True, bindings=None, ### # Special bigquery adapter methods ### - @available + @available.parse_none def make_date_partitioned_table(self, relation): return self.connections.create_date_partitioned_table( database=relation.database, @@ -318,7 +328,7 @@ def make_date_partitioned_table(self, relation): table_name=relation.identifier ) - @available + @available.parse(lambda *a, **k: '') def execute_model(self, model, materialization, sql_override=None, decorator=None): @@ -340,9 +350,8 @@ def execute_model(self, model, materialization, sql_override=None, return res - @available + @available.parse(_stub_relation) def create_temporary_table(self, sql, **kwargs): - # BQ queries always return a temp table with their results query_job, _ = self.connections.raw_execute(sql) bq_table = query_job.destination @@ -357,7 +366,7 @@ def create_temporary_table(self, sql, **kwargs): }, type=BigQueryRelation.Table) - @available + @available.parse_none def alter_table_add_columns(self, relation, columns): logger.debug('Adding columns ({}) to table {}".'.format( @@ -377,7 +386,7 @@ def alter_table_add_columns(self, relation, columns): new_table = google.cloud.bigquery.Table(table_ref, schema=new_schema) client.update_table(new_table, ['schema']) - @available + @available.parse_none def load_dataframe(self, database, schema, table_name, agate_table, column_override): bq_schema = self._agate_to_schema(agate_table, column_override) From 23a298c912bd7a34a9e3d2969acd2f67e0b6bac0 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 26 Apr 2019 09:36:45 -0600 Subject: [PATCH 029/860] remove non-destructive mode --- dbt/include/bigquery/macros/materializations/incremental.sql | 5 ----- dbt/include/bigquery/macros/materializations/table.sql | 1 - dbt/include/bigquery/macros/materializations/view.sql | 4 ++-- 3 files changed, 2 insertions(+), 8 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 98c65a1dd..fc84c11d9 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -3,13 +3,8 @@ {%- set unique_key = config.get('unique_key') -%} - {%- set non_destructive_mode = (flags.NON_DESTRUCTIVE == True) -%} {%- set full_refresh_mode = (flags.FULL_REFRESH == True) -%} - {% if non_destructive_mode %} - {{ exceptions.raise_compiler_error("--non-destructive mode is not supported on BigQuery") }} - {% endif %} - {%- set identifier = model['alias'] -%} {%- set old_relation = adapter.get_relation(database=database, schema=schema, identifier=identifier) -%} diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql index 1fb13779b..378b1c2ec 100644 --- a/dbt/include/bigquery/macros/materializations/table.sql +++ b/dbt/include/bigquery/macros/materializations/table.sql @@ -29,7 +29,6 @@ {% materialization table, adapter='bigquery' -%} {%- set identifier = model['alias'] -%} - {%- set non_destructive_mode = (flags.NON_DESTRUCTIVE == True) -%} {%- set old_relation = adapter.get_relation(database=database, schema=schema, identifier=identifier) -%} {%- set exists_not_as_table = (old_relation is not none and not old_relation.is_table) -%} {%- set target_relation = api.Relation.create(database=database, schema=schema, identifier=identifier, type='table') -%} diff --git a/dbt/include/bigquery/macros/materializations/view.sql b/dbt/include/bigquery/macros/materializations/view.sql index 561c38bb5..7821f3e12 100644 --- a/dbt/include/bigquery/macros/materializations/view.sql +++ b/dbt/include/bigquery/macros/materializations/view.sql @@ -1,6 +1,6 @@ -{% macro bigquery__handle_existing_table(full_refresh, non_destructive_mode, old_relation) %} - {%- if full_refresh and not non_destructive_mode -%} +{% macro bigquery__handle_existing_table(full_refresh, old_relation) %} + {%- if full_refresh -%} {{ adapter.drop_relation(old_relation) }} {%- else -%} {{ exceptions.relation_wrong_type(old_relation, 'view') }} From cb20130643368aa5d21534a7627e30975e8430ba Mon Sep 17 00:00:00 2001 From: Bastien Boutonnet Date: Sat, 27 Apr 2019 06:46:41 -0700 Subject: [PATCH 030/860] Revert "Merge branch 'dev/wilt-chamberlain' into snowflake_create_or_replace" This reverts commit 3ab8238cfbdbfe6ba7ca2cd33617a82ba8e44c9f, reversing changes made to 43a9db55b103b61be39da6a8b204b6751b891b3b. --- dbt/adapters/bigquery/__init__.py | 5 +- dbt/adapters/bigquery/connections.py | 72 +++++------ dbt/adapters/bigquery/impl.py | 118 ++++++++++-------- dbt/adapters/bigquery/relation.py | 102 +-------------- dbt/include/bigquery/macros/adapters.sql | 14 +-- .../macros/materializations/archive.sql | 11 +- .../macros/materializations/incremental.sql | 8 +- setup.py | 2 +- 8 files changed, 119 insertions(+), 213 deletions(-) diff --git a/dbt/adapters/bigquery/__init__.py b/dbt/adapters/bigquery/__init__.py index c45656772..5707c1690 100644 --- a/dbt/adapters/bigquery/__init__.py +++ b/dbt/adapters/bigquery/__init__.py @@ -1,7 +1,6 @@ -from dbt.adapters.bigquery.connections import BigQueryConnectionManager # noqa +from dbt.adapters.bigquery.connections import BigQueryConnectionManager from dbt.adapters.bigquery.connections import BigQueryCredentials -from dbt.adapters.bigquery.relation import BigQueryRelation # noqa -from dbt.adapters.bigquery.relation import BigQueryColumn # noqa +from dbt.adapters.bigquery.relation import BigQueryRelation from dbt.adapters.bigquery.impl import BigQueryAdapter from dbt.adapters.base import AdapterPlugin diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 24731c1a7..fbe0448ad 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -1,3 +1,4 @@ +import abc from contextlib import contextmanager import google.auth @@ -9,6 +10,7 @@ import dbt.clients.agate_helper import dbt.exceptions from dbt.adapters.base import BaseConnectionManager, Credentials +from dbt.compat import abstractclassmethod from dbt.logger import GLOBAL_LOGGER as logger @@ -75,11 +77,8 @@ def handle_error(cls, error, message, sql): raise dbt.exceptions.DatabaseException(error_msg) - def clear_transaction(self): - pass - @contextmanager - def exception_handler(self, sql): + def exception_handler(self, sql, connection_name='master'): try: yield @@ -94,11 +93,6 @@ def exception_handler(self, sql): except Exception as e: logger.debug("Unhandled error while running:\n{}".format(sql)) logger.debug(e) - if isinstance(e, dbt.exceptions.RuntimeException): - # during a sql query, an internal to dbt exception was raised. - # this sounds a lot like a signal handler and probably has - # useful information, so raise it without modification. - raise raise dbt.exceptions.RuntimeException(dbt.compat.to_string(e)) def cancel_open(self): @@ -110,10 +104,10 @@ def close(cls, connection): return connection - def begin(self): + def begin(self, name): pass - def commit(self): + def commit(self, connection): pass @classmethod @@ -153,7 +147,7 @@ def open(cls, connection): try: handle = cls.get_bigquery_client(connection.credentials) - except google.auth.exceptions.DefaultCredentialsError: + except google.auth.exceptions.DefaultCredentialsError as e: logger.info("Please log into GCP to continue") dbt.clients.gcloud.setup_default_credentials() @@ -184,25 +178,25 @@ def get_table_from_response(cls, resp): rows = [dict(row.items()) for row in resp] return dbt.clients.agate_helper.table_from_data(rows, column_names) - def raw_execute(self, sql, fetch=False): - conn = self.get_thread_connection() + def raw_execute(self, sql, name=None, fetch=False): + conn = self.get(name) client = conn.handle - logger.debug('On %s: %s', conn.name, sql) + logger.debug('On %s: %s', name, sql) job_config = google.cloud.bigquery.QueryJobConfig() job_config.use_legacy_sql = False query_job = client.query(sql, job_config) # this blocks until the query has completed - with self.exception_handler(sql): + with self.exception_handler(sql, conn.name): iterator = query_job.result() return query_job, iterator - def execute(self, sql, auto_begin=False, fetch=None): + def execute(self, sql, name=None, auto_begin=False, fetch=None): # auto_begin is ignored on bigquery, and only included for consistency - _, iterator = self.raw_execute(sql, fetch=fetch) + _, iterator = self.raw_execute(sql, name=name, fetch=fetch) if fetch: res = self.get_table_from_response(iterator) @@ -213,31 +207,32 @@ def execute(self, sql, auto_begin=False, fetch=None): status = 'OK' return status, res - def create_bigquery_table(self, database, schema, table_name, callback, - sql): + def create_bigquery_table(self, database, schema, table_name, conn_name, + callback, sql): """Create a bigquery table. The caller must supply a callback that takes one argument, a `google.cloud.bigquery.Table`, and mutates it. """ - conn = self.get_thread_connection() + conn = self.get(conn_name) client = conn.handle view_ref = self.table_ref(database, schema, table_name, conn) view = google.cloud.bigquery.Table(view_ref) callback(view) - with self.exception_handler(sql): + with self.exception_handler(sql, conn.name): client.create_table(view) - def create_view(self, database, schema, table_name, sql): + def create_view(self, database, schema, table_name, conn_name, sql): def callback(table): table.view_query = sql table.view_use_legacy_sql = False - self.create_bigquery_table(database, schema, table_name, callback, sql) + self.create_bigquery_table(database, schema, table_name, conn_name, + callback, sql) - def create_table(self, database, schema, table_name, sql): - conn = self.get_thread_connection() + def create_table(self, database, schema, table_name, conn_name, sql): + conn = self.get(conn_name) client = conn.handle table_ref = self.table_ref(database, schema, table_name, conn) @@ -248,15 +243,16 @@ def create_table(self, database, schema, table_name, sql): query_job = client.query(sql, job_config=job_config) # this waits for the job to complete - with self.exception_handler(sql): + with self.exception_handler(sql, conn_name): query_job.result(timeout=self.get_timeout(conn)) - def create_date_partitioned_table(self, database, schema, table_name): + def create_date_partitioned_table(self, database, schema, table_name, + conn_name): def callback(table): table.partitioning_type = 'DAY' - self.create_bigquery_table(database, schema, table_name, callback, - 'CREATE DAY PARTITIONED TABLE') + self.create_bigquery_table(database, schema, table_name, conn_name, + callback, 'CREATE DAY PARTITIONED TABLE') @staticmethod def dataset(database, schema, conn): @@ -267,24 +263,24 @@ def table_ref(self, database, schema, table_name, conn): dataset = self.dataset(database, schema, conn) return dataset.table(table_name) - def get_bq_table(self, database, schema, identifier): + def get_bq_table(self, database, schema, identifier, conn_name=None): """Get a bigquery table for a schema/model.""" - conn = self.get_thread_connection() + conn = self.get(conn_name) table_ref = self.table_ref(database, schema, identifier, conn) return conn.handle.get_table(table_ref) - def drop_dataset(self, database, schema): - conn = self.get_thread_connection() + def drop_dataset(self, database, schema, conn_name=None): + conn = self.get(conn_name) dataset = self.dataset(database, schema, conn) client = conn.handle - with self.exception_handler('drop dataset'): + with self.exception_handler('drop dataset', conn.name): for table in client.list_tables(dataset): client.delete_table(table.reference) client.delete_dataset(dataset) - def create_dataset(self, database, schema): - conn = self.get_thread_connection() + def create_dataset(self, database, schema, conn_name=None): + conn = self.get(conn_name) client = conn.handle dataset = self.dataset(database, schema, conn) @@ -295,5 +291,5 @@ def create_dataset(self, database, schema): except google.api_core.exceptions.NotFound: pass - with self.exception_handler('create dataset'): + with self.exception_handler('create dataset', conn.name): client.create_dataset(dataset) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 012affa13..e503148be 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -5,13 +5,13 @@ import dbt.compat import dbt.deprecations import dbt.exceptions +import dbt.schema import dbt.flags as flags import dbt.clients.gcloud import dbt.clients.agate_helper from dbt.adapters.base import BaseAdapter, available from dbt.adapters.bigquery import BigQueryRelation -from dbt.adapters.bigquery import BigQueryColumn from dbt.adapters.bigquery import BigQueryConnectionManager from dbt.contracts.connection import Connection from dbt.logger import GLOBAL_LOGGER as logger @@ -26,6 +26,19 @@ import agate +def column_to_bq_schema(col): + """Convert a column to a bigquery schema object. This is here instead of + in dbt.schema to avoid importing google libraries there. + """ + kwargs = {} + if len(col.fields) > 0: + fields = [column_to_bq_schema(field) for field in col.fields] + kwargs = {"fields": fields} + + return google.cloud.bigquery.SchemaField(col.name, col.dtype, col.mode, + **kwargs) + + class BigQueryAdapter(BaseAdapter): RELATION_TYPES = { @@ -35,7 +48,7 @@ class BigQueryAdapter(BaseAdapter): } Relation = BigQueryRelation - Column = BigQueryColumn + Column = dbt.schema.BigQueryColumn ConnectionManager = BigQueryConnectionManager AdapterSpecificConfigs = frozenset({"cluster_by", "partition_by"}) @@ -52,12 +65,13 @@ def date_function(cls): def is_cancelable(cls): return False - def drop_relation(self, relation): - is_cached = self._schema_is_cached(relation.database, relation.schema) + def drop_relation(self, relation, model_name=None): + is_cached = self._schema_is_cached(relation.database, relation.schema, + model_name) if is_cached: self.cache.drop(relation) - conn = self.connections.get_thread_connection() + conn = self.connections.get(model_name) client = conn.handle dataset = self.connections.dataset(relation.database, relation.schema, @@ -65,37 +79,32 @@ def drop_relation(self, relation): relation_object = dataset.table(relation.identifier) client.delete_table(relation_object) - def truncate_relation(self, relation): + def truncate_relation(self, relation, model_name=None): raise dbt.exceptions.NotImplementedException( '`truncate` is not implemented for this adapter!' ) - def rename_relation(self, from_relation, to_relation): + def rename_relation(self, from_relation, to_relation, model_name=None): raise dbt.exceptions.NotImplementedException( '`rename_relation` is not implemented for this adapter!' ) - @available - def list_schemas(self, database): - conn = self.connections.get_thread_connection() + def list_schemas(self, database, model_name=None): + conn = self.connections.get(model_name) client = conn.handle - with self.connections.exception_handler('list dataset'): + with self.connections.exception_handler('list dataset', conn.name): all_datasets = client.list_datasets(project=database, include_all=True) return [ds.dataset_id for ds in all_datasets] - @available - def check_schema_exists(self, database, schema): - superself = super(BigQueryAdapter, self) - return superself.check_schema_exists(database, schema) - - def get_columns_in_relation(self, relation): + def get_columns_in_relation(self, relation, model_name=None): try: table = self.connections.get_bq_table( database=relation.database, schema=relation.schema, - identifier=relation.table_name + identifier=relation.table_name, + conn_name=model_name ) return self._get_dbt_columns_from_bq_table(table) @@ -103,17 +112,17 @@ def get_columns_in_relation(self, relation): logger.debug("get_columns_in_relation error: {}".format(e)) return [] - def expand_column_types(self, goal, current): + def expand_column_types(self, goal, current, model_name=None): # This is a no-op on BigQuery pass - def list_relations_without_caching(self, information_schema, schema): - connection = self.connections.get_thread_connection() + def list_relations_without_caching(self, database, schema, + model_name=None): + connection = self.connections.get(model_name) client = connection.handle - bigquery_dataset = self.connections.dataset( - information_schema.database, schema, connection - ) + bigquery_dataset = self.connections.dataset(database, schema, + connection) all_tables = client.list_tables( bigquery_dataset, @@ -131,17 +140,18 @@ def list_relations_without_caching(self, information_schema, schema): # the implementation of list_relations for other adapters try: return [self._bq_table_to_relation(table) for table in all_tables] - except google.api_core.exceptions.NotFound: + except google.api_core.exceptions.NotFound as e: return [] - def get_relation(self, database, schema, identifier): - if self._schema_is_cached(database, schema): + def get_relation(self, database, schema, identifier, model_name=None): + if self._schema_is_cached(database, schema, model_name): # if it's in the cache, use the parent's model of going through # the relations cache and picking out the relation return super(BigQueryAdapter, self).get_relation( database=database, schema=schema, - identifier=identifier + identifier=identifier, + model_name=model_name ) try: @@ -150,16 +160,16 @@ def get_relation(self, database, schema, identifier): table = None return self._bq_table_to_relation(table) - def create_schema(self, database, schema): + def create_schema(self, database, schema, model_name=None): logger.debug('Creating schema "%s.%s".', database, schema) - self.connections.create_dataset(database, schema) + self.connections.create_dataset(database, schema, model_name) - def drop_schema(self, database, schema): + def drop_schema(self, database, schema, model_name=None): logger.debug('Dropping schema "%s.%s".', database, schema) - if not self.check_schema_exists(database, schema): + if not self.check_schema_exists(database, schema, model_name): return - self.connections.drop_dataset(database, schema) + self.connections.drop_dataset(database, schema, model_name) @classmethod def quote(cls, identifier): @@ -221,14 +231,16 @@ def _agate_to_schema(self, agate_table, column_override): def _materialize_as_view(self, model): model_database = model.get('database') model_schema = model.get('schema') + model_name = model.get('name') model_alias = model.get('alias') model_sql = model.get('injected_sql') - logger.debug("Model SQL ({}):\n{}".format(model_alias, model_sql)) + logger.debug("Model SQL ({}):\n{}".format(model_name, model_sql)) self.connections.create_view( database=model_database, schema=model_schema, table_name=model_alias, + conn_name=model_name, sql=model_sql ) return "CREATE VIEW" @@ -236,6 +248,7 @@ def _materialize_as_view(self, model): def _materialize_as_table(self, model, model_sql, decorator=None): model_database = model.get('database') model_schema = model.get('schema') + model_name = model.get('name') model_alias = model.get('alias') if decorator is None: @@ -247,6 +260,7 @@ def _materialize_as_table(self, model, model_sql, decorator=None): self.connections.create_table( database=model_database, schema=model_schema, + conn_name=model_name, table_name=table_name, sql=model_sql ) @@ -292,10 +306,10 @@ def warning_on_hooks(hook_type): dbt.ui.printer.COLOR_FG_YELLOW) @available - def add_query(self, sql, auto_begin=True, bindings=None, - abridge_sql_log=False): - if self.nice_connection_name() in ['on-run-start', 'on-run-end']: - self.warning_on_hooks(self.nice_connection_name()) + def add_query(self, sql, model_name=None, auto_begin=True, + bindings=None, abridge_sql_log=False): + if model_name in ['on-run-start', 'on-run-end']: + self.warning_on_hooks(model_name) else: raise dbt.exceptions.NotImplementedException( '`add_query` is not implemented for this adapter!') @@ -304,24 +318,24 @@ def add_query(self, sql, auto_begin=True, bindings=None, # Special bigquery adapter methods ### @available - def make_date_partitioned_table(self, relation): + def make_date_partitioned_table(self, relation, model_name=None): return self.connections.create_date_partitioned_table( database=relation.database, schema=relation.schema, - table_name=relation.identifier + table_name=relation.identifier, + conn_name=model_name ) @available def execute_model(self, model, materialization, sql_override=None, - decorator=None): + decorator=None, model_name=None): if sql_override is None: sql_override = model.get('injected_sql') if flags.STRICT_MODE: - connection = self.connections.get_thread_connection() + connection = self.connections.get(model.get('name')) assert isinstance(connection, Connection) - assert(connection.name == model.get('name')) if materialization == 'view': res = self._materialize_as_view(model) @@ -334,10 +348,10 @@ def execute_model(self, model, materialization, sql_override=None, return res @available - def create_temporary_table(self, sql, **kwargs): + def create_temporary_table(self, sql, model_name=None, **kwargs): # BQ queries always return a temp table with their results - query_job, _ = self.connections.raw_execute(sql) + query_job, _ = self.connections.raw_execute(sql, model_name) bq_table = query_job.destination return self.Relation.create( @@ -351,12 +365,12 @@ def create_temporary_table(self, sql, **kwargs): type=BigQueryRelation.Table) @available - def alter_table_add_columns(self, relation, columns): + def alter_table_add_columns(self, relation, columns, model_name=None): logger.debug('Adding columns ({}) to table {}".'.format( columns, relation)) - conn = self.connections.get_thread_connection() + conn = self.connections.get(model_name) client = conn.handle table_ref = self.connections.table_ref(relation.database, @@ -364,7 +378,7 @@ def alter_table_add_columns(self, relation, columns): relation.identifier, conn) table = client.get_table(table_ref) - new_columns = [col.column_to_bq_schema() for col in columns] + new_columns = [column_to_bq_schema(col) for col in columns] new_schema = table.schema + new_columns new_table = google.cloud.bigquery.Table(table_ref, schema=new_schema) @@ -372,9 +386,9 @@ def alter_table_add_columns(self, relation, columns): @available def load_dataframe(self, database, schema, table_name, agate_table, - column_override): + column_override, model_name=None): bq_schema = self._agate_to_schema(agate_table, column_override) - conn = self.connections.get_thread_connection() + conn = self.connections.get(model_name) client = conn.handle table = self.connections.table_ref(database, schema, table_name, conn) @@ -388,7 +402,7 @@ def load_dataframe(self, database, schema, table_name, agate_table, job_config=load_config) timeout = self.connections.get_timeout(conn) - with self.connections.exception_handler("LOAD TABLE"): + with self.connections.exception_handler("LOAD TABLE", conn.name): self.poll_until_job_completes(job, timeout) ### @@ -459,7 +473,7 @@ def _get_stats_columns(cls, table, relation_type): return zip(column_names, column_values) def get_catalog(self, manifest): - connection = self.connections.get_thread_connection() + connection = self.connections.get('catalog') client = connection.handle schemas = manifest.get_used_schemas() diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py index e4f982b2c..a489512d6 100644 --- a/dbt/adapters/bigquery/relation.py +++ b/dbt/adapters/bigquery/relation.py @@ -1,8 +1,6 @@ -from dbt.adapters.base.relation import BaseRelation, Column +from dbt.adapters.base.relation import BaseRelation from dbt.utils import filter_null_values -import google.cloud.bigquery - class BigQueryRelation(BaseRelation): External = "external" @@ -109,101 +107,3 @@ def dataset(self): @property def identifier(self): return self.path.get('identifier') - - -class BigQueryColumn(Column): - TYPE_LABELS = { - 'STRING': 'STRING', - 'TIMESTAMP': 'TIMESTAMP', - 'FLOAT': 'FLOAT64', - 'INTEGER': 'INT64', - 'RECORD': 'RECORD', - } - - def __init__(self, column, dtype, fields=None, mode='NULLABLE'): - super(BigQueryColumn, self).__init__(column, dtype) - - if fields is None: - fields = [] - - self.fields = self.wrap_subfields(fields) - self.mode = mode - - @classmethod - def wrap_subfields(cls, fields): - return [BigQueryColumn.create_from_field(field) for field in fields] - - @classmethod - def create_from_field(cls, field): - return BigQueryColumn(field.name, cls.translate_type(field.field_type), - field.fields, field.mode) - - @classmethod - def _flatten_recursive(cls, col, prefix=None): - if prefix is None: - prefix = [] - - if len(col.fields) == 0: - prefixed_name = ".".join(prefix + [col.column]) - new_col = BigQueryColumn(prefixed_name, col.dtype, col.fields, - col.mode) - return [new_col] - - new_fields = [] - for field in col.fields: - new_prefix = prefix + [col.column] - new_fields.extend(cls._flatten_recursive(field, new_prefix)) - - return new_fields - - def flatten(self): - return self._flatten_recursive(self) - - @property - def quoted(self): - return '`{}`'.format(self.column) - - def literal(self, value): - return "cast({} as {})".format(value, self.dtype) - - @property - def data_type(self): - if self.dtype.upper() == 'RECORD': - subcols = [ - "{} {}".format(col.name, col.data_type) for col in self.fields - ] - field_type = 'STRUCT<{}>'.format(", ".join(subcols)) - - else: - field_type = self.dtype - - if self.mode.upper() == 'REPEATED': - return 'ARRAY<{}>'.format(field_type) - - else: - return field_type - - def is_string(self): - return self.dtype.lower() == 'string' - - def is_numeric(self): - return False - - def can_expand_to(self, other_column): - """returns True if both columns are strings""" - return self.is_string() and other_column.is_string() - - def __repr__(self): - return "".format(self.name, self.data_type, - self.mode) - - def column_to_bq_schema(self): - """Convert a column to a bigquery schema object. - """ - kwargs = {} - if len(self.fields) > 0: - fields = [field.column_to_bq_schema() for field in self.fields] - kwargs = {"fields": fields} - - return google.cloud.bigquery.SchemaField(self.name, self.dtype, - self.mode, **kwargs) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 1c87ce4dc..d9700f359 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -59,21 +59,11 @@ {% endmacro %} -{% macro bigquery__list_relations_without_caching(information_schema, schema) -%} - {{ return(adapter.list_relations_without_caching(information_schema, schema)) }} +{% macro bigquery__list_relations_without_caching(database, schema) -%} + {{ return(adapter.list_relations_without_caching(database, schema)) }} {% endmacro %} {% macro bigquery__current_timestamp() -%} CURRENT_TIMESTAMP() {%- endmacro %} - - -{% macro bigquery__list_schemas(database) %} - {{ return(adapter.list_schemas()) }} -{% endmacro %} - - -{% macro bigquery__check_schema_exists(information_schema, schema) %} - {{ return(adapter.check_schema_exists(information_schema.database, schema)) }} -{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/archive.sql b/dbt/include/bigquery/macros/materializations/archive.sql index 7a95f440f..5548b71a7 100644 --- a/dbt/include/bigquery/macros/materializations/archive.sql +++ b/dbt/include/bigquery/macros/materializations/archive.sql @@ -4,10 +4,11 @@ {% endmacro %} -{% macro bigquery__archive_hash_arguments(args) %} - to_hex(md5(concat({% for arg in args %}coalesce(cast({{ arg }} as string), ''){% if not loop.last %}, '|',{% endif %}{% endfor %}))) +{% macro bigquery__archive_scd_hash() %} + to_hex(md5(concat(cast(`dbt_pk` as string), '|', cast(`dbt_updated_at` as string)))) {% endmacro %} + {% macro bigquery__create_columns(relation, columns) %} {{ adapter.alter_table_add_columns(relation, columns) }} {% endmacro %} @@ -15,8 +16,8 @@ {% macro bigquery__archive_update(target_relation, tmp_relation) %} update {{ target_relation }} as dest - set dest.dbt_valid_to = tmp.dbt_valid_to + set dest.{{ adapter.quote('valid_to') }} = tmp.{{ adapter.quote('valid_to') }} from {{ tmp_relation }} as tmp - where tmp.dbt_scd_id = dest.dbt_scd_id - and change_type = 'update'; + where tmp.{{ adapter.quote('scd_id') }} = dest.{{ adapter.quote('scd_id') }} + and {{ adapter.quote('change_type') }} = 'update'; {% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 98c65a1dd..d561ea487 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -2,6 +2,7 @@ {% materialization incremental, adapter='bigquery' -%} {%- set unique_key = config.get('unique_key') -%} + {%- set sql_where = config.get('sql_where') -%} {%- set non_destructive_mode = (flags.NON_DESTRUCTIVE == True) -%} {%- set full_refresh_mode = (flags.FULL_REFRESH == True) -%} @@ -33,7 +34,12 @@ {% set source_sql -%} {#-- wrap sql in parens to make it a subquery --#} ( - {{ sql }} + select * from ( + {{ sql }} + ) + {% if sql_where %} + where ({{ sql_where }}) or ({{ sql_where }}) is null + {% endif %} ) {%- endset -%} diff --git a/setup.py b/setup.py index 1d21eba4e..d563e6b36 100644 --- a/setup.py +++ b/setup.py @@ -3,7 +3,7 @@ from distutils.core import setup package_name = "dbt-bigquery" -package_version = "0.13.0" +package_version = "0.13.0a1" description = """The bigquery adapter plugin for dbt (data build tool)""" From 983c2bc8ccff6be05bd07390023c36bfbe8233b8 Mon Sep 17 00:00:00 2001 From: Bastien Boutonnet Date: Sat, 27 Apr 2019 10:18:30 -0700 Subject: [PATCH 031/860] Revert "Revert "Merge branch 'dev/wilt-chamberlain' into snowflake_create_or_replace"" This reverts commit 4f62978de56b2e7f09778dedd9d0f6c440dd59c7. --- dbt/adapters/bigquery/__init__.py | 5 +- dbt/adapters/bigquery/connections.py | 72 ++++++----- dbt/adapters/bigquery/impl.py | 118 ++++++++---------- dbt/adapters/bigquery/relation.py | 102 ++++++++++++++- dbt/include/bigquery/macros/adapters.sql | 14 ++- .../macros/materializations/archive.sql | 11 +- .../macros/materializations/incremental.sql | 8 +- setup.py | 2 +- 8 files changed, 213 insertions(+), 119 deletions(-) diff --git a/dbt/adapters/bigquery/__init__.py b/dbt/adapters/bigquery/__init__.py index 5707c1690..c45656772 100644 --- a/dbt/adapters/bigquery/__init__.py +++ b/dbt/adapters/bigquery/__init__.py @@ -1,6 +1,7 @@ -from dbt.adapters.bigquery.connections import BigQueryConnectionManager +from dbt.adapters.bigquery.connections import BigQueryConnectionManager # noqa from dbt.adapters.bigquery.connections import BigQueryCredentials -from dbt.adapters.bigquery.relation import BigQueryRelation +from dbt.adapters.bigquery.relation import BigQueryRelation # noqa +from dbt.adapters.bigquery.relation import BigQueryColumn # noqa from dbt.adapters.bigquery.impl import BigQueryAdapter from dbt.adapters.base import AdapterPlugin diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index fbe0448ad..24731c1a7 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -1,4 +1,3 @@ -import abc from contextlib import contextmanager import google.auth @@ -10,7 +9,6 @@ import dbt.clients.agate_helper import dbt.exceptions from dbt.adapters.base import BaseConnectionManager, Credentials -from dbt.compat import abstractclassmethod from dbt.logger import GLOBAL_LOGGER as logger @@ -77,8 +75,11 @@ def handle_error(cls, error, message, sql): raise dbt.exceptions.DatabaseException(error_msg) + def clear_transaction(self): + pass + @contextmanager - def exception_handler(self, sql, connection_name='master'): + def exception_handler(self, sql): try: yield @@ -93,6 +94,11 @@ def exception_handler(self, sql, connection_name='master'): except Exception as e: logger.debug("Unhandled error while running:\n{}".format(sql)) logger.debug(e) + if isinstance(e, dbt.exceptions.RuntimeException): + # during a sql query, an internal to dbt exception was raised. + # this sounds a lot like a signal handler and probably has + # useful information, so raise it without modification. + raise raise dbt.exceptions.RuntimeException(dbt.compat.to_string(e)) def cancel_open(self): @@ -104,10 +110,10 @@ def close(cls, connection): return connection - def begin(self, name): + def begin(self): pass - def commit(self, connection): + def commit(self): pass @classmethod @@ -147,7 +153,7 @@ def open(cls, connection): try: handle = cls.get_bigquery_client(connection.credentials) - except google.auth.exceptions.DefaultCredentialsError as e: + except google.auth.exceptions.DefaultCredentialsError: logger.info("Please log into GCP to continue") dbt.clients.gcloud.setup_default_credentials() @@ -178,25 +184,25 @@ def get_table_from_response(cls, resp): rows = [dict(row.items()) for row in resp] return dbt.clients.agate_helper.table_from_data(rows, column_names) - def raw_execute(self, sql, name=None, fetch=False): - conn = self.get(name) + def raw_execute(self, sql, fetch=False): + conn = self.get_thread_connection() client = conn.handle - logger.debug('On %s: %s', name, sql) + logger.debug('On %s: %s', conn.name, sql) job_config = google.cloud.bigquery.QueryJobConfig() job_config.use_legacy_sql = False query_job = client.query(sql, job_config) # this blocks until the query has completed - with self.exception_handler(sql, conn.name): + with self.exception_handler(sql): iterator = query_job.result() return query_job, iterator - def execute(self, sql, name=None, auto_begin=False, fetch=None): + def execute(self, sql, auto_begin=False, fetch=None): # auto_begin is ignored on bigquery, and only included for consistency - _, iterator = self.raw_execute(sql, name=name, fetch=fetch) + _, iterator = self.raw_execute(sql, fetch=fetch) if fetch: res = self.get_table_from_response(iterator) @@ -207,32 +213,31 @@ def execute(self, sql, name=None, auto_begin=False, fetch=None): status = 'OK' return status, res - def create_bigquery_table(self, database, schema, table_name, conn_name, - callback, sql): + def create_bigquery_table(self, database, schema, table_name, callback, + sql): """Create a bigquery table. The caller must supply a callback that takes one argument, a `google.cloud.bigquery.Table`, and mutates it. """ - conn = self.get(conn_name) + conn = self.get_thread_connection() client = conn.handle view_ref = self.table_ref(database, schema, table_name, conn) view = google.cloud.bigquery.Table(view_ref) callback(view) - with self.exception_handler(sql, conn.name): + with self.exception_handler(sql): client.create_table(view) - def create_view(self, database, schema, table_name, conn_name, sql): + def create_view(self, database, schema, table_name, sql): def callback(table): table.view_query = sql table.view_use_legacy_sql = False - self.create_bigquery_table(database, schema, table_name, conn_name, - callback, sql) + self.create_bigquery_table(database, schema, table_name, callback, sql) - def create_table(self, database, schema, table_name, conn_name, sql): - conn = self.get(conn_name) + def create_table(self, database, schema, table_name, sql): + conn = self.get_thread_connection() client = conn.handle table_ref = self.table_ref(database, schema, table_name, conn) @@ -243,16 +248,15 @@ def create_table(self, database, schema, table_name, conn_name, sql): query_job = client.query(sql, job_config=job_config) # this waits for the job to complete - with self.exception_handler(sql, conn_name): + with self.exception_handler(sql): query_job.result(timeout=self.get_timeout(conn)) - def create_date_partitioned_table(self, database, schema, table_name, - conn_name): + def create_date_partitioned_table(self, database, schema, table_name): def callback(table): table.partitioning_type = 'DAY' - self.create_bigquery_table(database, schema, table_name, conn_name, - callback, 'CREATE DAY PARTITIONED TABLE') + self.create_bigquery_table(database, schema, table_name, callback, + 'CREATE DAY PARTITIONED TABLE') @staticmethod def dataset(database, schema, conn): @@ -263,24 +267,24 @@ def table_ref(self, database, schema, table_name, conn): dataset = self.dataset(database, schema, conn) return dataset.table(table_name) - def get_bq_table(self, database, schema, identifier, conn_name=None): + def get_bq_table(self, database, schema, identifier): """Get a bigquery table for a schema/model.""" - conn = self.get(conn_name) + conn = self.get_thread_connection() table_ref = self.table_ref(database, schema, identifier, conn) return conn.handle.get_table(table_ref) - def drop_dataset(self, database, schema, conn_name=None): - conn = self.get(conn_name) + def drop_dataset(self, database, schema): + conn = self.get_thread_connection() dataset = self.dataset(database, schema, conn) client = conn.handle - with self.exception_handler('drop dataset', conn.name): + with self.exception_handler('drop dataset'): for table in client.list_tables(dataset): client.delete_table(table.reference) client.delete_dataset(dataset) - def create_dataset(self, database, schema, conn_name=None): - conn = self.get(conn_name) + def create_dataset(self, database, schema): + conn = self.get_thread_connection() client = conn.handle dataset = self.dataset(database, schema, conn) @@ -291,5 +295,5 @@ def create_dataset(self, database, schema, conn_name=None): except google.api_core.exceptions.NotFound: pass - with self.exception_handler('create dataset', conn.name): + with self.exception_handler('create dataset'): client.create_dataset(dataset) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index e503148be..012affa13 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -5,13 +5,13 @@ import dbt.compat import dbt.deprecations import dbt.exceptions -import dbt.schema import dbt.flags as flags import dbt.clients.gcloud import dbt.clients.agate_helper from dbt.adapters.base import BaseAdapter, available from dbt.adapters.bigquery import BigQueryRelation +from dbt.adapters.bigquery import BigQueryColumn from dbt.adapters.bigquery import BigQueryConnectionManager from dbt.contracts.connection import Connection from dbt.logger import GLOBAL_LOGGER as logger @@ -26,19 +26,6 @@ import agate -def column_to_bq_schema(col): - """Convert a column to a bigquery schema object. This is here instead of - in dbt.schema to avoid importing google libraries there. - """ - kwargs = {} - if len(col.fields) > 0: - fields = [column_to_bq_schema(field) for field in col.fields] - kwargs = {"fields": fields} - - return google.cloud.bigquery.SchemaField(col.name, col.dtype, col.mode, - **kwargs) - - class BigQueryAdapter(BaseAdapter): RELATION_TYPES = { @@ -48,7 +35,7 @@ class BigQueryAdapter(BaseAdapter): } Relation = BigQueryRelation - Column = dbt.schema.BigQueryColumn + Column = BigQueryColumn ConnectionManager = BigQueryConnectionManager AdapterSpecificConfigs = frozenset({"cluster_by", "partition_by"}) @@ -65,13 +52,12 @@ def date_function(cls): def is_cancelable(cls): return False - def drop_relation(self, relation, model_name=None): - is_cached = self._schema_is_cached(relation.database, relation.schema, - model_name) + def drop_relation(self, relation): + is_cached = self._schema_is_cached(relation.database, relation.schema) if is_cached: self.cache.drop(relation) - conn = self.connections.get(model_name) + conn = self.connections.get_thread_connection() client = conn.handle dataset = self.connections.dataset(relation.database, relation.schema, @@ -79,32 +65,37 @@ def drop_relation(self, relation, model_name=None): relation_object = dataset.table(relation.identifier) client.delete_table(relation_object) - def truncate_relation(self, relation, model_name=None): + def truncate_relation(self, relation): raise dbt.exceptions.NotImplementedException( '`truncate` is not implemented for this adapter!' ) - def rename_relation(self, from_relation, to_relation, model_name=None): + def rename_relation(self, from_relation, to_relation): raise dbt.exceptions.NotImplementedException( '`rename_relation` is not implemented for this adapter!' ) - def list_schemas(self, database, model_name=None): - conn = self.connections.get(model_name) + @available + def list_schemas(self, database): + conn = self.connections.get_thread_connection() client = conn.handle - with self.connections.exception_handler('list dataset', conn.name): + with self.connections.exception_handler('list dataset'): all_datasets = client.list_datasets(project=database, include_all=True) return [ds.dataset_id for ds in all_datasets] - def get_columns_in_relation(self, relation, model_name=None): + @available + def check_schema_exists(self, database, schema): + superself = super(BigQueryAdapter, self) + return superself.check_schema_exists(database, schema) + + def get_columns_in_relation(self, relation): try: table = self.connections.get_bq_table( database=relation.database, schema=relation.schema, - identifier=relation.table_name, - conn_name=model_name + identifier=relation.table_name ) return self._get_dbt_columns_from_bq_table(table) @@ -112,17 +103,17 @@ def get_columns_in_relation(self, relation, model_name=None): logger.debug("get_columns_in_relation error: {}".format(e)) return [] - def expand_column_types(self, goal, current, model_name=None): + def expand_column_types(self, goal, current): # This is a no-op on BigQuery pass - def list_relations_without_caching(self, database, schema, - model_name=None): - connection = self.connections.get(model_name) + def list_relations_without_caching(self, information_schema, schema): + connection = self.connections.get_thread_connection() client = connection.handle - bigquery_dataset = self.connections.dataset(database, schema, - connection) + bigquery_dataset = self.connections.dataset( + information_schema.database, schema, connection + ) all_tables = client.list_tables( bigquery_dataset, @@ -140,18 +131,17 @@ def list_relations_without_caching(self, database, schema, # the implementation of list_relations for other adapters try: return [self._bq_table_to_relation(table) for table in all_tables] - except google.api_core.exceptions.NotFound as e: + except google.api_core.exceptions.NotFound: return [] - def get_relation(self, database, schema, identifier, model_name=None): - if self._schema_is_cached(database, schema, model_name): + def get_relation(self, database, schema, identifier): + if self._schema_is_cached(database, schema): # if it's in the cache, use the parent's model of going through # the relations cache and picking out the relation return super(BigQueryAdapter, self).get_relation( database=database, schema=schema, - identifier=identifier, - model_name=model_name + identifier=identifier ) try: @@ -160,16 +150,16 @@ def get_relation(self, database, schema, identifier, model_name=None): table = None return self._bq_table_to_relation(table) - def create_schema(self, database, schema, model_name=None): + def create_schema(self, database, schema): logger.debug('Creating schema "%s.%s".', database, schema) - self.connections.create_dataset(database, schema, model_name) + self.connections.create_dataset(database, schema) - def drop_schema(self, database, schema, model_name=None): + def drop_schema(self, database, schema): logger.debug('Dropping schema "%s.%s".', database, schema) - if not self.check_schema_exists(database, schema, model_name): + if not self.check_schema_exists(database, schema): return - self.connections.drop_dataset(database, schema, model_name) + self.connections.drop_dataset(database, schema) @classmethod def quote(cls, identifier): @@ -231,16 +221,14 @@ def _agate_to_schema(self, agate_table, column_override): def _materialize_as_view(self, model): model_database = model.get('database') model_schema = model.get('schema') - model_name = model.get('name') model_alias = model.get('alias') model_sql = model.get('injected_sql') - logger.debug("Model SQL ({}):\n{}".format(model_name, model_sql)) + logger.debug("Model SQL ({}):\n{}".format(model_alias, model_sql)) self.connections.create_view( database=model_database, schema=model_schema, table_name=model_alias, - conn_name=model_name, sql=model_sql ) return "CREATE VIEW" @@ -248,7 +236,6 @@ def _materialize_as_view(self, model): def _materialize_as_table(self, model, model_sql, decorator=None): model_database = model.get('database') model_schema = model.get('schema') - model_name = model.get('name') model_alias = model.get('alias') if decorator is None: @@ -260,7 +247,6 @@ def _materialize_as_table(self, model, model_sql, decorator=None): self.connections.create_table( database=model_database, schema=model_schema, - conn_name=model_name, table_name=table_name, sql=model_sql ) @@ -306,10 +292,10 @@ def warning_on_hooks(hook_type): dbt.ui.printer.COLOR_FG_YELLOW) @available - def add_query(self, sql, model_name=None, auto_begin=True, - bindings=None, abridge_sql_log=False): - if model_name in ['on-run-start', 'on-run-end']: - self.warning_on_hooks(model_name) + def add_query(self, sql, auto_begin=True, bindings=None, + abridge_sql_log=False): + if self.nice_connection_name() in ['on-run-start', 'on-run-end']: + self.warning_on_hooks(self.nice_connection_name()) else: raise dbt.exceptions.NotImplementedException( '`add_query` is not implemented for this adapter!') @@ -318,24 +304,24 @@ def add_query(self, sql, model_name=None, auto_begin=True, # Special bigquery adapter methods ### @available - def make_date_partitioned_table(self, relation, model_name=None): + def make_date_partitioned_table(self, relation): return self.connections.create_date_partitioned_table( database=relation.database, schema=relation.schema, - table_name=relation.identifier, - conn_name=model_name + table_name=relation.identifier ) @available def execute_model(self, model, materialization, sql_override=None, - decorator=None, model_name=None): + decorator=None): if sql_override is None: sql_override = model.get('injected_sql') if flags.STRICT_MODE: - connection = self.connections.get(model.get('name')) + connection = self.connections.get_thread_connection() assert isinstance(connection, Connection) + assert(connection.name == model.get('name')) if materialization == 'view': res = self._materialize_as_view(model) @@ -348,10 +334,10 @@ def execute_model(self, model, materialization, sql_override=None, return res @available - def create_temporary_table(self, sql, model_name=None, **kwargs): + def create_temporary_table(self, sql, **kwargs): # BQ queries always return a temp table with their results - query_job, _ = self.connections.raw_execute(sql, model_name) + query_job, _ = self.connections.raw_execute(sql) bq_table = query_job.destination return self.Relation.create( @@ -365,12 +351,12 @@ def create_temporary_table(self, sql, model_name=None, **kwargs): type=BigQueryRelation.Table) @available - def alter_table_add_columns(self, relation, columns, model_name=None): + def alter_table_add_columns(self, relation, columns): logger.debug('Adding columns ({}) to table {}".'.format( columns, relation)) - conn = self.connections.get(model_name) + conn = self.connections.get_thread_connection() client = conn.handle table_ref = self.connections.table_ref(relation.database, @@ -378,7 +364,7 @@ def alter_table_add_columns(self, relation, columns, model_name=None): relation.identifier, conn) table = client.get_table(table_ref) - new_columns = [column_to_bq_schema(col) for col in columns] + new_columns = [col.column_to_bq_schema() for col in columns] new_schema = table.schema + new_columns new_table = google.cloud.bigquery.Table(table_ref, schema=new_schema) @@ -386,9 +372,9 @@ def alter_table_add_columns(self, relation, columns, model_name=None): @available def load_dataframe(self, database, schema, table_name, agate_table, - column_override, model_name=None): + column_override): bq_schema = self._agate_to_schema(agate_table, column_override) - conn = self.connections.get(model_name) + conn = self.connections.get_thread_connection() client = conn.handle table = self.connections.table_ref(database, schema, table_name, conn) @@ -402,7 +388,7 @@ def load_dataframe(self, database, schema, table_name, agate_table, job_config=load_config) timeout = self.connections.get_timeout(conn) - with self.connections.exception_handler("LOAD TABLE", conn.name): + with self.connections.exception_handler("LOAD TABLE"): self.poll_until_job_completes(job, timeout) ### @@ -473,7 +459,7 @@ def _get_stats_columns(cls, table, relation_type): return zip(column_names, column_values) def get_catalog(self, manifest): - connection = self.connections.get('catalog') + connection = self.connections.get_thread_connection() client = connection.handle schemas = manifest.get_used_schemas() diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py index a489512d6..e4f982b2c 100644 --- a/dbt/adapters/bigquery/relation.py +++ b/dbt/adapters/bigquery/relation.py @@ -1,6 +1,8 @@ -from dbt.adapters.base.relation import BaseRelation +from dbt.adapters.base.relation import BaseRelation, Column from dbt.utils import filter_null_values +import google.cloud.bigquery + class BigQueryRelation(BaseRelation): External = "external" @@ -107,3 +109,101 @@ def dataset(self): @property def identifier(self): return self.path.get('identifier') + + +class BigQueryColumn(Column): + TYPE_LABELS = { + 'STRING': 'STRING', + 'TIMESTAMP': 'TIMESTAMP', + 'FLOAT': 'FLOAT64', + 'INTEGER': 'INT64', + 'RECORD': 'RECORD', + } + + def __init__(self, column, dtype, fields=None, mode='NULLABLE'): + super(BigQueryColumn, self).__init__(column, dtype) + + if fields is None: + fields = [] + + self.fields = self.wrap_subfields(fields) + self.mode = mode + + @classmethod + def wrap_subfields(cls, fields): + return [BigQueryColumn.create_from_field(field) for field in fields] + + @classmethod + def create_from_field(cls, field): + return BigQueryColumn(field.name, cls.translate_type(field.field_type), + field.fields, field.mode) + + @classmethod + def _flatten_recursive(cls, col, prefix=None): + if prefix is None: + prefix = [] + + if len(col.fields) == 0: + prefixed_name = ".".join(prefix + [col.column]) + new_col = BigQueryColumn(prefixed_name, col.dtype, col.fields, + col.mode) + return [new_col] + + new_fields = [] + for field in col.fields: + new_prefix = prefix + [col.column] + new_fields.extend(cls._flatten_recursive(field, new_prefix)) + + return new_fields + + def flatten(self): + return self._flatten_recursive(self) + + @property + def quoted(self): + return '`{}`'.format(self.column) + + def literal(self, value): + return "cast({} as {})".format(value, self.dtype) + + @property + def data_type(self): + if self.dtype.upper() == 'RECORD': + subcols = [ + "{} {}".format(col.name, col.data_type) for col in self.fields + ] + field_type = 'STRUCT<{}>'.format(", ".join(subcols)) + + else: + field_type = self.dtype + + if self.mode.upper() == 'REPEATED': + return 'ARRAY<{}>'.format(field_type) + + else: + return field_type + + def is_string(self): + return self.dtype.lower() == 'string' + + def is_numeric(self): + return False + + def can_expand_to(self, other_column): + """returns True if both columns are strings""" + return self.is_string() and other_column.is_string() + + def __repr__(self): + return "".format(self.name, self.data_type, + self.mode) + + def column_to_bq_schema(self): + """Convert a column to a bigquery schema object. + """ + kwargs = {} + if len(self.fields) > 0: + fields = [field.column_to_bq_schema() for field in self.fields] + kwargs = {"fields": fields} + + return google.cloud.bigquery.SchemaField(self.name, self.dtype, + self.mode, **kwargs) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index d9700f359..1c87ce4dc 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -59,11 +59,21 @@ {% endmacro %} -{% macro bigquery__list_relations_without_caching(database, schema) -%} - {{ return(adapter.list_relations_without_caching(database, schema)) }} +{% macro bigquery__list_relations_without_caching(information_schema, schema) -%} + {{ return(adapter.list_relations_without_caching(information_schema, schema)) }} {% endmacro %} {% macro bigquery__current_timestamp() -%} CURRENT_TIMESTAMP() {%- endmacro %} + + +{% macro bigquery__list_schemas(database) %} + {{ return(adapter.list_schemas()) }} +{% endmacro %} + + +{% macro bigquery__check_schema_exists(information_schema, schema) %} + {{ return(adapter.check_schema_exists(information_schema.database, schema)) }} +{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/archive.sql b/dbt/include/bigquery/macros/materializations/archive.sql index 5548b71a7..7a95f440f 100644 --- a/dbt/include/bigquery/macros/materializations/archive.sql +++ b/dbt/include/bigquery/macros/materializations/archive.sql @@ -4,11 +4,10 @@ {% endmacro %} -{% macro bigquery__archive_scd_hash() %} - to_hex(md5(concat(cast(`dbt_pk` as string), '|', cast(`dbt_updated_at` as string)))) +{% macro bigquery__archive_hash_arguments(args) %} + to_hex(md5(concat({% for arg in args %}coalesce(cast({{ arg }} as string), ''){% if not loop.last %}, '|',{% endif %}{% endfor %}))) {% endmacro %} - {% macro bigquery__create_columns(relation, columns) %} {{ adapter.alter_table_add_columns(relation, columns) }} {% endmacro %} @@ -16,8 +15,8 @@ {% macro bigquery__archive_update(target_relation, tmp_relation) %} update {{ target_relation }} as dest - set dest.{{ adapter.quote('valid_to') }} = tmp.{{ adapter.quote('valid_to') }} + set dest.dbt_valid_to = tmp.dbt_valid_to from {{ tmp_relation }} as tmp - where tmp.{{ adapter.quote('scd_id') }} = dest.{{ adapter.quote('scd_id') }} - and {{ adapter.quote('change_type') }} = 'update'; + where tmp.dbt_scd_id = dest.dbt_scd_id + and change_type = 'update'; {% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index d561ea487..98c65a1dd 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -2,7 +2,6 @@ {% materialization incremental, adapter='bigquery' -%} {%- set unique_key = config.get('unique_key') -%} - {%- set sql_where = config.get('sql_where') -%} {%- set non_destructive_mode = (flags.NON_DESTRUCTIVE == True) -%} {%- set full_refresh_mode = (flags.FULL_REFRESH == True) -%} @@ -34,12 +33,7 @@ {% set source_sql -%} {#-- wrap sql in parens to make it a subquery --#} ( - select * from ( - {{ sql }} - ) - {% if sql_where %} - where ({{ sql_where }}) or ({{ sql_where }}) is null - {% endif %} + {{ sql }} ) {%- endset -%} diff --git a/setup.py b/setup.py index d563e6b36..1d21eba4e 100644 --- a/setup.py +++ b/setup.py @@ -3,7 +3,7 @@ from distutils.core import setup package_name = "dbt-bigquery" -package_version = "0.13.0a1" +package_version = "0.13.0" description = """The bigquery adapter plugin for dbt (data build tool)""" From a37c1bee32f5fbddaaeb4855b05d16de419e8853 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Fri, 3 May 2019 09:22:37 -0400 Subject: [PATCH 032/860] =?UTF-8?q?Bump=20version:=200.13.0=20=E2=86=92=20?= =?UTF-8?q?0.13.1a1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 1d21eba4e..17a5aa56f 100644 --- a/setup.py +++ b/setup.py @@ -3,7 +3,7 @@ from distutils.core import setup package_name = "dbt-bigquery" -package_version = "0.13.0" +package_version = "0.13.1a1" description = """The bigquery adapter plugin for dbt (data build tool)""" From a1cdbd3395b9678697145181b3933b24f518f76b Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Thu, 9 May 2019 15:14:49 -0400 Subject: [PATCH 033/860] 0.13.1a2 bump, include README.md in pypi description --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 17a5aa56f..aad4064fd 100644 --- a/setup.py +++ b/setup.py @@ -3,7 +3,7 @@ from distutils.core import setup package_name = "dbt-bigquery" -package_version = "0.13.1a1" +package_version = "0.13.1a2" description = """The bigquery adapter plugin for dbt (data build tool)""" From 0bf4af2a3390c56ac231e5d7a91fd43d94180072 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Thu, 9 May 2019 15:58:44 -0400 Subject: [PATCH 034/860] update plugin readmes --- README.md | 32 ++++++++++++++++++++++++++++++++ setup.py | 6 +++++- 2 files changed, 37 insertions(+), 1 deletion(-) create mode 100644 README.md diff --git a/README.md b/README.md new file mode 100644 index 000000000..d586baeea --- /dev/null +++ b/README.md @@ -0,0 +1,32 @@ +

+ dbt logo +

+ +**[dbt](https://www.getdbt.com/)** (data build tool) enables data analysts and engineers to transform their data using the same practices that software engineers use to build applications. + +dbt is the T in ELT. Organize, cleanse, denormalize, filter, rename, and pre-aggregate the raw data in your warehouse so that it's ready for analysis. + +## dbt-bigquery + +The `dbt-bigquery` package contains all of the code required to make dbt operate on a BigQuery database. For +more information on using dbt with BigQuery, consult [the docs](https://docs.getdbt.com/docs/profile-bigquery). + + +## Find out more + +- Check out the [Introduction to dbt](https://dbt.readme.io/docs/introduction). +- Read the [dbt Viewpoint](https://dbt.readme.io/docs/viewpoint). + +## Join thousands of analysts in the dbt community + +- Join the [chat](http://slack.getdbt.com/) on Slack. +- Find community posts on [dbt Discourse](https://discourse.getdbt.com). + +## Reporting bugs and contributing code + +- Want to report a bug or request a feature? Let us know on [Slack](http://slack.getdbt.com/), or open [an issue](https://github.com/fishtown-analytics/dbt/issues/new). +- Want to help us build dbt? Check out the [Contributing Getting Started Guide](/CONTRIBUTING.md) + +## Code of Conduct + +Everyone interacting in the dbt project's codebases, issue trackers, chat rooms, and mailing lists is expected to follow the [PyPA Code of Conduct](https://www.pypa.io/en/latest/code-of-conduct/). diff --git a/setup.py b/setup.py index aad4064fd..c92ad1be6 100644 --- a/setup.py +++ b/setup.py @@ -6,12 +6,16 @@ package_version = "0.13.1a2" description = """The bigquery adapter plugin for dbt (data build tool)""" +this_directory = os.path.abspath(os.path.dirname(__file__)) +with open(os.path.join(this_directory, 'README.md'), encoding='utf-8') as f: + long_description = f.read() setup( name=package_name, version=package_version, description=description, - long_description=description, + long_description=long_description, + long_description_content_type='text/markdown', author="Fishtown Analytics", author_email="info@fishtownanalytics.com", url="https://github.com/fishtown-analytics/dbt", From 82cf084c78cd555e7b2b7d688fedfc2c518cfc84 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 9 May 2019 20:21:24 -0600 Subject: [PATCH 035/860] =?UTF-8?q?Bump=20version:=200.13.0=20=E2=86=92=20?= =?UTF-8?q?0.14.0a1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 1d21eba4e..8c781baa5 100644 --- a/setup.py +++ b/setup.py @@ -3,7 +3,7 @@ from distutils.core import setup package_name = "dbt-bigquery" -package_version = "0.13.0" +package_version = "0.14.0a1" description = """The bigquery adapter plugin for dbt (data build tool)""" From 5a3e5944b8d41901666c0487a86428d0ccd26301 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Fri, 10 May 2019 10:19:27 -0400 Subject: [PATCH 036/860] add missing import --- setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/setup.py b/setup.py index c92ad1be6..8fc18fea2 100644 --- a/setup.py +++ b/setup.py @@ -1,6 +1,7 @@ #!/usr/bin/env python from setuptools import find_packages from distutils.core import setup +import os package_name = "dbt-bigquery" package_version = "0.13.1a2" From d1faa8cc21cadc62e8895d7afc4f808ece237731 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Fri, 10 May 2019 10:52:57 -0400 Subject: [PATCH 037/860] py2 compat --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 8fc18fea2..475163f46 100644 --- a/setup.py +++ b/setup.py @@ -8,7 +8,7 @@ description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) -with open(os.path.join(this_directory, 'README.md'), encoding='utf-8') as f: +with open(os.path.join(this_directory, 'README.md')) as f: long_description = f.read() setup( From f0921df4bc1d07d0cf463be333cfde49ea687e85 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Mon, 13 May 2019 11:58:03 -0400 Subject: [PATCH 038/860] =?UTF-8?q?Bump=20version:=200.13.1a2=20=E2=86=92?= =?UTF-8?q?=200.13.1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 475163f46..e9d17673c 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.13.1a2" +package_version = "0.13.1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From f396c6e0f52cbd530da1becf024c67b5ad313739 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Fri, 26 Apr 2019 14:00:42 -0400 Subject: [PATCH 039/860] Implement archival using a merge abstraction --- dbt/adapters/bigquery/impl.py | 16 ---------------- dbt/include/bigquery/macros/adapters.sql | 11 +++++++++++ .../bigquery/macros/materializations/archive.sql | 16 +++------------- 3 files changed, 14 insertions(+), 29 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 8710ccb13..0e8046a26 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -343,22 +343,6 @@ def execute_model(self, model, materialization, sql_override=None, return res - @available.parse(_stub_relation) - def create_temporary_table(self, sql, **kwargs): - # BQ queries always return a temp table with their results - query_job, _ = self.connections.raw_execute(sql) - bq_table = query_job.destination - - return self.Relation.create( - database=bq_table.project, - schema=bq_table.dataset_id, - identifier=bq_table.table_id, - quote_policy={ - 'schema': True, - 'identifier': True - }, - type=BigQueryRelation.Table) - @available.parse_none def alter_table_add_columns(self, relation, columns): diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 1c87ce4dc..83c7926ff 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -34,6 +34,11 @@ create or replace table {{ relation }} {{ partition_by(raw_partition_by) }} {{ cluster_by(raw_cluster_by) }} + {% if temporary %} + OPTIONS( + expiration_timestamp=TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour) + ) + {% endif %} as ( {{ sql }} ); @@ -54,6 +59,12 @@ {{ adapter.drop_schema(database_name, schema_name) }} {% endmacro %} +{% macro bigquery__drop_relation(relation) -%} + {% call statement('drop_relation') -%} + drop {{ relation.type }} if exists {{ relation }} + {%- endcall %} +{% endmacro %} + {% macro bigquery__get_columns_in_relation(relation) -%} {{ return(adapter.get_columns_in_relation(relation)) }} {% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/archive.sql b/dbt/include/bigquery/macros/materializations/archive.sql index 7a95f440f..87b105897 100644 --- a/dbt/include/bigquery/macros/materializations/archive.sql +++ b/dbt/include/bigquery/macros/materializations/archive.sql @@ -1,9 +1,3 @@ -{% macro bigquery__create_temporary_table(sql, relation) %} - {% set tmp_relation = adapter.create_temporary_table(sql) %} - {{ return(tmp_relation) }} -{% endmacro %} - - {% macro bigquery__archive_hash_arguments(args) %} to_hex(md5(concat({% for arg in args %}coalesce(cast({{ arg }} as string), ''){% if not loop.last %}, '|',{% endif %}{% endfor %}))) {% endmacro %} @@ -12,11 +6,7 @@ {{ adapter.alter_table_add_columns(relation, columns) }} {% endmacro %} - -{% macro bigquery__archive_update(target_relation, tmp_relation) %} - update {{ target_relation }} as dest - set dest.dbt_valid_to = tmp.dbt_valid_to - from {{ tmp_relation }} as tmp - where tmp.dbt_scd_id = dest.dbt_scd_id - and change_type = 'update'; +{% macro bigquery__post_archive(staging_relation) %} + -- Clean up the archive temp table + {% do drop_relation(staging_relation) %} {% endmacro %} From d612d645449f6b1c881453c5e88963507184eca5 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Thu, 30 May 2019 11:12:22 -0400 Subject: [PATCH 040/860] fixups --- dbt/include/bigquery/macros/adapters.sql | 27 ++++++++++++------------ 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index c2d969d11..6dc98d2f7 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -26,29 +26,28 @@ {%- endmacro -%} -{% macro table_options() %} - {%- set raw_persist_docs = config.get('persist_docs', {}) -%} +{% macro bigquery_table_options(persist_docs) %} + {% set opts = {} %} - {%- if raw_persist_docs is mapping -%} - {%- set raw_relation = raw_persist_docs.get('relation', false) -%} - OPTIONS( - {%- if raw_relation -%} - description={{ model.description | tojson }} - {% endif %} - ) - {%- else -%} - {{ exceptions.raise_compiler_error("Invalid value provided for 'persist_docs'. Expected dict but got value: " ~ raw_persist_docs) }} + {% set description = get_relation_comment(persist_docs, model) %} + {%- if description is not none -%} + {% do opts.update({'description': "'" ~ description ~ "'"}) %} {% endif %} + + OPTIONS({% for opt_key, opt_val in opts.items() %} + {{ opt_key }}={{ opt_val }} + {% endfor %}) {%- endmacro -%} {% macro bigquery__create_table_as(temporary, relation, sql) -%} {%- set raw_partition_by = config.get('partition_by', none) -%} {%- set raw_cluster_by = config.get('cluster_by', none) -%} + {%- set raw_persist_docs = config.get('persist_docs', {}) -%} create or replace table {{ relation }} {{ partition_by(raw_partition_by) }} {{ cluster_by(raw_cluster_by) }} - {{ table_options() }} + {{ bigquery_table_options(persist_docs=raw_persist_docs) }} as ( {{ sql }} ); @@ -56,8 +55,10 @@ {% macro bigquery__create_view_as(relation, sql) -%} + {%- set raw_persist_docs = config.get('persist_docs', {}) -%} + create or replace view {{ relation }} - {{ table_options() }} + {{ bigquery_table_options(persist_docs=raw_persist_docs) }} as ( {{ sql }} ); From bde900dc8cf04cb07c1485dd393c2fb36cd01e4d Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 5 Jun 2019 08:40:37 -0600 Subject: [PATCH 041/860] archives -> snapshots, except legacy stuff --- .../macros/materializations/{archive.sql => snapshot.sql} | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) rename dbt/include/bigquery/macros/materializations/{archive.sql => snapshot.sql} (70%) diff --git a/dbt/include/bigquery/macros/materializations/archive.sql b/dbt/include/bigquery/macros/materializations/snapshot.sql similarity index 70% rename from dbt/include/bigquery/macros/materializations/archive.sql rename to dbt/include/bigquery/macros/materializations/snapshot.sql index 87b105897..4cd5a0446 100644 --- a/dbt/include/bigquery/macros/materializations/archive.sql +++ b/dbt/include/bigquery/macros/materializations/snapshot.sql @@ -1,4 +1,4 @@ -{% macro bigquery__archive_hash_arguments(args) %} +{% macro bigquery__snapshot_hash_arguments(args) %} to_hex(md5(concat({% for arg in args %}coalesce(cast({{ arg }} as string), ''){% if not loop.last %}, '|',{% endif %}{% endfor %}))) {% endmacro %} @@ -6,7 +6,7 @@ {{ adapter.alter_table_add_columns(relation, columns) }} {% endmacro %} -{% macro bigquery__post_archive(staging_relation) %} - -- Clean up the archive temp table +{% macro bigquery__post_snapshot(staging_relation) %} + -- Clean up the snapshot temp table {% do drop_relation(staging_relation) %} {% endmacro %} From 1d53adb19f5ed4e7eb659a5558e3599398ee28ce Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 12 Jun 2019 20:29:50 -0600 Subject: [PATCH 042/860] list a ton of datasets instead of all to avoid pagination --- dbt/adapters/bigquery/impl.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 0e8046a26..e663decaf 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -91,8 +91,9 @@ def list_schemas(self, database): client = conn.handle with self.connections.exception_handler('list dataset'): + # this is similar to how we have to deal with listing tables all_datasets = client.list_datasets(project=database, - include_all=True) + max_results=10000) return [ds.dataset_id for ds in all_datasets] @available From 57e3e521b234026e93a96ba32d958c2d316bdd60 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 19 Jun 2019 11:33:49 -0600 Subject: [PATCH 043/860] oops, fix bq/sf too --- dbt/adapters/bigquery/relation.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py index e4f982b2c..066eab8a1 100644 --- a/dbt/adapters/bigquery/relation.py +++ b/dbt/adapters/bigquery/relation.py @@ -15,13 +15,14 @@ class BigQueryRelation(BaseRelation): 'quote_policy': { 'database': True, 'schema': True, - 'identifier': True + 'identifier': True, }, 'include_policy': { 'database': True, 'schema': True, - 'identifier': True - } + 'identifier': True, + }, + 'dbt_created': False, } SCHEMA = { @@ -43,9 +44,10 @@ class BigQueryRelation(BaseRelation): 'include_policy': BaseRelation.POLICY_SCHEMA, 'quote_policy': BaseRelation.POLICY_SCHEMA, 'quote_character': {'type': 'string'}, + 'dbt_created': {'type': 'boolean'}, }, 'required': ['metadata', 'type', 'path', 'include_policy', - 'quote_policy', 'quote_character'] + 'quote_policy', 'quote_character', 'dbt_created'] } def matches(self, database=None, schema=None, identifier=None): @@ -60,7 +62,7 @@ def matches(self, database=None, schema=None, identifier=None): pass for k, v in search.items(): - if self.get_path_part(k) != v: + if not self._is_exactish_match(k, v): return False return True From ebc7a3f82e151d1ec8f3f27fc1079a40f3a979ab Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 21 Jun 2019 11:16:11 -0400 Subject: [PATCH 044/860] Remove python 2.7 remove tests for 2.7 remove all dbt.compat remove six assorted 2-removal related cleanups do things that we could not do before due to py2 make super super() classes always derive from obect in 3.x Enum-ify enum-y things azure pipelines -> python 3.7 mock is part of unittest now update freezegun --- dbt/adapters/bigquery/connections.py | 2 +- dbt/adapters/bigquery/impl.py | 10 +++------- dbt/adapters/bigquery/relation.py | 2 +- 3 files changed, 5 insertions(+), 9 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 24731c1a7..f211ba1d2 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -99,7 +99,7 @@ def exception_handler(self, sql): # this sounds a lot like a signal handler and probably has # useful information, so raise it without modification. raise - raise dbt.exceptions.RuntimeException(dbt.compat.to_string(e)) + raise dbt.exceptions.RuntimeException(str(e)) def cancel_open(self): pass diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index e663decaf..e18c3f666 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -2,7 +2,6 @@ import copy -import dbt.compat import dbt.deprecations import dbt.exceptions import dbt.flags as flags @@ -98,8 +97,7 @@ def list_schemas(self, database): @available def check_schema_exists(self, database, schema): - superself = super(BigQueryAdapter, self) - return superself.check_schema_exists(database, schema) + return super().check_schema_exists(database, schema) def get_columns_in_relation(self, relation): try: @@ -149,7 +147,7 @@ def get_relation(self, database, schema, identifier): if self._schema_is_cached(database, schema): # if it's in the cache, use the parent's model of going through # the relations cache and picking out the relation - return super(BigQueryAdapter, self).get_relation( + return super().get_relation( database=database, schema=schema, identifier=identifier @@ -393,9 +391,7 @@ def _flat_columns_in_table(self, table): Resolves child columns as having the name "parent.child". """ for col in self._get_dbt_columns_from_bq_table(table): - flattened = col.flatten() - for subcol in flattened: - yield subcol + yield from col.flatten() @classmethod def _get_stats_column_names(cls): diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py index 066eab8a1..8110adc4c 100644 --- a/dbt/adapters/bigquery/relation.py +++ b/dbt/adapters/bigquery/relation.py @@ -123,7 +123,7 @@ class BigQueryColumn(Column): } def __init__(self, column, dtype, fields=None, mode='NULLABLE'): - super(BigQueryColumn, self).__init__(column, dtype) + super().__init__(column, dtype) if fields is None: fields = [] From c1aa67259e8c7243151f3459fe92a9cdc24c4d56 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Tue, 25 Jun 2019 09:45:09 -0400 Subject: [PATCH 045/860] =?UTF-8?q?Bump=20version:=200.14.0a1=20=E2=86=92?= =?UTF-8?q?=200.14.0a2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 720be3761..583ac52e8 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.14.0a1" +package_version = "0.14.0a2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 6220554e34668830ce0b9c92c8fbd29dc32b19ba Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 25 Jun 2019 09:46:21 -0400 Subject: [PATCH 046/860] Load agate tables at seed materialization time --- dbt/include/bigquery/macros/materializations/seed.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/seed.sql b/dbt/include/bigquery/macros/materializations/seed.sql index 17e1bd989..33e46dbaa 100644 --- a/dbt/include/bigquery/macros/materializations/seed.sql +++ b/dbt/include/bigquery/macros/materializations/seed.sql @@ -1,15 +1,15 @@ -{% macro bigquery__create_csv_table(model) %} +{% macro bigquery__create_csv_table(model, csv_table) %} -- no-op {% endmacro %} -{% macro bigquery__reset_csv_table(model, full_refresh, old_relation) %} +{% macro bigquery__reset_csv_table(model, full_refresh, old_relation, csv_table) %} {{ adapter.drop_relation(old_relation) }} {% endmacro %} -{% macro bigquery__load_csv_rows(model) %} +{% macro bigquery__load_csv_rows(model, csv_table) %} {%- set column_override = model['config'].get('column_types', {}) -%} - {{ adapter.load_dataframe(model['database'], model['schema'], model['alias'], model['agate_table'], column_override) }} + {{ adapter.load_dataframe(model['database'], model['schema'], model['alias'], csv_table, column_override) }} {% endmacro %} From ff3ef2be61d688d516d63ed4957e5c418a212e7e Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 25 Jun 2019 12:48:29 -0400 Subject: [PATCH 047/860] set name back to agate_table in the sql --- dbt/include/bigquery/macros/materializations/seed.sql | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/seed.sql b/dbt/include/bigquery/macros/materializations/seed.sql index 33e46dbaa..d62867374 100644 --- a/dbt/include/bigquery/macros/materializations/seed.sql +++ b/dbt/include/bigquery/macros/materializations/seed.sql @@ -1,15 +1,16 @@ -{% macro bigquery__create_csv_table(model, csv_table) %} +{% macro bigquery__create_csv_table(model, agate_table) %} -- no-op {% endmacro %} -{% macro bigquery__reset_csv_table(model, full_refresh, old_relation, csv_table) %} +{% macro bigquery__reset_csv_table(model, full_refresh, old_relation, agate_table) %} {{ adapter.drop_relation(old_relation) }} {% endmacro %} -{% macro bigquery__load_csv_rows(model, csv_table) %} +{% macro bigquery__load_csv_rows(model, agate_table) %} {%- set column_override = model['config'].get('column_types', {}) -%} - {{ adapter.load_dataframe(model['database'], model['schema'], model['alias'], csv_table, column_override) }} + {{ adapter.load_dataframe(model['database'], model['schema'], model['alias'], + agate_table, column_override) }} {% endmacro %} From 9afea8d7feae88028877d287b9530aa2cd0ad039 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Mon, 1 Jul 2019 15:29:32 -0400 Subject: [PATCH 048/860] =?UTF-8?q?Bump=20version:=200.14.0a2=20=E2=86=92?= =?UTF-8?q?=200.14.0rc1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 583ac52e8..c078d827b 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.14.0a2" +package_version = "0.14.0rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 5b16acae8e3d9ebfc17d3d0698f30c31a5bb31c6 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Tue, 9 Jul 2019 21:44:06 -0400 Subject: [PATCH 049/860] =?UTF-8?q?Bump=20version:=200.14.0rc1=20=E2=86=92?= =?UTF-8?q?=200.14.0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index c078d827b..531a66e49 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.14.0rc1" +package_version = "0.14.0" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 75cbdff10c7bddf0a439d81242f3dda807557ce0 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 24 Jun 2019 09:39:06 -0400 Subject: [PATCH 050/860] Convert dbt to use dataclasses and hologram for representing things Most of the things that previously used manually created jsonschemas Split tests into their own node type Change tests to reflect that tables require a freshness block add a lot more debug-logging on exceptions Make things that get passed to Var() tell it about their vars finally make .empty a property documentation resource type is now a property, not serialized added a Mergeable helper mixin to perform simple merges Convert some oneOf checks into if-else chains to get better errors Add more tests Use "Any" as value in type defs - accept the warning from hologram for now, PR out to suppress it set default values for enabled/materialized Clean up the Parsed/Compiled type hierarchy Allow generic snapshot definitions remove the "graph" entry in the context - This improves performance on large projects significantly Update changelog to reflect removing graph --- dbt/adapters/bigquery/connections.py | 58 +++++++++++----------------- 1 file changed, 23 insertions(+), 35 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index f211ba1d2..21bb21d8f 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -11,40 +11,28 @@ from dbt.adapters.base import BaseConnectionManager, Credentials from dbt.logger import GLOBAL_LOGGER as logger +from hologram.helpers import StrEnum -BIGQUERY_CREDENTIALS_CONTRACT = { - 'type': 'object', - 'additionalProperties': False, - 'properties': { - 'method': { - 'enum': ['oauth', 'service-account', 'service-account-json'], - }, - 'database': { - 'type': 'string', - }, - 'schema': { - 'type': 'string', - }, - 'keyfile': { - 'type': 'string', - }, - 'keyfile_json': { - 'type': 'object', - }, - 'timeout_seconds': { - 'type': 'integer', - }, - 'location': { - 'type': 'string', - }, - }, - 'required': ['method', 'database', 'schema'], -} +from dataclasses import dataclass +from typing import Optional, Any, Dict +class BigQueryConnectionMethod(StrEnum): + OAUTH = 'oauth' + SERVICE_ACCOUNT = 'service-account' + SERVICE_ACCOUNT_JSON = 'service-account-json' + + +@dataclass class BigQueryCredentials(Credentials): - SCHEMA = BIGQUERY_CREDENTIALS_CONTRACT - ALIASES = { + method: BigQueryConnectionMethod + database: str + schema: str + keyfile: Optional[str] = None + keyfile_json: Optional[Dict[str, Any]] = None + timeout_seconds: Optional[int] = 300 + location: Optional[str] = None + _ALIASES = { 'project': 'database', 'dataset': 'schema', } @@ -121,15 +109,15 @@ def get_bigquery_credentials(cls, profile_credentials): method = profile_credentials.method creds = google.oauth2.service_account.Credentials - if method == 'oauth': + if method == BigQueryConnectionMethod.OAUTH: credentials, project_id = google.auth.default(scopes=cls.SCOPE) return credentials - elif method == 'service-account': + elif method == BigQueryConnectionMethod.SERVICE_ACCOUNT: keyfile = profile_credentials.keyfile return creds.from_service_account_file(keyfile, scopes=cls.SCOPE) - elif method == 'service-account-json': + elif method == BigQueryConnectionMethod.SERVICE_ACCOUNT_JSON: details = profile_credentials.keyfile_json return creds.from_service_account_info(details, scopes=cls.SCOPE) @@ -175,8 +163,8 @@ def open(cls, connection): @classmethod def get_timeout(cls, conn): - credentials = conn['credentials'] - return credentials.get('timeout_seconds', cls.QUERY_TIMEOUT) + credentials = conn.credentials + return credentials.timeout_seconds @classmethod def get_table_from_response(cls, resp): From 04bb3809be5166c63cab9b27e6edb776799889aa Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Tue, 16 Jul 2019 23:24:19 -0400 Subject: [PATCH 051/860] possible fix for re-used check cols on BQ --- dbt/adapters/bigquery/connections.py | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 24731c1a7..678b272dc 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -202,15 +202,28 @@ def raw_execute(self, sql, fetch=False): def execute(self, sql, auto_begin=False, fetch=None): # auto_begin is ignored on bigquery, and only included for consistency - _, iterator = self.raw_execute(sql, fetch=fetch) + query_job, iterator = self.raw_execute(sql, fetch=fetch) if fetch: res = self.get_table_from_response(iterator) else: res = dbt.clients.agate_helper.empty_table() - # If we get here, the query succeeded - status = 'OK' + if query_job.statement_type == 'CREATE_VIEW': + status = 'CREATE VIEW' + + elif query_job.statement_type == 'CREATE_TABLE_AS_SELECT': + conn = self.get_thread_connection() + client = conn.handle + table = client.get_table(query_job.destination) + status = 'CREATE TABLE ({})'.format(table.num_rows) + + elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: + status = '{} ({})'.format(query_job.statement_type, query_job.num_dml_affected_rows) + + else: + status = 'OK' + return status, res def create_bigquery_table(self, database, schema, table_name, callback, From f18ce9890fbde81909042731bedbf1436ef22b67 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Tue, 16 Jul 2019 23:43:03 -0400 Subject: [PATCH 052/860] touchup var name and sql formatting --- .../bigquery/macros/materializations/snapshot.sql | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/snapshot.sql b/dbt/include/bigquery/macros/materializations/snapshot.sql index 4cd5a0446..2cfbdb295 100644 --- a/dbt/include/bigquery/macros/materializations/snapshot.sql +++ b/dbt/include/bigquery/macros/materializations/snapshot.sql @@ -1,6 +1,9 @@ -{% macro bigquery__snapshot_hash_arguments(args) %} - to_hex(md5(concat({% for arg in args %}coalesce(cast({{ arg }} as string), ''){% if not loop.last %}, '|',{% endif %}{% endfor %}))) -{% endmacro %} +{% macro bigquery__snapshot_hash_arguments(args) -%} + to_hex(md5(concat({% for arg in args %} + coalesce(cast({{ arg }} as string), ''){% if not loop.last %}, '|',{% endif -%} + {% endfor %} + ))) +{%- endmacro %} {% macro bigquery__create_columns(relation, columns) %} {{ adapter.alter_table_add_columns(relation, columns) }} From 041391fd8cb8000ac9e8ec5a53ae9d0119bf4da4 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Sun, 21 Jul 2019 13:40:43 -0400 Subject: [PATCH 053/860] pep8 --- dbt/adapters/bigquery/connections.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 678b272dc..5743b0aa5 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -219,7 +219,10 @@ def execute(self, sql, auto_begin=False, fetch=None): status = 'CREATE TABLE ({})'.format(table.num_rows) elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: - status = '{} ({})'.format(query_job.statement_type, query_job.num_dml_affected_rows) + status = '{} ({})'.format( + query_job.statement_type, + query_job.num_dml_affected_rows + ) else: status = 'OK' From eb0a2054ffebc5758f061276db8ab796d20592e6 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Mon, 22 Jul 2019 11:14:03 -0400 Subject: [PATCH 054/860] snapshot surrogate key whitespace control --- dbt/include/bigquery/macros/materializations/snapshot.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/snapshot.sql b/dbt/include/bigquery/macros/materializations/snapshot.sql index 2cfbdb295..836a44c8d 100644 --- a/dbt/include/bigquery/macros/materializations/snapshot.sql +++ b/dbt/include/bigquery/macros/materializations/snapshot.sql @@ -1,7 +1,7 @@ {% macro bigquery__snapshot_hash_arguments(args) -%} - to_hex(md5(concat({% for arg in args %} + to_hex(md5(concat({%- for arg in args -%} coalesce(cast({{ arg }} as string), ''){% if not loop.last %}, '|',{% endif -%} - {% endfor %} + {%- endfor -%} ))) {%- endmacro %} From ee0425a41c17a2a51b20b1e0184617bfc677e1b9 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Mon, 29 Jul 2019 22:35:16 -0400 Subject: [PATCH 055/860] (#1626) fix for RPC error with BQ nested fields --- dbt/adapters/bigquery/connections.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 5743b0aa5..ea1e312ad 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -181,8 +181,8 @@ def get_timeout(cls, conn): @classmethod def get_table_from_response(cls, resp): column_names = [field.name for field in resp.schema] - rows = [dict(row.items()) for row in resp] - return dbt.clients.agate_helper.table_from_data(rows, column_names) + return dbt.clients.agate_helper.table_from_data_explicit(resp, + column_names) def raw_execute(self, sql, fetch=False): conn = self.get_thread_connection() From b83adf6d1f821dd3351b1091ee49edf75b672698 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Mon, 29 Jul 2019 23:31:15 -0400 Subject: [PATCH 056/860] serialize as json --- dbt/adapters/bigquery/connections.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index ea1e312ad..1ce850c06 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -181,8 +181,8 @@ def get_timeout(cls, conn): @classmethod def get_table_from_response(cls, resp): column_names = [field.name for field in resp.schema] - return dbt.clients.agate_helper.table_from_data_explicit(resp, - column_names) + return dbt.clients.agate_helper.table_from_data_flat(resp, + column_names) def raw_execute(self, sql, fetch=False): conn = self.get_thread_connection() From 8a959ced647b852149cf2f00a62d5c091ae6e096 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 19 Jul 2019 14:50:28 -0400 Subject: [PATCH 057/860] Make database/schema mandatory for all credentials --- dbt/adapters/bigquery/connections.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index abfe3e382..a73785519 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -26,8 +26,6 @@ class BigQueryConnectionMethod(StrEnum): @dataclass class BigQueryCredentials(Credentials): method: BigQueryConnectionMethod - database: str - schema: str keyfile: Optional[str] = None keyfile_json: Optional[Dict[str, Any]] = None timeout_seconds: Optional[int] = 300 From fabb4923542c5f49e12d687acf605401b5c6ab5c Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 23 Jul 2019 19:46:26 -0600 Subject: [PATCH 058/860] I can't live like this anymore, bumped to 0.15.0a1 --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 531a66e49..eeb2ff402 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.14.0" +package_version = "0.15.0a1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 6fb432b2198f91f84c52f193cc4331d25dfa5382 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 17 Jul 2019 14:45:22 -0400 Subject: [PATCH 059/860] Partial parsing Refactor parsing Store files in the manifest some speed improvements test fixes Remove some old python 2 compatibility stuff --- dbt/__init__.py | 1 - dbt/adapters/__init__.py | 1 - dbt/include/__init__.py | 1 - 3 files changed, 3 deletions(-) delete mode 100644 dbt/__init__.py delete mode 100644 dbt/adapters/__init__.py delete mode 100644 dbt/include/__init__.py diff --git a/dbt/__init__.py b/dbt/__init__.py deleted file mode 100644 index 69e3be50d..000000000 --- a/dbt/__init__.py +++ /dev/null @@ -1 +0,0 @@ -__path__ = __import__('pkgutil').extend_path(__path__, __name__) diff --git a/dbt/adapters/__init__.py b/dbt/adapters/__init__.py deleted file mode 100644 index 69e3be50d..000000000 --- a/dbt/adapters/__init__.py +++ /dev/null @@ -1 +0,0 @@ -__path__ = __import__('pkgutil').extend_path(__path__, __name__) diff --git a/dbt/include/__init__.py b/dbt/include/__init__.py deleted file mode 100644 index 69e3be50d..000000000 --- a/dbt/include/__init__.py +++ /dev/null @@ -1 +0,0 @@ -__path__ = __import__('pkgutil').extend_path(__path__, __name__) From 004a640146a5ba9874f038e9593b5820363ec8e1 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 31 Jul 2019 08:37:27 -0600 Subject: [PATCH 060/860] PR feedback fix windows tests mypy + added more stubs for easy modules remove __future__ imports from python2 gitignore --- dbt/adapters/bigquery/impl.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index e18c3f666..ab9eefef3 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -1,5 +1,3 @@ -from __future__ import absolute_import - import copy import dbt.deprecations From cc33289fbf096f9429c826715bec580b7bebd454 Mon Sep 17 00:00:00 2001 From: Stephen Date: Thu, 8 Aug 2019 21:50:29 +0100 Subject: [PATCH 061/860] Support job priority in BigQuery --- dbt/adapters/bigquery/connections.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 1ce850c06..d8d592d79 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -37,6 +37,9 @@ 'location': { 'type': 'string', }, + 'priority': { + 'enum': ['interactive', 'batch'], + }, }, 'required': ['method', 'database', 'schema'], } @@ -192,6 +195,9 @@ def raw_execute(self, sql, fetch=False): job_config = google.cloud.bigquery.QueryJobConfig() job_config.use_legacy_sql = False + priority = conn.credentials.get('priority', 'interactive') + job_config.priority = google.cloud.bigquery.QueryPriority.BATCH if priority == 'batch' \ + else google.cloud.bigquery.QueryPriority.INTERACTIVE query_job = client.query(sql, job_config) # this blocks until the query has completed From 29464eba16183d3ebae3e2c2dddd81ad6ace7596 Mon Sep 17 00:00:00 2001 From: Stephen Date: Fri, 9 Aug 2019 23:22:15 +0100 Subject: [PATCH 062/860] Add unit test for priority --- dbt/adapters/bigquery/connections.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index d8d592d79..761b32ceb 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -195,9 +195,14 @@ def raw_execute(self, sql, fetch=False): job_config = google.cloud.bigquery.QueryJobConfig() job_config.use_legacy_sql = False + priority = conn.credentials.get('priority', 'interactive') - job_config.priority = google.cloud.bigquery.QueryPriority.BATCH if priority == 'batch' \ - else google.cloud.bigquery.QueryPriority.INTERACTIVE + if priority == "batch": + job_config.priority = google.cloud.bigquery.QueryPriority.BATCH + else: + job_config.priority = \ + google.cloud.bigquery.QueryPriority.INTERACTIVE + query_job = client.query(sql, job_config) # this blocks until the query has completed From fcc333e4072cecffe41c7b3ba9a839b60d097d07 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Mon, 12 Aug 2019 20:18:30 -0400 Subject: [PATCH 063/860] =?UTF-8?q?Bump=20version:=200.14.0=20=E2=86=92=20?= =?UTF-8?q?0.14.1a1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 531a66e49..c4029f58b 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.14.0" +package_version = "0.14.1a1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From acbe25eb6c7224607ff7a5f7e1f6f0c0c6820b99 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Fri, 16 Aug 2019 15:18:00 -0400 Subject: [PATCH 064/860] (#1687) no-op column expansion on Snowflake + BQ --- dbt/adapters/bigquery/impl.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index e663decaf..fb52ed880 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -118,6 +118,10 @@ def expand_column_types(self, goal, current): # This is a no-op on BigQuery pass + def expand_target_column_types(self, from_relation, to_relation): + # This is a no-op on BigQuery + pass + def list_relations_without_caching(self, information_schema, schema): connection = self.connections.get_thread_connection() client = connection.handle From 423f2fdf6447725d150e701b2d17a0a790cf32be Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Thu, 22 Aug 2019 12:02:16 -0400 Subject: [PATCH 065/860] =?UTF-8?q?Bump=20version:=200.14.1a1=20=E2=86=92?= =?UTF-8?q?=200.14.1rc1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index c4029f58b..53310b7f8 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.14.1a1" +package_version = "0.14.1rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From ddbf46d735f2f5c2dc5e057340ab90e2288b3dcd Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Wed, 28 Aug 2019 18:08:14 -0400 Subject: [PATCH 066/860] =?UTF-8?q?Bump=20version:=200.14.1rc1=20=E2=86=92?= =?UTF-8?q?=200.14.1rc2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 53310b7f8..446146ac6 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.14.1rc1" +package_version = "0.14.1rc2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 20be529ceadfd6f514fc7fa76349cea2dff986cf Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 27 Aug 2019 17:11:50 -0600 Subject: [PATCH 067/860] Replace logging with logbook Log rpc as json, preserve existing log behavior Force werkzeug logs into logbook handling Make rpc a package and set up mypy checking on it fix a lot of logging-related bugs Provide a log manager to toggle various log functionalities add disabled flag to file handler for "dbt debug" and friends Test changes to support logbook lots of fiddly process/thread handling --- dbt/adapters/bigquery/connections.py | 2 +- dbt/adapters/bigquery/impl.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index cc93f43c3..d27aa0fc8 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -179,7 +179,7 @@ def raw_execute(self, sql, fetch=False): conn = self.get_thread_connection() client = conn.handle - logger.debug('On %s: %s', conn.name, sql) + logger.debug('On {}: {}', conn.name, sql) job_config = google.cloud.bigquery.QueryJobConfig() job_config.use_legacy_sql = False diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 741c55718..2bf73db55 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -162,11 +162,11 @@ def get_relation(self, database, schema, identifier): return self._bq_table_to_relation(table) def create_schema(self, database, schema): - logger.debug('Creating schema "%s.%s".', database, schema) + logger.debug('Creating schema "{}.{}".', database, schema) self.connections.create_dataset(database, schema) def drop_schema(self, database, schema): - logger.debug('Dropping schema "%s.%s".', database, schema) + logger.debug('Dropping schema "{}.{}".', database, schema) if not self.check_schema_exists(database, schema): return From ac630e95fb43e1d9cd98940cea3f1063568f765b Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Tue, 3 Sep 2019 19:52:54 -0400 Subject: [PATCH 068/860] =?UTF-8?q?Bump=20version:=200.14.1rc2=20=E2=86=92?= =?UTF-8?q?=200.14.1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 446146ac6..cfa2a848e 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.14.1rc2" +package_version = "0.14.1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From b5f6217571869e66de142cf046a6d7181f00ee80 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Thu, 12 Sep 2019 15:58:43 -0400 Subject: [PATCH 069/860] =?UTF-8?q?Bump=20version:=200.14.1=20=E2=86=92=20?= =?UTF-8?q?0.14.2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index cfa2a848e..d540eb5b2 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.14.1" +package_version = "0.14.2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From d82f939183002c4ffc2f973c66f0b3bc17e5e656 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 11 Sep 2019 11:35:46 -0600 Subject: [PATCH 070/860] bigquery error handling: log strings, not Exception objects --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index d27aa0fc8..00df1a82a 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -60,7 +60,7 @@ class BigQueryConnectionManager(BaseConnectionManager): @classmethod def handle_error(cls, error, message, sql): logger.debug(message.format(sql=sql)) - logger.debug(error) + logger.debug(str(error)) error_msg = "\n".join( [item['message'] for item in error.errors]) From d23ace02a305610d35757db7afcf3032d14a345e Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 17 Sep 2019 10:12:12 -0600 Subject: [PATCH 071/860] expose the cache to macros make new cache manipulation methods mark the methods available to the sql context move cache manipulation into appropriate macros and methods update the changelog fix some type checking --- dbt/adapters/bigquery/impl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 2bf73db55..fe793ad5f 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -62,7 +62,7 @@ def is_cancelable(cls): def drop_relation(self, relation): is_cached = self._schema_is_cached(relation.database, relation.schema) if is_cached: - self.cache.drop(relation) + self.cache_dropped(relation) conn = self.connections.get_thread_connection() client = conn.handle From 7b169b19d6bc39464a88985fcfc118da49b43c24 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 20 Sep 2019 13:58:36 -0600 Subject: [PATCH 072/860] Add support for a materialization return value Materializations now return a list of relations they added - those get added to the cache Updated existing materializations Added backwards compatibility support + a deprecation warning --- dbt/include/bigquery/macros/materializations/incremental.sql | 2 ++ dbt/include/bigquery/macros/materializations/table.sql | 2 ++ dbt/include/bigquery/macros/materializations/view.sql | 2 +- 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index fc84c11d9..7c8525fcc 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -49,4 +49,6 @@ {{ run_hooks(post_hooks) }} + {{ return({'relations': [target_relation]}) }} + {%- endmaterialization %} diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql index 378b1c2ec..8ac3956ff 100644 --- a/dbt/include/bigquery/macros/materializations/table.sql +++ b/dbt/include/bigquery/macros/materializations/table.sql @@ -70,4 +70,6 @@ {{ run_hooks(post_hooks) }} + {{ return({'relations': [target_relation]}) }} + {% endmaterialization %} diff --git a/dbt/include/bigquery/macros/materializations/view.sql b/dbt/include/bigquery/macros/materializations/view.sql index 7821f3e12..04cac1c1c 100644 --- a/dbt/include/bigquery/macros/materializations/view.sql +++ b/dbt/include/bigquery/macros/materializations/view.sql @@ -9,5 +9,5 @@ {% materialization view, adapter='bigquery' -%} - {{ create_or_replace_view(run_outside_transaction_hooks=False) }} + {{ return(create_or_replace_view(run_outside_transaction_hooks=False)) }} {%- endmaterialization %} From 492fbdda93607a2944266eea854d6402243b4579 Mon Sep 17 00:00:00 2001 From: Darren Haken Date: Fri, 27 Sep 2019 15:36:21 +0100 Subject: [PATCH 073/860] Add trailing comma when creating BQ options to each list item - This is a bug in the existing code, BQ needs commas between option items. Discovered it when creating a temp table using dbt_utils --- dbt/include/bigquery/macros/adapters.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 42082b03a..60d5d4e50 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -38,7 +38,7 @@ {% endif %} OPTIONS({% for opt_key, opt_val in opts.items() %} - {{ opt_key }}={{ opt_val }} + {{ opt_key }}={{ opt_val }}{{ "," if not loop.last }} {% endfor %}) {%- endmacro -%} From c13483b9409f627710ddd3f0084641fb1388bcfb Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 24 Sep 2019 09:40:55 -0600 Subject: [PATCH 074/860] Convert Relation types to hologram.JsonSchemaMixin Fix a lot of mypy things, add a number of adapter-ish modules to it Split relations and columns into separate files split context.common into base + common - base is all that's required for the config renderer Move Credentials into connection contracts since that's what they really are Removed model_name/table_name -> consolidated to identifier - I hope I did not break seeds, which claimed to care about render(False) Unify shared 'external' relation type with bigquery's own hack workarounds for some import cycles with plugin registration and config p arsing Assorted backwards compatibility fixes around types, deep_merge vs shallow merge Remove APIObject --- dbt/adapters/bigquery/__init__.py | 2 +- dbt/adapters/bigquery/column.py | 121 ++++++++++++++++ dbt/adapters/bigquery/connections.py | 2 +- dbt/adapters/bigquery/impl.py | 14 +- dbt/adapters/bigquery/relation.py | 209 +++------------------------ 5 files changed, 151 insertions(+), 197 deletions(-) create mode 100644 dbt/adapters/bigquery/column.py diff --git a/dbt/adapters/bigquery/__init__.py b/dbt/adapters/bigquery/__init__.py index c45656772..daff48a32 100644 --- a/dbt/adapters/bigquery/__init__.py +++ b/dbt/adapters/bigquery/__init__.py @@ -1,7 +1,7 @@ from dbt.adapters.bigquery.connections import BigQueryConnectionManager # noqa from dbt.adapters.bigquery.connections import BigQueryCredentials from dbt.adapters.bigquery.relation import BigQueryRelation # noqa -from dbt.adapters.bigquery.relation import BigQueryColumn # noqa +from dbt.adapters.bigquery.column import BigQueryColumn # noqa from dbt.adapters.bigquery.impl import BigQueryAdapter from dbt.adapters.base import AdapterPlugin diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py new file mode 100644 index 000000000..8c8a442b4 --- /dev/null +++ b/dbt/adapters/bigquery/column.py @@ -0,0 +1,121 @@ +from dataclasses import dataclass +from typing import Optional, List, TypeVar, Iterable, Type + +from dbt.adapters.base.column import Column + +from google.cloud.bigquery import SchemaField + +Self = TypeVar('Self', bound='BigQueryColumn') + + +@dataclass(init=False) +class BigQueryColumn(Column): + TYPE_LABELS = { + 'STRING': 'STRING', + 'TIMESTAMP': 'TIMESTAMP', + 'FLOAT': 'FLOAT64', + 'INTEGER': 'INT64', + 'RECORD': 'RECORD', + } + fields: List[Self] + mode: str + + def __init__( + self, + column: str, + dtype: str, + fields: Optional[Iterable[SchemaField]] = None, + mode: str = 'NULLABLE', + ) -> None: + super().__init__(column, dtype) + + if fields is None: + fields = [] + + self.fields = self.wrap_subfields(fields) + self.mode = mode + + @classmethod + def wrap_subfields( + cls: Type[Self], fields: Iterable[SchemaField] + ) -> List[Self]: + return [cls.create_from_field(field) for field in fields] + + @classmethod + def create_from_field(cls: Type[Self], field: SchemaField) -> Self: + return cls( + field.name, + cls.translate_type(field.field_type), + field.fields, + field.mode, + ) + + @classmethod + def _flatten_recursive( + cls: Type[Self], col: Self, prefix: Optional[str] = None + ) -> List[Self]: + if prefix is None: + prefix = [] + + if len(col.fields) == 0: + prefixed_name = ".".join(prefix + [col.column]) + new_col = cls(prefixed_name, col.dtype, col.fields, col.mode) + return [new_col] + + new_fields = [] + for field in col.fields: + new_prefix = prefix + [col.column] + new_fields.extend(cls._flatten_recursive(field, new_prefix)) + + return new_fields + + def flatten(self): + return self._flatten_recursive(self) + + @property + def quoted(self): + return '`{}`'.format(self.column) + + def literal(self, value): + return "cast({} as {})".format(value, self.dtype) + + @property + def data_type(self) -> str: + if self.dtype.upper() == 'RECORD': + subcols = [ + "{} {}".format(col.name, col.data_type) for col in self.fields + ] + field_type = 'STRUCT<{}>'.format(", ".join(subcols)) + + else: + field_type = self.dtype + + if self.mode.upper() == 'REPEATED': + return 'ARRAY<{}>'.format(field_type) + + else: + return field_type + + def is_string(self) -> bool: + return self.dtype.lower() == 'string' + + def is_numeric(self) -> bool: + return False + + def can_expand_to(self: Self, other_column: Self) -> bool: + """returns True if both columns are strings""" + return self.is_string() and other_column.is_string() + + def __repr__(self) -> str: + return "".format(self.name, self.data_type, + self.mode) + + def column_to_bq_schema(self) -> SchemaField: + """Convert a column to a bigquery schema object. + """ + kwargs = {} + if len(self.fields) > 0: + fields = [field.column_to_bq_schema() for field in self.fields] + kwargs = {"fields": fields} + + return SchemaField(self.name, self.dtype, self.mode, **kwargs) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 00df1a82a..b1bf8b44e 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -92,7 +92,7 @@ def exception_handler(self, sql): raise raise dbt.exceptions.RuntimeException(str(e)) - def cancel_open(self): + def cancel_open(self) -> None: pass @classmethod diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index fe793ad5f..596aff2a6 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -6,8 +6,10 @@ import dbt.clients.gcloud import dbt.clients.agate_helper -from dbt.adapters.base import BaseAdapter, available -from dbt.adapters.bigquery import BigQueryRelation +from dbt.adapters.base import BaseAdapter, available, RelationType +from dbt.adapters.bigquery.relation import ( + BigQueryRelation +) from dbt.adapters.bigquery import BigQueryColumn from dbt.adapters.bigquery import BigQueryConnectionManager from dbt.contracts.connection import Connection @@ -36,9 +38,9 @@ def _stub_relation(*args, **kwargs): class BigQueryAdapter(BaseAdapter): RELATION_TYPES = { - 'TABLE': BigQueryRelation.Table, - 'VIEW': BigQueryRelation.View, - 'EXTERNAL': BigQueryRelation.External + 'TABLE': RelationType.Table, + 'VIEW': RelationType.View, + 'EXTERNAL': RelationType.External } Relation = BigQueryRelation @@ -102,7 +104,7 @@ def get_columns_in_relation(self, relation): table = self.connections.get_bq_table( database=relation.database, schema=relation.schema, - identifier=relation.table_name + identifier=relation.identifier ) return self._get_dbt_columns_from_bq_table(table) diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py index 8110adc4c..509ae8e4e 100644 --- a/dbt/adapters/bigquery/relation.py +++ b/dbt/adapters/bigquery/relation.py @@ -1,60 +1,26 @@ -from dbt.adapters.base.relation import BaseRelation, Column -from dbt.utils import filter_null_values +from dataclasses import dataclass +from typing import Optional -import google.cloud.bigquery +from dbt.adapters.base.relation import ( + BaseRelation, ComponentName +) +from dbt.utils import filter_null_values +@dataclass(frozen=True, eq=False, repr=False) class BigQueryRelation(BaseRelation): - External = "external" - - DEFAULTS = { - 'metadata': { - 'type': 'BigQueryRelation' - }, - 'quote_character': '`', - 'quote_policy': { - 'database': True, - 'schema': True, - 'identifier': True, - }, - 'include_policy': { - 'database': True, - 'schema': True, - 'identifier': True, - }, - 'dbt_created': False, - } - - SCHEMA = { - 'type': 'object', - 'properties': { - 'metadata': { - 'type': 'object', - 'properties': { - 'type': { - 'type': 'string', - 'const': 'BigQueryRelation', - }, - }, - }, - 'type': { - 'enum': BaseRelation.RelationTypes + [External, None], - }, - 'path': BaseRelation.PATH_SCHEMA, - 'include_policy': BaseRelation.POLICY_SCHEMA, - 'quote_policy': BaseRelation.POLICY_SCHEMA, - 'quote_character': {'type': 'string'}, - 'dbt_created': {'type': 'boolean'}, - }, - 'required': ['metadata', 'type', 'path', 'include_policy', - 'quote_policy', 'quote_character', 'dbt_created'] - } - - def matches(self, database=None, schema=None, identifier=None): + quote_character: str = '`' + + def matches( + self, + database: Optional[str] = None, + schema: Optional[str] = None, + identifier: Optional[str] = None, + ) -> bool: search = filter_null_values({ - 'database': database, - 'schema': schema, - 'identifier': identifier + ComponentName.Database: database, + ComponentName.Schema: schema, + ComponentName.Identifier: identifier }) if not search: @@ -67,145 +33,10 @@ def matches(self, database=None, schema=None, identifier=None): return True - @classmethod - def create(cls, database=None, schema=None, - identifier=None, table_name=None, - type=None, **kwargs): - if table_name is None: - table_name = identifier - - return cls(type=type, - path={ - 'database': database, - 'schema': schema, - 'identifier': identifier - }, - table_name=table_name, - **kwargs) - - def quote(self, database=None, schema=None, identifier=None): - policy = filter_null_values({ - 'database': database, - 'schema': schema, - 'identifier': identifier - }) - - return self.incorporate(quote_policy=policy) - - @property - def database(self): - return self.path.get('database') - @property def project(self): - return self.path.get('database') - - @property - def schema(self): - return self.path.get('schema') + return self.database @property def dataset(self): - return self.path.get('schema') - - @property - def identifier(self): - return self.path.get('identifier') - - -class BigQueryColumn(Column): - TYPE_LABELS = { - 'STRING': 'STRING', - 'TIMESTAMP': 'TIMESTAMP', - 'FLOAT': 'FLOAT64', - 'INTEGER': 'INT64', - 'RECORD': 'RECORD', - } - - def __init__(self, column, dtype, fields=None, mode='NULLABLE'): - super().__init__(column, dtype) - - if fields is None: - fields = [] - - self.fields = self.wrap_subfields(fields) - self.mode = mode - - @classmethod - def wrap_subfields(cls, fields): - return [BigQueryColumn.create_from_field(field) for field in fields] - - @classmethod - def create_from_field(cls, field): - return BigQueryColumn(field.name, cls.translate_type(field.field_type), - field.fields, field.mode) - - @classmethod - def _flatten_recursive(cls, col, prefix=None): - if prefix is None: - prefix = [] - - if len(col.fields) == 0: - prefixed_name = ".".join(prefix + [col.column]) - new_col = BigQueryColumn(prefixed_name, col.dtype, col.fields, - col.mode) - return [new_col] - - new_fields = [] - for field in col.fields: - new_prefix = prefix + [col.column] - new_fields.extend(cls._flatten_recursive(field, new_prefix)) - - return new_fields - - def flatten(self): - return self._flatten_recursive(self) - - @property - def quoted(self): - return '`{}`'.format(self.column) - - def literal(self, value): - return "cast({} as {})".format(value, self.dtype) - - @property - def data_type(self): - if self.dtype.upper() == 'RECORD': - subcols = [ - "{} {}".format(col.name, col.data_type) for col in self.fields - ] - field_type = 'STRUCT<{}>'.format(", ".join(subcols)) - - else: - field_type = self.dtype - - if self.mode.upper() == 'REPEATED': - return 'ARRAY<{}>'.format(field_type) - - else: - return field_type - - def is_string(self): - return self.dtype.lower() == 'string' - - def is_numeric(self): - return False - - def can_expand_to(self, other_column): - """returns True if both columns are strings""" - return self.is_string() and other_column.is_string() - - def __repr__(self): - return "".format(self.name, self.data_type, - self.mode) - - def column_to_bq_schema(self): - """Convert a column to a bigquery schema object. - """ - kwargs = {} - if len(self.fields) > 0: - fields = [field.column_to_bq_schema() for field in self.fields] - kwargs = {"fields": fields} - - return google.cloud.bigquery.SchemaField(self.name, self.dtype, - self.mode, **kwargs) + return self.schema From cbf69f4cfd9763b58e7b1472770950a16a6ea154 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Mon, 14 Oct 2019 21:42:25 -0400 Subject: [PATCH 075/860] (#1576) use the information schema on BigQuery --- dbt/adapters/bigquery/impl.py | 153 ++++-------------- dbt/include/bigquery/macros/catalog.sql | 202 ++++++++++++++++++++++++ 2 files changed, 231 insertions(+), 124 deletions(-) create mode 100644 dbt/include/bigquery/macros/catalog.sql diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 596aff2a6..5ca6c2e57 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -1,5 +1,3 @@ -import copy - import dbt.deprecations import dbt.exceptions import dbt.flags as flags @@ -25,6 +23,9 @@ import agate +GET_CATALOG_MACRO_NAME = 'get_catalog' + + def _stub_relation(*args, **kwargs): return BigQueryRelation.create( database='', @@ -387,127 +388,31 @@ def load_dataframe(self, database, schema, table_name, agate_table, with self.connections.exception_handler("LOAD TABLE"): self.poll_until_job_completes(job, timeout) - ### - # The get_catalog implementation for bigquery - ### - def _flat_columns_in_table(self, table): - """An iterator over the flattened columns for a given schema and table. - Resolves child columns as having the name "parent.child". - """ - for col in self._get_dbt_columns_from_bq_table(table): - yield from col.flatten() - - @classmethod - def _get_stats_column_names(cls): - """Construct a tuple of the column names for stats. Each stat has 4 - columns of data. - """ - columns = [] - stats = ('num_bytes', 'num_rows', 'location', 'partitioning_type', - 'clustering_fields') - stat_components = ('label', 'value', 'description', 'include') - for stat_id in stats: - for stat_component in stat_components: - columns.append('stats:{}:{}'.format(stat_id, stat_component)) - return tuple(columns) - - @classmethod - def _get_stats_columns(cls, table, relation_type): - """Given a table, return an iterator of key/value pairs for stats - column names/values. - """ - column_names = cls._get_stats_column_names() - - # agate does not handle the array of column names gracefully - clustering_value = None - if table.clustering_fields is not None: - clustering_value = ','.join(table.clustering_fields) - # cast num_bytes/num_rows to str before they get to agate, or else - # agate will incorrectly decide they are booleans. - column_values = ( - 'Number of bytes', - str(table.num_bytes), - 'The number of bytes this table consumes', - relation_type == 'table', - - 'Number of rows', - str(table.num_rows), - 'The number of rows in this table', - relation_type == 'table', - - 'Location', - table.location, - 'The geographic location of this table', - True, - - 'Partitioning Type', - table.partitioning_type, - 'The partitioning type used for this table', - relation_type == 'table', - - 'Clustering Fields', - clustering_value, - 'The clustering fields for this table', - relation_type == 'table', - ) - return zip(column_names, column_values) - def get_catalog(self, manifest): - connection = self.connections.get_thread_connection() - client = connection.handle - - schemas = manifest.get_used_schemas() - - column_names = ( - 'table_database', - 'table_schema', - 'table_name', - 'table_type', - 'table_comment', - # does not exist in bigquery, but included for consistency - 'table_owner', - 'column_name', - 'column_index', - 'column_type', - 'column_comment', - ) - all_names = column_names + self._get_stats_column_names() - columns = [] + """Get the catalog for this manifest by running the get catalog macro. + Returns an agate.Table of catalog information. + """ - for database_name, schema_name in schemas: - relations = self.list_relations(database_name, schema_name) - for relation in relations: - - # This relation contains a subset of the info we care about. - # Fetch the full table object here - table_ref = self.connections.table_ref( - database_name, - relation.schema, - relation.identifier, - connection - ) - table = client.get_table(table_ref) - - flattened = self._flat_columns_in_table(table) - relation_stats = dict(self._get_stats_columns(table, - relation.type)) - - for index, column in enumerate(flattened, start=1): - column_data = ( - relation.database, - relation.schema, - relation.name, - relation.type, - None, - None, - column.name, - index, - column.data_type, - None, - ) - column_dict = dict(zip(column_names, column_data)) - column_dict.update(copy.deepcopy(relation_stats)) - - columns.append(column_dict) - - return dbt.clients.agate_helper.table_from_data(columns, all_names) + information_schemas = [] + for database, schema in manifest.get_used_schemas(): + information_schemas.append(self.Relation.create( + database=database, + schema=schema, + quote_policy={ + 'database': True, + 'schema': True + } + )) + + # make it a list so macros can index into it. + kwargs = {'information_schemas': information_schemas} + table = self.execute_macro(GET_CATALOG_MACRO_NAME, + kwargs=kwargs, + release=True) + + # BigQuery doesn't allow ":" chars in column names -- remap them here. + table = table.rename(column_names={ + col.name: col.name.replace('__', ':') for col in table.columns + }) + + return self._catalog_filter_table(table, manifest) diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql new file mode 100644 index 000000000..86b0e352e --- /dev/null +++ b/dbt/include/bigquery/macros/catalog.sql @@ -0,0 +1,202 @@ + +{% macro bigquery__get_catalog(information_schemas) -%} + + {%- call statement('catalog', fetch_result=True) -%} + {% for information_schema in information_schemas %} + ( + with tables as ( + select + project_id as table_database, + dataset_id as table_schema, + table_id as original_table_name, + + concat(project_id, '.', dataset_id, '.', table_id) as relation_id, + + row_count, + size_bytes as size_bytes, + case + when type = 1 then 'table' + when type = 2 then 'view' + else concat('unknown (', cast(type as string), ')') + end as table_type, + + REGEXP_CONTAINS(table_id, '^.+[0-9]{8}$') and type = 1 as is_date_shard, + REGEXP_EXTRACT(table_id, '^(.+)[0-9]{8}$') as shard_base_name, + REGEXP_EXTRACT(table_id, '^.+([0-9]{8})$') as shard_name + + from {{ information_schema }}.__TABLES__ + + ), + + extracted as ( + + select *, + case + when is_date_shard then shard_base_name + else original_table_name + end as table_name + + from tables + + ), + + unsharded_tables as ( + + select + table_database, + table_schema, + table_name, + table_type, + is_date_shard, + + struct( + min(shard_name) as shard_min, + max(shard_name) as shard_max, + count(*) as shard_count + ) as table_shards, + + sum(size_bytes) as size_bytes, + sum(row_count) as row_count, + + max(relation_id) as relation_id + + from extracted + group by 1,2,3,4,5 + + ), + + info_schema_columns as ( + + select + concat(table_catalog, '.', table_schema, '.', table_name) as relation_id, + table_catalog as table_database, + table_schema, + table_name, + + -- use the "real" column name from the paths query below + column_name as base_column_name, + ordinal_position as column_index, + cast(null as string) as column_comment, + + is_partitioning_column, + clustering_ordinal_position + + from {{ information_schema }}.INFORMATION_SCHEMA.COLUMNS + where ordinal_position is not null + + ), + + info_schema_column_paths as ( + + select + concat(table_catalog, '.', table_schema, '.', table_name) as relation_id, + field_path as column_name, + data_type as column_type, + column_name as base_column_name + + from {{ information_schema }}.INFORMATION_SCHEMA.COLUMN_FIELD_PATHS + where data_type not like 'STRUCT%' + + ), + + columns as ( + + select * except (base_column_name) + from info_schema_columns + join info_schema_column_paths using (relation_id, base_column_name) + + ), + + column_stats as ( + + select + table_database, + table_schema, + table_name, + max(relation_id) as relation_id, + max(case when is_partitioning_column = 'YES' then 1 else 0 end) = 1 as is_partitioned, + max(case when is_partitioning_column = 'YES' then column_name else null end) as partition_column, + max(case when clustering_ordinal_position is not null then 1 else 0 end) = 1 as is_clustered, + array_to_string( + array_agg( + case + when clustering_ordinal_position is not null then column_name + else null + end ignore nulls + order by clustering_ordinal_position + ), ', ' + ) as clustering_columns + + from columns + group by 1,2,3 + + ) + + select + unsharded_tables.table_database, + unsharded_tables.table_schema, + case + when is_date_shard then concat(unsharded_tables.table_name, '*') + else unsharded_tables.table_name + end as table_name, + unsharded_tables.table_type, + + columns.column_name, + -- invent a row number to account for nested fields -- BQ does + -- not treat these nested properties as independent fields + row_number() over ( + partition by relation_id + order by columns.column_index, columns.column_name + ) as column_index, + columns.column_type, + columns.column_comment, + + 'Shard count' as `stats__date_shards__label`, + table_shards.shard_count as `stats__date_shards__value`, + 'The number of date shards in this table' as `stats__date_shards__description`, + is_date_shard as `stats__date_shards__include`, + + 'Shard (min)' as `stats__date_shard_min__label`, + table_shards.shard_min as `stats__date_shard_min__value`, + 'The first date shard in this table' as `stats__date_shard_min__description`, + is_date_shard as `stats__date_shard_min__include`, + + 'Shard (max)' as `stats__date_shard_max__label`, + table_shards.shard_max as `stats__date_shard_max__value`, + 'The last date shard in this table' as `stats__date_shard_max__description`, + is_date_shard as `stats__date_shard_max__include`, + + '# Rows' as `stats__num_rows__label`, + row_count as `stats__num_rows__value`, + 'Approximate count of rows in this table' as `stats__num_rows__description`, + (unsharded_tables.table_type = 'table') as `stats__num_rows__include`, + + 'Approximate Size' as `stats__num_bytes__label`, + size_bytes as `stats__num_bytes__value`, + 'Approximate size of table as reported by BigQuery' as `stats__num_bytes__description`, + (unsharded_tables.table_type = 'table') as `stats__num_bytes__include`, + + 'Partitioned By' as `stats__partitioning_type__label`, + partition_column as `stats__partitioning_type__value`, + 'The partitioning column for this table' as `stats__partitioning_type__description`, + is_partitioned as `stats__partitioning_type__include`, + + 'Clustered By' as `stats__clustering_fields__label`, + clustering_columns as `stats__clustering_fields__value`, + 'The clustering columns for this table' as `stats__clustering_fields__description`, + is_clustered as `stats__clustering_fields__include` + + -- join using relation_id (an actual relation, not a shard prefix) to make + -- sure that column metadata is picked up through the join. This will only + -- return the column information for the "max" table in a date-sharded table set + from unsharded_tables + left join columns using (relation_id) + left join column_stats using (relation_id) + ) + + {% if not loop.last %} union all {% endif %} + {% endfor %} + {%- endcall -%} + {{ return(load_result('catalog').table) }} + +{% endmacro %} From b13833cc54681626a5421eff8c0ab3aa1543874a Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Tue, 13 Aug 2019 21:44:54 -0400 Subject: [PATCH 076/860] (#525) drop existing relation at end of full-refresh incremental build --- .../macros/materializations/incremental.sql | 63 ++++++++----------- 1 file changed, 25 insertions(+), 38 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 7c8525fcc..9899eac8c 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -2,50 +2,37 @@ {% materialization incremental, adapter='bigquery' -%} {%- set unique_key = config.get('unique_key') -%} - {%- set full_refresh_mode = (flags.FULL_REFRESH == True) -%} - {%- set identifier = model['alias'] -%} - - {%- set old_relation = adapter.get_relation(database=database, schema=schema, identifier=identifier) -%} - - {%- set target_relation = api.Relation.create(database=database, identifier=identifier, schema=schema, type='table') -%} + {%- set target_relation = this %} + {%- set existing_relation = load_relation(this) %} + {%- set tmp_relation = make_temp_relation(this) %} - {%- set exists_as_table = (old_relation is not none and old_relation.is_table) -%} - {%- set exists_not_as_table = (old_relation is not none and not old_relation.is_table) -%} - - {%- set should_drop = (full_refresh_mode or exists_not_as_table) -%} - {%- set force_create = (full_refresh_mode) -%} + {{ run_hooks(pre_hooks) }} - -- setup - {% if old_relation is none -%} - -- noop - {%- elif should_drop -%} - {{ adapter.drop_relation(old_relation) }} - {%- set old_relation = none -%} - {%- endif %} + {% if existing_relation is none %} + {% set build_sql = create_table_as(False, target_relation, sql) %} + {% elif existing_relation.is_view %} + {#-- There's no way to atomically replace a view with a table on BQ --#} + {{ adapter.drop_relation(existing_relation) }} + {% set build_sql = create_table_as(False, target_relation, sql) %} + {% elif full_refresh_mode %} + {% set build_sql = create_table_as(False, target_relation, sql) %} + {% else %} + {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %} - {% set source_sql -%} {#-- wrap sql in parens to make it a subquery --#} - ( - {{ sql }} - ) - {%- endset -%} - - - {{ run_hooks(pre_hooks) }} - - -- build model - {% if force_create or old_relation is none -%} - {%- call statement('main') -%} - {{ create_table_as(False, target_relation, sql) }} - {%- endcall -%} - {%- else -%} - {% set dest_columns = adapter.get_columns_in_relation(target_relation) %} - {%- call statement('main') -%} - {{ get_merge_sql(target_relation, source_sql, unique_key, dest_columns) }} - {% endcall %} - {%- endif %} + {% set source_sql -%} + ( + {{ sql }} + ) + {%- endset -%} + {% set build_sql = get_merge_sql(target_relation, source_sql, unique_key, dest_columns) %} + {% endif %} + + {%- call statement('main') -%} + {{ build_sql }} + {% endcall %} {{ run_hooks(post_hooks) }} From ce4662969af4384deeaabc78dbcdb639ca1baf16 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Mon, 14 Oct 2019 22:48:52 -0400 Subject: [PATCH 077/860] handle changing partition/cluster configs on BQ --- dbt/adapters/bigquery/impl.py | 42 +++++++++++++++++++ dbt/include/bigquery/macros/adapters.sql | 9 ++-- .../macros/materializations/incremental.sql | 8 ++++ 3 files changed, 56 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 596aff2a6..d9ce05f8c 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -346,6 +346,48 @@ def execute_model(self, model, materialization, sql_override=None, return res + def _get_table(self, relation): + logger.debug('Fetching metadata for relation {}'.format(relation)) + conn = self.connections.get_thread_connection() + client = conn.handle + table_ref = self.connections.table_ref( + relation.database, + relation.schema, + relation.identifier, + conn + ) + + # Handle 404 + try: + return client.get_table(table_ref) + except (google.cloud.exceptions.NotFound) as e: + return None + + @available.parse_none + def is_replaceable(self, relation, conf_partition, conf_cluster): + """ + Check if a given partition and clustering column spec for a table + can replace an existing relation in the database. BigQuery does not + allow tables to be replaced with another table that has a different + partitioning spec. This method returns True if the given config spec is + identical to that of the existing table. + """ + table = self._get_table(relation) + if not table: + return True + + table_partition = table.time_partitioning + if table_partition is not None: + table_partition = table_partition.field + + table_cluster = table.clustering_fields + + if isinstance(conf_cluster, str): + conf_cluster = [conf_cluster] + + return table_partition == conf_partition \ + and table_cluster == conf_cluster + @available.parse_none def alter_table_add_columns(self, relation, columns): diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 60d5d4e50..719860ea4 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -37,9 +37,12 @@ {% do opts.update({'expiration_timestamp': 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)'}) %} {% endif %} - OPTIONS({% for opt_key, opt_val in opts.items() %} - {{ opt_key }}={{ opt_val }}{{ "," if not loop.last }} - {% endfor %}) + {% set options -%} + OPTIONS({% for opt_key, opt_val in opts.items() %} + {{ opt_key }}={{ opt_val }}{{ "," if not loop.last }} + {% endfor %}) + {%- endset %} + {% do return(options) %} {%- endmacro -%} {% macro bigquery__create_table_as(temporary, relation, sql) -%} diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 9899eac8c..d2e62e4f8 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -8,6 +8,9 @@ {%- set existing_relation = load_relation(this) %} {%- set tmp_relation = make_temp_relation(this) %} + {%- set partition_by = config.get('partition_by', none) -%} + {%- set cluster_by = config.get('cluster_by', none) -%} + {{ run_hooks(pre_hooks) }} {% if existing_relation is none %} @@ -17,6 +20,11 @@ {{ adapter.drop_relation(existing_relation) }} {% set build_sql = create_table_as(False, target_relation, sql) %} {% elif full_refresh_mode %} + {#-- If the partition/cluster config has changed, then we must drop and recreate --#} + {% if not adapter.is_replaceable(existing_relation, partition_by, cluster_by) %} + {% do log("Hard refreshing " ~ existing_relation ~ " because it is not replaceable") %} + {{ adapter.drop_relation(existing_relation) }} + {% endif %} {% set build_sql = create_table_as(False, target_relation, sql) %} {% else %} {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %} From 830d0121d598a50a7c16102767ce1d78417a3494 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Mon, 14 Oct 2019 23:07:42 -0400 Subject: [PATCH 078/860] Include Location field in catalog from the SCHEMATA table --- dbt/include/bigquery/macros/catalog.sql | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql index 86b0e352e..d52d20a5c 100644 --- a/dbt/include/bigquery/macros/catalog.sql +++ b/dbt/include/bigquery/macros/catalog.sql @@ -4,7 +4,18 @@ {%- call statement('catalog', fetch_result=True) -%} {% for information_schema in information_schemas %} ( - with tables as ( + with schemas as ( + + select + catalog_name as table_database, + schema_name as table_schema, + location + + from `{{ information_schema.database }}`.INFORMATION_SCHEMA.SCHEMATA + + ), + + tables as ( select project_id as table_database, dataset_id as table_schema, @@ -102,7 +113,7 @@ columns as ( select * except (base_column_name) - from info_schema_columns + from info_schema_columns join info_schema_column_paths using (relation_id, base_column_name) ), @@ -151,6 +162,11 @@ columns.column_type, columns.column_comment, + 'Location' as `stats__location__label`, + location as `stats__location__value`, + 'The geographic location of this table' as `stats__location__description`, + location is not null as `stats__location__include`, + 'Shard count' as `stats__date_shards__label`, table_shards.shard_count as `stats__date_shards__value`, 'The number of date shards in this table' as `stats__date_shards__description`, @@ -190,6 +206,7 @@ -- sure that column metadata is picked up through the join. This will only -- return the column information for the "max" table in a date-sharded table set from unsharded_tables + left join schemas using(table_database, table_schema) left join columns using (relation_id) left join column_stats using (relation_id) ) From c7db67951d3b36e0fb32ce3f6418a9646393a7cd Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Mon, 14 Oct 2019 23:13:09 -0400 Subject: [PATCH 079/860] pep8; code cleanup --- dbt/adapters/bigquery/impl.py | 26 +++++++------------------- 1 file changed, 7 insertions(+), 19 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index d9ce05f8c..0cb5f2700 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -346,23 +346,6 @@ def execute_model(self, model, materialization, sql_override=None, return res - def _get_table(self, relation): - logger.debug('Fetching metadata for relation {}'.format(relation)) - conn = self.connections.get_thread_connection() - client = conn.handle - table_ref = self.connections.table_ref( - relation.database, - relation.schema, - relation.identifier, - conn - ) - - # Handle 404 - try: - return client.get_table(table_ref) - except (google.cloud.exceptions.NotFound) as e: - return None - @available.parse_none def is_replaceable(self, relation, conf_partition, conf_cluster): """ @@ -372,8 +355,13 @@ def is_replaceable(self, relation, conf_partition, conf_cluster): partitioning spec. This method returns True if the given config spec is identical to that of the existing table. """ - table = self._get_table(relation) - if not table: + try: + table = self.connections.get_bq_table( + database=relation.database, + schema=relation.schema, + identifier=relation.identifier + ) + except google.cloud.exceptions.NotFound: return True table_partition = table.time_partitioning From 97569363168a3ea58bb37ed5c6faeb3060f22f06 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Tue, 15 Oct 2019 11:31:09 -0400 Subject: [PATCH 080/860] pr feedback --- dbt/adapters/bigquery/impl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 0cb5f2700..b1b1da58b 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -346,7 +346,7 @@ def execute_model(self, model, materialization, sql_override=None, return res - @available.parse_none + @available.parse(lambda *a, **k: True) def is_replaceable(self, relation, conf_partition, conf_cluster): """ Check if a given partition and clustering column spec for a table From fdab3cea5c0818b483d7c1b72e04ab5d0a430244 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Tue, 15 Oct 2019 12:28:09 -0400 Subject: [PATCH 081/860] improve the BigQuery Relations understanding of the information schema --- dbt/adapters/bigquery/impl.py | 28 ++------------------ dbt/adapters/bigquery/relation.py | 34 +++++++++++++++++++++++++ dbt/include/bigquery/macros/catalog.sql | 8 +++--- 3 files changed, 40 insertions(+), 30 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 5ca6c2e57..ae34b2a7c 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -23,9 +23,6 @@ import agate -GET_CATALOG_MACRO_NAME = 'get_catalog' - - def _stub_relation(*args, **kwargs): return BigQueryRelation.create( database='', @@ -388,31 +385,10 @@ def load_dataframe(self, database, schema, table_name, agate_table, with self.connections.exception_handler("LOAD TABLE"): self.poll_until_job_completes(job, timeout) - def get_catalog(self, manifest): - """Get the catalog for this manifest by running the get catalog macro. - Returns an agate.Table of catalog information. - """ - - information_schemas = [] - for database, schema in manifest.get_used_schemas(): - information_schemas.append(self.Relation.create( - database=database, - schema=schema, - quote_policy={ - 'database': True, - 'schema': True - } - )) - - # make it a list so macros can index into it. - kwargs = {'information_schemas': information_schemas} - table = self.execute_macro(GET_CATALOG_MACRO_NAME, - kwargs=kwargs, - release=True) - + def _catalog_filter_table(self, table, manifest): # BigQuery doesn't allow ":" chars in column names -- remap them here. table = table.rename(column_names={ col.name: col.name.replace('__', ':') for col in table.columns }) - return self._catalog_filter_table(table, manifest) + return super()._catalog_filter_table(table, manifest) diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py index 509ae8e4e..008693c74 100644 --- a/dbt/adapters/bigquery/relation.py +++ b/dbt/adapters/bigquery/relation.py @@ -5,6 +5,10 @@ BaseRelation, ComponentName ) from dbt.utils import filter_null_values +from typing import TypeVar + + +Self = TypeVar('Self', bound='BigQueryRelation') @dataclass(frozen=True, eq=False, repr=False) @@ -40,3 +44,33 @@ def project(self): @property def dataset(self): return self.schema + + def information_schema(self: Self, identifier=None) -> Self: + # BigQuery (usually) addresses information schemas at the dataset + # level. This method overrides the BaseRelation method to return an + # Information Schema relation as project.dataset.information_schem + + include_policy = self.include_policy.replace( + database=self.database is not None, + schema=self.schema is not None, + identifier=True + ) + + # Quote everything on BigQuery -- identifiers are case-sensitive, + # even when quoted. + quote_policy = self.quote_policy.replace( + database=True, + schema=True, + identifier=True, + ) + + path = self.path.replace( + schema=self.schema, + identifier='INFORMATION_SCHEMA' + ) + + return self.replace( + quote_policy=quote_policy, + include_policy=include_policy, + path=path, + ) diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql index d52d20a5c..2d1b0f249 100644 --- a/dbt/include/bigquery/macros/catalog.sql +++ b/dbt/include/bigquery/macros/catalog.sql @@ -11,7 +11,7 @@ schema_name as table_schema, location - from `{{ information_schema.database }}`.INFORMATION_SCHEMA.SCHEMATA + from {{ information_schema.include(schema=False) }}.SCHEMATA ), @@ -35,7 +35,7 @@ REGEXP_EXTRACT(table_id, '^(.+)[0-9]{8}$') as shard_base_name, REGEXP_EXTRACT(table_id, '^.+([0-9]{8})$') as shard_name - from {{ information_schema }}.__TABLES__ + from {{ information_schema.include(identifier=False) }}.__TABLES__ ), @@ -92,7 +92,7 @@ is_partitioning_column, clustering_ordinal_position - from {{ information_schema }}.INFORMATION_SCHEMA.COLUMNS + from {{ information_schema }}.COLUMNS where ordinal_position is not null ), @@ -105,7 +105,7 @@ data_type as column_type, column_name as base_column_name - from {{ information_schema }}.INFORMATION_SCHEMA.COLUMN_FIELD_PATHS + from {{ information_schema }}.COLUMN_FIELD_PATHS where data_type not like 'STRUCT%' ), From eadfce84c4cd00dd0cbeedeb0eb177fd64aa37d6 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 17 Oct 2019 07:37:41 -0600 Subject: [PATCH 082/860] Handle a number of bad failure-path behaviors - When a connection is missing, raise a special exception - instead of RuntimeError, so we catch it better - Be graceful if a connection does not exist during nice_connection_name - Handle the case where exceptions caught by the Snowflake exception handler do not have a 'msg' attr - Re-raise exceptions in the adapter exception handlers "from" the originating error --- dbt/adapters/bigquery/connections.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index b1bf8b44e..7c9251cff 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -64,7 +64,7 @@ def handle_error(cls, error, message, sql): error_msg = "\n".join( [item['message'] for item in error.errors]) - raise dbt.exceptions.DatabaseException(error_msg) + raise dbt.exceptions.DatabaseException(error_msg) from error def clear_transaction(self): pass @@ -90,7 +90,7 @@ def exception_handler(self, sql): # this sounds a lot like a signal handler and probably has # useful information, so raise it without modification. raise - raise dbt.exceptions.RuntimeException(str(e)) + raise dbt.exceptions.RuntimeException(str(e)) from e def cancel_open(self) -> None: pass From f04b91e56196fb2c38268506cc569401036f0846 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Fri, 18 Oct 2019 14:00:59 -0400 Subject: [PATCH 083/860] =?UTF-8?q?Bump=20version:=200.15.0a1=20=E2=86=92?= =?UTF-8?q?=200.15.0b1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index eeb2ff402..290003510 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.15.0a1" +package_version = "0.15.0b1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 8bcf30b4c204fd04fe92cace1108bc7ef7368159 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 18 Oct 2019 15:03:48 -0600 Subject: [PATCH 084/860] Fix namespace package builds use find_namespace_packages() instead of find_packages() --- setup.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/setup.py b/setup.py index 290003510..b76f41c97 100644 --- a/setup.py +++ b/setup.py @@ -1,5 +1,5 @@ #!/usr/bin/env python -from setuptools import find_packages +from setuptools import find_namespace_packages from distutils.core import setup import os @@ -20,7 +20,7 @@ author="Fishtown Analytics", author_email="info@fishtownanalytics.com", url="https://github.com/fishtown-analytics/dbt", - packages=find_packages(), + packages=find_namespace_packages(include=['dbt.*']), package_data={ 'dbt': [ 'include/bigquery/dbt_project.yml', From 6835ec92d24e9108fa8543e72c4b856c91b63503 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 21 Oct 2019 07:46:11 -0600 Subject: [PATCH 085/860] More packaging fixes - Add both 'dbt' and 'dbt.*' to the namespace package search path - set zip_safe=False, because I think zipping breaks macros - Make CI tests use non-editable installs - make tox use editable mode for explicit-* so development is still possible without rebuilding tox - remove {toxinidir} stuff that breaks core installation - as of tox 1.6.1, requirements executes within toxinidir. - fix paths to account for installing things --- setup.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/setup.py b/setup.py index b76f41c97..3827f4400 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ author="Fishtown Analytics", author_email="info@fishtownanalytics.com", url="https://github.com/fishtown-analytics/dbt", - packages=find_namespace_packages(include=['dbt.*']), + packages=find_namespace_packages(include=['dbt', 'dbt.*']), package_data={ 'dbt': [ 'include/bigquery/dbt_project.yml', @@ -31,5 +31,6 @@ install_requires=[ 'dbt-core=={}'.format(package_version), 'google-cloud-bigquery>=1.0.0,<2', - ] + ], + zip_safe=False, ) From 3abc24d49aacaeee6bf25b511c29c32cc95e5255 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Tue, 22 Oct 2019 12:18:08 -0400 Subject: [PATCH 086/860] =?UTF-8?q?Bump=20version:=200.15.0b1=20=E2=86=92?= =?UTF-8?q?=200.15.0b2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 3827f4400..43882e9f4 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.15.0b1" +package_version = "0.15.0b2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From c28922e04bcbcae622ca5c2783e7662d479b54c6 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Tue, 22 Oct 2019 12:28:26 -0400 Subject: [PATCH 087/860] use setuptools instead of distutil.core --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 43882e9f4..83ccb3550 100644 --- a/setup.py +++ b/setup.py @@ -1,6 +1,6 @@ #!/usr/bin/env python from setuptools import find_namespace_packages -from distutils.core import setup +from setuptools import setup import os package_name = "dbt-bigquery" From 3db04377d98d1a394e590fd781b89b786b9f6740 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Tue, 22 Oct 2019 14:16:41 -0600 Subject: [PATCH 088/860] Add a configuration option `kms_key_name` to dbt_project.yml for BigQuery projects where a Cloud KMS key can be specified and used to encrypt models. The key is specified to bigquery in the ddl OPTIONS. --- dbt/adapters/bigquery/impl.py | 3 ++- dbt/include/bigquery/macros/adapters.sql | 17 +++++++++-------- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index d25cc984e..2aa878d56 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -45,7 +45,8 @@ class BigQueryAdapter(BaseAdapter): Column = BigQueryColumn ConnectionManager = BigQueryConnectionManager - AdapterSpecificConfigs = frozenset({"cluster_by", "partition_by"}) + AdapterSpecificConfigs = frozenset({"cluster_by", "partition_by", + "kms_key_name"}) ### # Implementations of abstract methods diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 719860ea4..8e3d61c5d 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -26,7 +26,7 @@ {%- endmacro -%} -{% macro bigquery_table_options(persist_docs, temporary) %} +{% macro bigquery_table_options(persist_docs, temporary, kms_key_name) %} {% set opts = {} %} {% set description = get_relation_comment(persist_docs, model) %} @@ -36,24 +36,25 @@ {% if temporary %} {% do opts.update({'expiration_timestamp': 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)'}) %} {% endif %} + {% if kms_key_name %} + {% do opts.update({'kms_key_name': "'" ~ kms_key_name ~ "'"}) %} + {% endif %} - {% set options -%} - OPTIONS({% for opt_key, opt_val in opts.items() %} - {{ opt_key }}={{ opt_val }}{{ "," if not loop.last }} - {% endfor %}) - {%- endset %} - {% do return(options) %} + OPTIONS({% for opt_key, opt_val in opts.items() %} + {{ opt_key }}={{ opt_val }} + {% endfor %}) {%- endmacro -%} {% macro bigquery__create_table_as(temporary, relation, sql) -%} {%- set raw_partition_by = config.get('partition_by', none) -%} {%- set raw_cluster_by = config.get('cluster_by', none) -%} {%- set raw_persist_docs = config.get('persist_docs', {}) -%} + {%- set raw_kms_key_name = config.get('kms_key_name', none) -%} create or replace table {{ relation }} {{ partition_by(raw_partition_by) }} {{ cluster_by(raw_cluster_by) }} - {{ bigquery_table_options(persist_docs=raw_persist_docs, temporary=temporary) }} + {{ bigquery_table_options(persist_docs=raw_persist_docs, temporary=temporary, kms_key_name=raw_kms_key_name) }} as ( {{ sql }} ); From 8bde585bbd91db9fe0160dd9ce4dfae98dbba1f2 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Thu, 24 Oct 2019 10:09:33 -0600 Subject: [PATCH 089/860] Sync latest version Merge branch 'dev/louisa-may-alcott' of https://github.com/fishtown-analytics/dbt into kms-encryption --- dbt/include/bigquery/macros/adapters.sql | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 8e3d61c5d..8e4ac856e 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -40,9 +40,12 @@ {% do opts.update({'kms_key_name': "'" ~ kms_key_name ~ "'"}) %} {% endif %} - OPTIONS({% for opt_key, opt_val in opts.items() %} - {{ opt_key }}={{ opt_val }} - {% endfor %}) + {% set options -%} + OPTIONS({% for opt_key, opt_val in opts.items() %} + {{ opt_key }}={{ opt_val }} + {% endfor %}) + {%- endset %} + {% do return(options)%} {%- endmacro -%} {% macro bigquery__create_table_as(temporary, relation, sql) -%} From 152eb9585965a8abaad1d36540daf056b7d8cf52 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Thu, 24 Oct 2019 10:09:33 -0600 Subject: [PATCH 090/860] Sync latest version Merge branch 'dev/louisa-may-alcott' of https://github.com/fishtown-analytics/dbt into kms-encryption --- dbt/include/bigquery/macros/adapters.sql | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 8e3d61c5d..8e4ac856e 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -40,9 +40,12 @@ {% do opts.update({'kms_key_name': "'" ~ kms_key_name ~ "'"}) %} {% endif %} - OPTIONS({% for opt_key, opt_val in opts.items() %} - {{ opt_key }}={{ opt_val }} - {% endfor %}) + {% set options -%} + OPTIONS({% for opt_key, opt_val in opts.items() %} + {{ opt_key }}={{ opt_val }} + {% endfor %}) + {%- endset %} + {% do return(options)%} {%- endmacro -%} {% macro bigquery__create_table_as(temporary, relation, sql) -%} From fdf6beb98d40284227d10280fc2bc97eb4b9e013 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 28 Oct 2019 15:14:53 -0600 Subject: [PATCH 091/860] on bigquery, default to External instead of None when we do not recognize the table type --- dbt/adapters/bigquery/impl.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index d25cc984e..463b8ad26 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -294,7 +294,10 @@ def _bq_table_to_relation(self, bq_table): 'schema': True, 'identifier': True }, - type=self.RELATION_TYPES.get(bq_table.table_type)) + type=self.RELATION_TYPES.get( + bq_table.table_type, RelationType.External + ), + ) @classmethod def warning_on_hooks(hook_type): From 576be950603c629df938e46c9161f48b5a913979 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 10 Oct 2019 04:53:30 -0600 Subject: [PATCH 092/860] inject query comments Make a fake "macro" that we parse specially with a single global context Macro takes an argument (the node, may be none) Users supply the text of the macro in their 'user_config' under a new 'query_comment' No macros available query generator is an attribute on the connection manager - has a thread-local comment str - when acquiring a connection, set the comment str new 'connection_for' context manager: like connection_named, except also use the node to set the query string Updated unit tests to account for query comments Added a hacky, brittle integration test - log to a custom stream and read that Trim down the "target" context value to use the opt-in connection_info - Make sure it contains a superset of the documented stuff - Make sure it does not contain any blacklisted items Change some asserts to raise InternalExceptions because assert error messages in threads are useless --- dbt/adapters/bigquery/connections.py | 4 +++- dbt/adapters/bigquery/impl.py | 12 ++++++++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 7c9251cff..51f37b5bf 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -45,7 +45,8 @@ def type(self): return 'bigquery' def _connection_keys(self): - return ('method', 'database', 'schema', 'location') + return ('method', 'database', 'schema', 'location', 'priority', + 'timeout_seconds') class BigQueryConnectionManager(BaseConnectionManager): @@ -200,6 +201,7 @@ def raw_execute(self, sql, fetch=False): return query_job, iterator def execute(self, sql, auto_begin=False, fetch=None): + sql = self._add_query_comment(sql) # auto_begin is ignored on bigquery, and only included for consistency query_job, iterator = self.raw_execute(sql, fetch=fetch) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 463b8ad26..74a3b4fde 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -334,8 +334,16 @@ def execute_model(self, model, materialization, sql_override=None, if flags.STRICT_MODE: connection = self.connections.get_thread_connection() - assert isinstance(connection, Connection) - assert(connection.name == model.get('name')) + if not isinstance(connection, Connection): + raise dbt.exceptions.CompilerException( + f'Got {connection} - not a Connection!' + ) + model_uid = model.get('unique_id') + if connection.name != model_uid: + raise dbt.exceptions.InternalException( + f'Connection had name "{connection.name}", expected model ' + f'unique id of "{model_uid}"' + ) if materialization == 'view': res = self._materialize_as_view(model) From 3919173759f4d586a2c30f258fbeb91973266a6f Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 30 Oct 2019 20:38:23 -0600 Subject: [PATCH 093/860] Make bigquery adapters SQLAdapters Implement more things via macros Refactor Relations vs InformationSchemas to handle BQ better Fix a bug where bigquery cached uppercase schema names wrong - by using information_schema this just goes away :) --- dbt/adapters/bigquery/connections.py | 9 +++- dbt/adapters/bigquery/impl.py | 50 ++---------------- dbt/adapters/bigquery/relation.py | 54 ++++++++++--------- dbt/include/bigquery/macros/adapters.sql | 67 ++++++++++++++++++++---- dbt/include/bigquery/macros/catalog.sql | 8 +-- 5 files changed, 100 insertions(+), 88 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 51f37b5bf..e2fc2f4f8 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -296,7 +296,14 @@ def drop_dataset(self, database, schema): client = conn.handle with self.exception_handler('drop dataset'): - for table in client.list_tables(dataset): + try: + tables = list(client.list_tables(dataset)) + except google.api_core.exceptions.NotFound: + # the dataset doesn't exist. return here to match + # 'drop schema if exists' behavior. If anything 404s after this + # then there are real problems that should cause us to raise. + return + for table in tables: client.delete_table(table.reference) client.delete_dataset(dataset) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 74a3b4fde..c116b20ad 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -4,7 +4,8 @@ import dbt.clients.gcloud import dbt.clients.agate_helper -from dbt.adapters.base import BaseAdapter, available, RelationType +from dbt.adapters.base import available, RelationType +from dbt.adapters.sql import SQLAdapter from dbt.adapters.bigquery.relation import ( BigQueryRelation ) @@ -33,7 +34,7 @@ def _stub_relation(*args, **kwargs): ) -class BigQueryAdapter(BaseAdapter): +class BigQueryAdapter(SQLAdapter): RELATION_TYPES = { 'TABLE': RelationType.Table, @@ -82,21 +83,6 @@ def rename_relation(self, from_relation, to_relation): '`rename_relation` is not implemented for this adapter!' ) - @available - def list_schemas(self, database): - conn = self.connections.get_thread_connection() - client = conn.handle - - with self.connections.exception_handler('list dataset'): - # this is similar to how we have to deal with listing tables - all_datasets = client.list_datasets(project=database, - max_results=10000) - return [ds.dataset_id for ds in all_datasets] - - @available - def check_schema_exists(self, database, schema): - return super().check_schema_exists(database, schema) - def get_columns_in_relation(self, relation): try: table = self.connections.get_bq_table( @@ -118,33 +104,6 @@ def expand_target_column_types(self, from_relation, to_relation): # This is a no-op on BigQuery pass - def list_relations_without_caching(self, information_schema, schema): - connection = self.connections.get_thread_connection() - client = connection.handle - - bigquery_dataset = self.connections.dataset( - information_schema.database, schema, connection - ) - - all_tables = client.list_tables( - bigquery_dataset, - # BigQuery paginates tables by alphabetizing them, and using - # the name of the last table on a page as the key for the - # next page. If that key table gets dropped before we run - # list_relations, then this will 404. So, we avoid this - # situation by making the page size sufficiently large. - # see: https://github.com/fishtown-analytics/dbt/issues/726 - # TODO: cache the list of relations up front, and then we - # won't need to do this - max_results=100000) - - # This will 404 if the dataset does not exist. This behavior mirrors - # the implementation of list_relations for other adapters - try: - return [self._bq_table_to_relation(table) for table in all_tables] - except google.api_core.exceptions.NotFound: - return [] - def get_relation(self, database, schema, identifier): if self._schema_is_cached(database, schema): # if it's in the cache, use the parent's model of going through @@ -167,9 +126,6 @@ def create_schema(self, database, schema): def drop_schema(self, database, schema): logger.debug('Dropping schema "{}.{}".', database, schema) - - if not self.check_schema_exists(database, schema): - return self.connections.drop_dataset(database, schema) @classmethod diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py index 008693c74..7cca7a1c2 100644 --- a/dbt/adapters/bigquery/relation.py +++ b/dbt/adapters/bigquery/relation.py @@ -2,7 +2,7 @@ from typing import Optional from dbt.adapters.base.relation import ( - BaseRelation, ComponentName + BaseRelation, ComponentName, InformationSchema ) from dbt.utils import filter_null_values from typing import TypeVar @@ -45,32 +45,36 @@ def project(self): def dataset(self): return self.schema - def information_schema(self: Self, identifier=None) -> Self: - # BigQuery (usually) addresses information schemas at the dataset - # level. This method overrides the BaseRelation method to return an - # Information Schema relation as project.dataset.information_schem + def information_schema( + self, identifier: Optional[str] = None + ) -> 'BigQueryInformationSchema': + return BigQueryInformationSchema.from_relation(self, identifier) - include_policy = self.include_policy.replace( - database=self.database is not None, - schema=self.schema is not None, - identifier=True - ) - # Quote everything on BigQuery -- identifiers are case-sensitive, - # even when quoted. - quote_policy = self.quote_policy.replace( - database=True, - schema=True, - identifier=True, - ) +@dataclass(frozen=True, eq=False, repr=False) +class BigQueryInformationSchema(InformationSchema): + quote_character: str = '`' - path = self.path.replace( - schema=self.schema, - identifier='INFORMATION_SCHEMA' - ) + @classmethod + def get_include_policy(cls, relation, information_schema_view): + schema = True + if information_schema_view in ('SCHEMATA', 'SCHEMATA_OPTIONS', None): + schema = False + + identifier = True + if information_schema_view == '__TABLES__': + identifier = False - return self.replace( - quote_policy=quote_policy, - include_policy=include_policy, - path=path, + return relation.quote_policy.replace( + schema=schema, + identifier=identifier, ) + + def replace(self, **kwargs): + if 'information_schema_view' in kwargs: + view = kwargs['information_schema_view'] + # we also need to update the include policy, unless the caller did + # in which case it's their problem + if 'include_policy' not in kwargs: + kwargs['include_policy'] = self.get_include_policy(self, view) + return super().replace(**kwargs) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 719860ea4..eb9f63c32 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -89,21 +89,66 @@ {% endmacro %} -{% macro bigquery__list_relations_without_caching(information_schema, schema) -%} - {{ return(adapter.list_relations_without_caching(information_schema, schema)) }} +{% macro bigquery__list_schemas(database) -%} + {% set sql %} + select distinct schema_name + from {{ information_schema_name(database) }}.SCHEMATA + where UPPER(catalog_name) like UPPER('{{ database }}') + {% endset %} + {{ return(run_query(sql)) }} {% endmacro %} +{% macro empty_table() %} + {# This is the only way I know in jinja to get an empty agate table #} + {% do store_result('_empty_table', '', None) %} + {{ return(load_result('_empty_table')['table']) }} +{% endmacro %} -{% macro bigquery__current_timestamp() -%} - CURRENT_TIMESTAMP() -{%- endmacro %} +{%- macro bigquery_similar_schemas(database, schema) -%} + {%- set sql -%} + select distinct schema_name + from {{ information_schema_name(database) }}.SCHEMATA + where UPPER(catalog_name) like UPPER('{{ database }}') + and UPPER(schema_name) like UPPER('{{ schema }}') + {%- endset -%} + {%- set schemas = [] -%} + {%- for row in run_query(sql) -%} + {%- do schemas.append(row['schema_name']) %} + {%- endfor -%} + {{ return(schemas) }} +{%- endmacro -%} -{% macro bigquery__list_schemas(database) %} - {{ return(adapter.list_schemas()) }} -{% endmacro %} +{% macro bigquery__list_relations_without_caching(information_schema, schema) -%} + {# In bigquery, you can't query the full information schema, you can only do so + by schema (so 'database.schema.information_schema.tables'). But our schema + value is case-insensitive for annoying reasons involving quoting. So you + have figure out what schemas match the given schema first, and query them each. + #} + {%- set schema_candidates = bigquery_similar_schemas(information_schema.database, schema) -%} + {%- if (schema_candidates | length) == 0 -%} + {{ return(empty_table()) }} + {%- endif -%} + {%- set query -%} + {%- for s in schema_candidates %} + select + table_catalog as database, + table_name as name, + table_schema as schema, + case when table_type = 'BASE TABLE' then 'table' + when table_type = 'VIEW' then 'view' + when table_type = 'EXTERNAL TABLE' then 'external' + else table_type + end as table_type + from {{ information_schema.replace(information_schema_view='TABLES') }} + {% if not loop.last %}union all{% endif %} + {%- endfor %} + {%- endset -%} + {{ return(run_query(query)) }} +{%- endmacro %} -{% macro bigquery__check_schema_exists(information_schema, schema) %} - {{ return(adapter.check_schema_exists(information_schema.database, schema)) }} -{% endmacro %} + +{% macro bigquery__current_timestamp() -%} + CURRENT_TIMESTAMP() +{%- endmacro %} diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql index 2d1b0f249..2f5254913 100644 --- a/dbt/include/bigquery/macros/catalog.sql +++ b/dbt/include/bigquery/macros/catalog.sql @@ -11,7 +11,7 @@ schema_name as table_schema, location - from {{ information_schema.include(schema=False) }}.SCHEMATA + from {{ information_schema.replace(information_schema_view='SCHEMATA') }} ), @@ -35,7 +35,7 @@ REGEXP_EXTRACT(table_id, '^(.+)[0-9]{8}$') as shard_base_name, REGEXP_EXTRACT(table_id, '^.+([0-9]{8})$') as shard_name - from {{ information_schema.include(identifier=False) }}.__TABLES__ + from {{ information_schema.replace(information_schema_view='__TABLES__') }} ), @@ -92,7 +92,7 @@ is_partitioning_column, clustering_ordinal_position - from {{ information_schema }}.COLUMNS + from {{ information_schema.replace(information_schema_view='COLUMNS') }} where ordinal_position is not null ), @@ -105,7 +105,7 @@ data_type as column_type, column_name as base_column_name - from {{ information_schema }}.COLUMN_FIELD_PATHS + from {{ information_schema.replace(information_schema_view='COLUMN_FIELD_PATHS') }} where data_type not like 'STRUCT%' ), From 6b059581dde3adcf5bdc438e999c37cf1c16cd4f Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 1 Nov 2019 08:51:29 -0600 Subject: [PATCH 094/860] Fix BQ list_relations_without_caching + check_schema_exists --- dbt/adapters/bigquery/impl.py | 21 ++++++++++++ dbt/include/bigquery/macros/adapters.sql | 42 ++++++------------------ 2 files changed, 31 insertions(+), 32 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index c116b20ad..af25dcda8 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -83,6 +83,27 @@ def rename_relation(self, from_relation, to_relation): '`rename_relation` is not implemented for this adapter!' ) + @available.parse(lambda *a, **k: False) + def check_schema_exists(self, database: str, schema: str) -> bool: + conn = self.connections.get_thread_connection() + client = conn.handle + + bigquery_dataset = self.connections.dataset( + database, schema, conn + ) + # try to do things with the dataset. If it doesn't exist it will 404. + # we have to do it this way to handle underscore-prefixed datasets, + # which appear in neither the information_schema.schemata view nor the + # list_datasets method. + try: + next(client.list_tables(bigquery_dataset, max_results=1)) + except StopIteration: + pass + except google.api_core.exceptions.NotFound: + # the schema does not exist + return False + return True + def get_columns_in_relation(self, relation): try: table = self.connections.get_bq_table( diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index eb9f63c32..9589f148b 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -105,45 +105,23 @@ {% endmacro %} -{%- macro bigquery_similar_schemas(database, schema) -%} - {%- set sql -%} - select distinct schema_name - from {{ information_schema_name(database) }}.SCHEMATA - where UPPER(catalog_name) like UPPER('{{ database }}') - and UPPER(schema_name) like UPPER('{{ schema }}') - {%- endset -%} - {%- set schemas = [] -%} - {%- for row in run_query(sql) -%} - {%- do schemas.append(row['schema_name']) %} - {%- endfor -%} - {{ return(schemas) }} -{%- endmacro -%} - - {% macro bigquery__list_relations_without_caching(information_schema, schema) -%} {# In bigquery, you can't query the full information schema, you can only do so by schema (so 'database.schema.information_schema.tables'). But our schema value is case-insensitive for annoying reasons involving quoting. So you have figure out what schemas match the given schema first, and query them each. #} - {%- set schema_candidates = bigquery_similar_schemas(information_schema.database, schema) -%} - {%- if (schema_candidates | length) == 0 -%} - {{ return(empty_table()) }} - {%- endif -%} {%- set query -%} - {%- for s in schema_candidates %} - select - table_catalog as database, - table_name as name, - table_schema as schema, - case when table_type = 'BASE TABLE' then 'table' - when table_type = 'VIEW' then 'view' - when table_type = 'EXTERNAL TABLE' then 'external' - else table_type - end as table_type - from {{ information_schema.replace(information_schema_view='TABLES') }} - {% if not loop.last %}union all{% endif %} - {%- endfor %} + select + table_catalog as database, + table_name as name, + table_schema as schema, + case when table_type = 'BASE TABLE' then 'table' + when table_type = 'VIEW' then 'view' + when table_type = 'EXTERNAL TABLE' then 'external' + else table_type + end as table_type + from {{ information_schema.replace(information_schema_view='TABLES') }} {%- endset -%} {{ return(run_query(query)) }} {%- endmacro %} From 8125e00c32ac13bedfd0a470c0e443a69185a38c Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 1 Nov 2019 10:29:40 -0600 Subject: [PATCH 095/860] so much for being a SQLAdapter --- dbt/adapters/bigquery/impl.py | 45 +++++++++++++++++++++--- dbt/include/bigquery/macros/adapters.sql | 45 ++++++------------------ 2 files changed, 52 insertions(+), 38 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index af25dcda8..0a420db4e 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -4,8 +4,7 @@ import dbt.clients.gcloud import dbt.clients.agate_helper -from dbt.adapters.base import available, RelationType -from dbt.adapters.sql import SQLAdapter +from dbt.adapters.base import BaseAdapter, available, RelationType from dbt.adapters.bigquery.relation import ( BigQueryRelation ) @@ -34,7 +33,7 @@ def _stub_relation(*args, **kwargs): ) -class BigQueryAdapter(SQLAdapter): +class BigQueryAdapter(BaseAdapter): RELATION_TYPES = { 'TABLE': RelationType.Table, @@ -83,6 +82,17 @@ def rename_relation(self, from_relation, to_relation): '`rename_relation` is not implemented for this adapter!' ) + @available + def list_schemas(self, database): + conn = self.connections.get_thread_connection() + client = conn.handle + + with self.connections.exception_handler('list dataset'): + # this is similar to how we have to deal with listing tables + all_datasets = client.list_datasets(project=database, + max_results=10000) + return [ds.dataset_id for ds in all_datasets] + @available.parse(lambda *a, **k: False) def check_schema_exists(self, database: str, schema: str) -> bool: conn = self.connections.get_thread_connection() @@ -96,7 +106,7 @@ def check_schema_exists(self, database: str, schema: str) -> bool: # which appear in neither the information_schema.schemata view nor the # list_datasets method. try: - next(client.list_tables(bigquery_dataset, max_results=1)) + next(iter(client.list_tables(bigquery_dataset, max_results=1))) except StopIteration: pass except google.api_core.exceptions.NotFound: @@ -125,6 +135,33 @@ def expand_target_column_types(self, from_relation, to_relation): # This is a no-op on BigQuery pass + def list_relations_without_caching(self, information_schema, schema): + connection = self.connections.get_thread_connection() + client = connection.handle + + bigquery_dataset = self.connections.dataset( + information_schema.database, information_schema.schema, connection + ) + + all_tables = client.list_tables( + bigquery_dataset, + # BigQuery paginates tables by alphabetizing them, and using + # the name of the last table on a page as the key for the + # next page. If that key table gets dropped before we run + # list_relations, then this will 404. So, we avoid this + # situation by making the page size sufficiently large. + # see: https://github.com/fishtown-analytics/dbt/issues/726 + # TODO: cache the list of relations up front, and then we + # won't need to do this + max_results=100000) + + # This will 404 if the dataset does not exist. This behavior mirrors + # the implementation of list_relations for other adapters + try: + return [self._bq_table_to_relation(table) for table in all_tables] + except google.api_core.exceptions.NotFound: + return [] + def get_relation(self, database, schema, identifier): if self._schema_is_cached(database, schema): # if it's in the cache, use the parent's model of going through diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 9589f148b..126e6d3cd 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -89,44 +89,21 @@ {% endmacro %} -{% macro bigquery__list_schemas(database) -%} - {% set sql %} - select distinct schema_name - from {{ information_schema_name(database) }}.SCHEMATA - where UPPER(catalog_name) like UPPER('{{ database }}') - {% endset %} - {{ return(run_query(sql)) }} -{% endmacro %} - -{% macro empty_table() %} - {# This is the only way I know in jinja to get an empty agate table #} - {% do store_result('_empty_table', '', None) %} - {{ return(load_result('_empty_table')['table']) }} -{% endmacro %} - - {% macro bigquery__list_relations_without_caching(information_schema, schema) -%} - {# In bigquery, you can't query the full information schema, you can only do so - by schema (so 'database.schema.information_schema.tables'). But our schema - value is case-insensitive for annoying reasons involving quoting. So you - have figure out what schemas match the given schema first, and query them each. - #} - {%- set query -%} - select - table_catalog as database, - table_name as name, - table_schema as schema, - case when table_type = 'BASE TABLE' then 'table' - when table_type = 'VIEW' then 'view' - when table_type = 'EXTERNAL TABLE' then 'external' - else table_type - end as table_type - from {{ information_schema.replace(information_schema_view='TABLES') }} - {%- endset -%} - {{ return(run_query(query)) }} + {{ return(adapter.list_relations_without_caching(information_schema, schema)) }} {%- endmacro %} {% macro bigquery__current_timestamp() -%} CURRENT_TIMESTAMP() {%- endmacro %} + + +{% macro bigquery__list_schemas(database) -%} + {{ return(adapter.list_schemas()) }} +{% endmacro %} + + +{% macro bigquery__check_schema_exists(information_schema, schema) %} + {{ return(adapter.check_schema_exists(information_schema.database, schema)) }} +{% endmacro %} From 1f5204a45e523fa0efb7d648f5e6a9b2bcde16a4 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 4 Nov 2019 08:57:58 -0700 Subject: [PATCH 096/860] handle funky case bigquery models, add tests --- dbt/adapters/bigquery/impl.py | 87 +++++++++++++++++++++++++---------- 1 file changed, 62 insertions(+), 25 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 0a420db4e..648cf2a87 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -1,3 +1,5 @@ +from typing import Dict, List, Optional, Any + import dbt.deprecations import dbt.exceptions import dbt.flags as flags @@ -6,12 +8,13 @@ from dbt.adapters.base import BaseAdapter, available, RelationType from dbt.adapters.bigquery.relation import ( - BigQueryRelation + BigQueryRelation, BigQueryInformationSchema ) from dbt.adapters.bigquery import BigQueryColumn from dbt.adapters.bigquery import BigQueryConnectionManager from dbt.contracts.connection import Connection from dbt.logger import GLOBAL_LOGGER as logger +from dbt.utils import filter_null_values import google.auth import google.api_core @@ -52,14 +55,14 @@ class BigQueryAdapter(BaseAdapter): ### @classmethod - def date_function(cls): + def date_function(cls) -> str: return 'CURRENT_TIMESTAMP()' @classmethod - def is_cancelable(cls): + def is_cancelable(cls) -> bool: return False - def drop_relation(self, relation): + def drop_relation(self, relation: BigQueryRelation) -> None: is_cached = self._schema_is_cached(relation.database, relation.schema) if is_cached: self.cache_dropped(relation) @@ -72,18 +75,20 @@ def drop_relation(self, relation): relation_object = dataset.table(relation.identifier) client.delete_table(relation_object) - def truncate_relation(self, relation): + def truncate_relation(self, relation: BigQueryRelation) -> None: raise dbt.exceptions.NotImplementedException( '`truncate` is not implemented for this adapter!' ) - def rename_relation(self, from_relation, to_relation): + def rename_relation( + self, from_relation: BigQueryRelation, to_relation: BigQueryRelation + ) -> None: raise dbt.exceptions.NotImplementedException( '`rename_relation` is not implemented for this adapter!' ) @available - def list_schemas(self, database): + def list_schemas(self, database: str) -> List[str]: conn = self.connections.get_thread_connection() client = conn.handle @@ -114,7 +119,9 @@ def check_schema_exists(self, database: str, schema: str) -> bool: return False return True - def get_columns_in_relation(self, relation): + def get_columns_in_relation( + self, relation: BigQueryRelation + ) -> List[BigQueryColumn]: try: table = self.connections.get_bq_table( database=relation.database, @@ -127,15 +134,21 @@ def get_columns_in_relation(self, relation): logger.debug("get_columns_in_relation error: {}".format(e)) return [] - def expand_column_types(self, goal, current): + def expand_column_types( + self, goal: BigQueryRelation, current: BigQueryRelation + ) -> None: # This is a no-op on BigQuery pass - def expand_target_column_types(self, from_relation, to_relation): + def expand_target_column_types( + self, from_relation: BigQueryRelation, to_relation: BigQueryRelation + ) -> None: # This is a no-op on BigQuery pass - def list_relations_without_caching(self, information_schema, schema): + def list_relations_without_caching( + self, information_schema: BigQueryInformationSchema, schema: str + ) -> List[BigQueryRelation]: connection = self.connections.get_thread_connection() client = connection.handle @@ -162,7 +175,9 @@ def list_relations_without_caching(self, information_schema, schema): except google.api_core.exceptions.NotFound: return [] - def get_relation(self, database, schema, identifier): + def get_relation( + self, database: str, schema: str, identifier: str + ) -> BigQueryRelation: if self._schema_is_cached(database, schema): # if it's in the cache, use the parent's model of going through # the relations cache and picking out the relation @@ -178,47 +193,62 @@ def get_relation(self, database, schema, identifier): table = None return self._bq_table_to_relation(table) - def create_schema(self, database, schema): + def create_schema(self, database: str, schema: str) -> None: logger.debug('Creating schema "{}.{}".', database, schema) self.connections.create_dataset(database, schema) - def drop_schema(self, database, schema): + def drop_schema(self, database: str, schema: str) -> None: logger.debug('Dropping schema "{}.{}".', database, schema) self.connections.drop_dataset(database, schema) @classmethod - def quote(cls, identifier): + def quote(cls, identifier: str) -> str: return '`{}`'.format(identifier) @classmethod - def convert_text_type(cls, agate_table, col_idx): + def convert_text_type(cls, agate_table: agate.Table, col_idx: int) -> str: return "string" @classmethod - def convert_number_type(cls, agate_table, col_idx): + def convert_number_type( + cls, agate_table: agate.Table, col_idx: int + ) -> str: decimals = agate_table.aggregate(agate.MaxPrecision(col_idx)) return "float64" if decimals else "int64" @classmethod - def convert_boolean_type(cls, agate_table, col_idx): + def convert_boolean_type( + cls, agate_table: agate.Table, col_idx: int + ) -> str: return "bool" @classmethod - def convert_datetime_type(cls, agate_table, col_idx): + def convert_datetime_type( + cls, agate_table: agate.Table, col_idx: int + ) -> str: return "datetime" @classmethod - def convert_date_type(cls, agate_table, col_idx): + def convert_date_type(cls, agate_table: agate.Table, col_idx: int) -> str: return "date" @classmethod - def convert_time_type(cls, agate_table, col_idx): + def convert_time_type(cls, agate_table: agate.Table, col_idx: int) -> str: return "time" ### # Implementation details ### - def _get_dbt_columns_from_bq_table(self, table): + def _make_match_kwargs( + self, database: str, schema: str, identifier: str + ) -> Dict[str, str]: + return filter_null_values({ + 'database': database, + 'identifier': identifier, + 'schema': schema, + }) + + def _get_dbt_columns_from_bq_table(self, table) -> List[BigQueryColumn]: "Translates BQ SchemaField dicts into dbt BigQueryColumn objects" columns = [] @@ -231,7 +261,9 @@ def _get_dbt_columns_from_bq_table(self, table): return columns - def _agate_to_schema(self, agate_table, column_override): + def _agate_to_schema( + self, agate_table: agate.Table, column_override: Dict[str, str] + ) -> List[google.cloud.bigquery.SchemaField]: """Convert agate.Table with column names to a list of bigquery schemas. """ bq_schema = [] @@ -243,7 +275,7 @@ def _agate_to_schema(self, agate_table, column_override): ) return bq_schema - def _materialize_as_view(self, model): + def _materialize_as_view(self, model: Dict[str, Any]) -> str: model_database = model.get('database') model_schema = model.get('schema') model_alias = model.get('alias') @@ -258,7 +290,12 @@ def _materialize_as_view(self, model): ) return "CREATE VIEW" - def _materialize_as_table(self, model, model_sql, decorator=None): + def _materialize_as_table( + self, + model: Dict[str, Any], + model_sql: str, + decorator: Optional[str] = None, + ) -> str: model_database = model.get('database') model_schema = model.get('schema') model_alias = model.get('alias') From 6310d9fab961bd5960c43b9be81c046c5a17ce4f Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 4 Nov 2019 11:02:30 -0700 Subject: [PATCH 097/860] use exists_ok, not_found_ok, and delete_contents on bigquery dataset operations --- dbt/adapters/bigquery/connections.py | 22 ++++------------------ setup.py | 2 +- 2 files changed, 5 insertions(+), 19 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index e2fc2f4f8..f9e89cb0c 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -296,28 +296,14 @@ def drop_dataset(self, database, schema): client = conn.handle with self.exception_handler('drop dataset'): - try: - tables = list(client.list_tables(dataset)) - except google.api_core.exceptions.NotFound: - # the dataset doesn't exist. return here to match - # 'drop schema if exists' behavior. If anything 404s after this - # then there are real problems that should cause us to raise. - return - for table in tables: - client.delete_table(table.reference) - client.delete_dataset(dataset) + client.delete_dataset( + dataset, delete_contents=True, not_found_ok=True + ) def create_dataset(self, database, schema): conn = self.get_thread_connection() client = conn.handle dataset = self.dataset(database, schema, conn) - # Emulate 'create schema if not exists ...' - try: - client.get_dataset(dataset) - return - except google.api_core.exceptions.NotFound: - pass - with self.exception_handler('create dataset'): - client.create_dataset(dataset) + client.create_dataset(dataset, exists_ok=True) diff --git a/setup.py b/setup.py index 83ccb3550..d58743529 100644 --- a/setup.py +++ b/setup.py @@ -30,7 +30,7 @@ }, install_requires=[ 'dbt-core=={}'.format(package_version), - 'google-cloud-bigquery>=1.0.0,<2', + 'google-cloud-bigquery>=1.15.0,<2', ], zip_safe=False, ) From c6ccf7e216f05ef4ae94274d9a9386709eee0b77 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 4 Nov 2019 09:24:31 -0700 Subject: [PATCH 098/860] add some arbitrary-ish classifiers (importantly: the versions we support) --- setup.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/setup.py b/setup.py index d58743529..9fcbc1abc 100644 --- a/setup.py +++ b/setup.py @@ -33,4 +33,17 @@ 'google-cloud-bigquery>=1.15.0,<2', ], zip_safe=False, + classifiers=[ + 'Development Status :: 5 - Production/Stable', + + 'License :: OSI Approved :: Apache Software License', + + 'Operating System :: Microsoft :: Windows', + 'Operating System :: MacOS :: MacOS X', + 'Operating System :: POSIX :: Linux', + + 'Programming Language :: Python :: 3.6', + 'Programming Language :: Python :: 3.7', + 'Programming Language :: Python :: 3.8', + ], ) From ff3448437c084e67a7e2617c421dbccabeaed505 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Thu, 7 Nov 2019 10:27:33 -0500 Subject: [PATCH 099/860] =?UTF-8?q?Bump=20version:=200.15.0b2=20=E2=86=92?= =?UTF-8?q?=200.15.0rc1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 9fcbc1abc..eb962f8dd 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.15.0b2" +package_version = "0.15.0rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 33b2905c211d0ed56817cff19a09b6ca330d1cd8 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Tue, 19 Nov 2019 11:08:23 -0500 Subject: [PATCH 100/860] =?UTF-8?q?Bump=20version:=200.15.0rc1=20=E2=86=92?= =?UTF-8?q?=200.15.0rc2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index eb962f8dd..bff7b015c 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.15.0rc1" +package_version = "0.15.0rc2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 14adc99a61d136bbeafb4eefa68c3672d8e16303 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Fri, 22 Nov 2019 17:00:29 -0500 Subject: [PATCH 101/860] (#1950) Coalesce BQ types and names for external tables --- dbt/include/bigquery/macros/catalog.sql | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql index 2f5254913..a8b5f73fb 100644 --- a/dbt/include/bigquery/macros/catalog.sql +++ b/dbt/include/bigquery/macros/catalog.sql @@ -28,10 +28,10 @@ case when type = 1 then 'table' when type = 2 then 'view' - else concat('unknown (', cast(type as string), ')') + else 'external' end as table_type, - REGEXP_CONTAINS(table_id, '^.+[0-9]{8}$') and type = 1 as is_date_shard, + REGEXP_CONTAINS(table_id, '^.+[0-9]{8}$') and coalesce(type, 0) = 1 as is_date_shard, REGEXP_EXTRACT(table_id, '^(.+)[0-9]{8}$') as shard_base_name, REGEXP_EXTRACT(table_id, '^.+([0-9]{8})$') as shard_name @@ -57,7 +57,7 @@ table_database, table_schema, table_name, - table_type, + coalesce(table_type, 'external') as table_type, is_date_shard, struct( @@ -152,14 +152,16 @@ end as table_name, unsharded_tables.table_type, - columns.column_name, + -- coalesce name and type for External tables - these columns are not + -- present in the COLUMN_FIELD_PATHS resultset + coalesce(columns.column_name, '') as column_name, -- invent a row number to account for nested fields -- BQ does -- not treat these nested properties as independent fields row_number() over ( partition by relation_id order by columns.column_index, columns.column_name ) as column_index, - columns.column_type, + coalesce(columns.column_type, '') as column_type, columns.column_comment, 'Location' as `stats__location__label`, From 22de03a81f882c345c7cedb78160b071bef8e6e0 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Mon, 25 Nov 2019 09:35:06 -0500 Subject: [PATCH 102/860] change fallback column name from "null" to "unknown" --- dbt/include/bigquery/macros/catalog.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql index a8b5f73fb..a20d9865e 100644 --- a/dbt/include/bigquery/macros/catalog.sql +++ b/dbt/include/bigquery/macros/catalog.sql @@ -154,14 +154,14 @@ -- coalesce name and type for External tables - these columns are not -- present in the COLUMN_FIELD_PATHS resultset - coalesce(columns.column_name, '') as column_name, + coalesce(columns.column_name, '') as column_name, -- invent a row number to account for nested fields -- BQ does -- not treat these nested properties as independent fields row_number() over ( partition by relation_id order by columns.column_index, columns.column_name ) as column_index, - coalesce(columns.column_type, '') as column_type, + coalesce(columns.column_type, '') as column_type, columns.column_comment, 'Location' as `stats__location__label`, From c8ff543ffdb886384af1cba02a9904d1e4997a0f Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Mon, 25 Nov 2019 13:38:30 -0500 Subject: [PATCH 103/860] =?UTF-8?q?Bump=20version:=200.15.0rc2=20=E2=86=92?= =?UTF-8?q?=200.15.0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index bff7b015c..1a14db7b8 100644 --- a/setup.py +++ b/setup.py @@ -4,7 +4,7 @@ import os package_name = "dbt-bigquery" -package_version = "0.15.0rc2" +package_version = "0.15.0" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 5569a18ddbffb21da71f8b6f233dcaaeba3dcc74 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Wed, 27 Nov 2019 06:03:44 -0700 Subject: [PATCH 104/860] Implement retries in BQ adapter --- dbt/adapters/bigquery/connections.py | 133 +++++++++++++++++++++------ dbt/adapters/bigquery/impl.py | 5 +- 2 files changed, 109 insertions(+), 29 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index f9e89cb0c..3cbe451cf 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -7,6 +7,7 @@ import google.oauth2 import google.cloud.exceptions import google.cloud.bigquery +from google.api_core import retry import dbt.clients.agate_helper import dbt.exceptions @@ -35,6 +36,7 @@ class BigQueryCredentials(Credentials): timeout_seconds: Optional[int] = 300 location: Optional[str] = None priority: Optional[Priority] = None + retries: Optional[int] = 1 _ALIASES = { 'project': 'database', 'dataset': 'schema', @@ -57,6 +59,9 @@ class BigQueryConnectionManager(BaseConnectionManager): 'https://www.googleapis.com/auth/drive') QUERY_TIMEOUT = 300 + RETRIES = 1 + DEFAULT_INITIAL_DELAY = 1.0 # Seconds + DEFAULT_MAXIMUM_DELAY = 1.0 # Seconds @classmethod def handle_error(cls, error, message, sql): @@ -170,6 +175,11 @@ def get_timeout(cls, conn): credentials = conn.credentials return credentials.timeout_seconds + @classmethod + def get_retries(cls, conn): + credentials = conn['credentials'] + return credentials.get('retries', cls.RETRIES) + @classmethod def get_table_from_response(cls, resp): column_names = [field.name for field in resp.schema] @@ -182,21 +192,18 @@ def raw_execute(self, sql, fetch=False): logger.debug('On {}: {}', conn.name, sql) - job_config = google.cloud.bigquery.QueryJobConfig() - job_config.use_legacy_sql = False + job_params = {'use_legacy_sql': False} - priority = conn.credentials.priority - if priority == Priority.Batch: - job_config.priority = google.cloud.bigquery.QueryPriority.BATCH + priority = conn.credentials.get('priority', 'interactive') + if priority == 'batch': + job_params['priority'] = google.cloud.bigquery.QueryPriority.BATCH else: - job_config.priority = \ - google.cloud.bigquery.QueryPriority.INTERACTIVE + job_params[ + 'priority'] = google.cloud.bigquery.QueryPriority.INTERACTIVE - query_job = client.query(sql, job_config) + fn = lambda: self._query_and_results(client, sql, conn, job_params) - # this blocks until the query has completed - with self.exception_handler(sql): - iterator = query_job.result() + query_job, iterator = self._retry_and_handle(msg=sql, conn=conn, fn=fn) return query_job, iterator @@ -219,6 +226,20 @@ def execute(self, sql, auto_begin=False, fetch=None): table = client.get_table(query_job.destination) status = 'CREATE TABLE ({})'.format(table.num_rows) + elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: + elif query_job.statement_type == 'CREATE_TABLE_AS_SELECT': + conn = self.get_thread_connection() + client = conn.handle + table = client.get_table(query_job.destination) + status = 'CREATE TABLE ({})'.format(table.num_rows) + + elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: + elif query_job.statement_type == 'CREATE_TABLE_AS_SELECT': + conn = self.get_thread_connection() + client = conn.handle + table = client.get_table(query_job.destination) + status = 'CREATE TABLE ({})'.format(table.num_rows) + elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: status = '{} ({})'.format( query_job.statement_type, @@ -243,8 +264,8 @@ def create_bigquery_table(self, database, schema, table_name, callback, view = google.cloud.bigquery.Table(view_ref) callback(view) - with self.exception_handler(sql): - client.create_table(view) + fn = lambda: client.create_table(view) + self._retry_and_handle(msg=sql, conn=conn, fn=fn) def create_view(self, database, schema, table_name, sql): def callback(table): @@ -257,16 +278,11 @@ def create_table(self, database, schema, table_name, sql): conn = self.get_thread_connection() client = conn.handle - table_ref = self.table_ref(database, schema, table_name, conn) - job_config = google.cloud.bigquery.QueryJobConfig() - job_config.destination = table_ref - job_config.write_disposition = 'WRITE_TRUNCATE' - - query_job = client.query(sql, job_config=job_config) + job_params = {'destination': table_ref, + 'write_disposition': 'WRITE_TRUNCATE'} - # this waits for the job to complete - with self.exception_handler(sql): - query_job.result(timeout=self.get_timeout(conn)) + fn = lambda: self._query_and_results(client, sql, conn, job_params) + self._retry_and_handle(msg=sql, conn=conn, fn=fn) def create_date_partitioned_table(self, database, schema, table_name): def callback(table): @@ -295,15 +311,76 @@ def drop_dataset(self, database, schema): dataset = self.dataset(database, schema, conn) client = conn.handle - with self.exception_handler('drop dataset'): - client.delete_dataset( - dataset, delete_contents=True, not_found_ok=True - ) + def _drop_tables_then_dataset(): + for table in client.list_tables(dataset): + client.delete_table(table.reference) + client.delete_dataset(dataset) + + + self._retry_and_handle( + msg='drop dataset', conn=conn, fn=_drop_tables_then_dataset) def create_dataset(self, database, schema): conn = self.get_thread_connection() client = conn.handle dataset = self.dataset(database, schema, conn) - with self.exception_handler('create dataset'): - client.create_dataset(dataset, exists_ok=True) + # Emulate 'create schema if not exists ...' + try: + client.get_dataset(dataset) + return + except google.api_core.exceptions.NotFound: + pass + + fn = lambda: client.create_dataset(dataset) + self._retry_and_handle(msg='create dataset', conn=conn, fn=fn) + + def _query_and_results(self, client, sql, conn, job_params): + """Query the client and wait for results.""" + # Cannot reuse job_config if destination is set and ddl is used + job_config = google.cloud.bigquery.QueryJobConfig(**job_params) + query_job = client.query(sql, job_config=job_config) + iterator = query_job.result(timeout=self.get_timeout(conn)) + + return query_job, iterator + + def _retry_and_handle(self, msg, conn, fn): + """retry a function call within the context of exception_handler.""" + with self.exception_handler(msg): + return retry.retry_target( + target=fn, + predicate=_ErrorCounter(self.get_retries(conn)).count_error, + sleep_generator=self._retry_generator(), + deadline=None) + + def _retry_generator(self): + """Generates retry intervals that exponentially back off.""" + return retry.exponential_sleep_generator( + initial=self.DEFAULT_INITIAL_DELAY, + maximum=self.DEFAULT_MAXIMUM_DELAY) + +class _ErrorCounter(object): + """Counts errors seen up to a threshold then raises the next error.""" + + def __init__(self, retries): + self.retries = retries + self.error_count = 0 + + def count_error(self, error): + if self.retries == 0: + return False # Don't log + self.error_count +=1 + if _is_retryable(error) and self.error_count <= self.retries: + logger.warning( + 'Retry attempt %s of %s after error: %s', + self.error_count, self.retries, repr(error)) + return True + else: + logger.warning( + 'Not Retrying after %s previous attempts. Error: %s', + self.error_count - 1, repr(error)) + return False + +def _is_retryable(error): + """Return true for 500 level (retryable) errors.""" + return isinstance(error, google.cloud.exceptions.ServerError) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 5f8027cd8..1dc2716b6 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -93,11 +93,14 @@ def list_schemas(self, database: str) -> List[str]: conn = self.connections.get_thread_connection() client = conn.handle - with self.connections.exception_handler('list dataset'): + def query_schemas(): # this is similar to how we have to deal with listing tables all_datasets = client.list_datasets(project=database, max_results=10000) return [ds.dataset_id for ds in all_datasets] + + return self.connections._retry_and_handle( + msg='list dataset', conn=conn, fn=query_schemas) @available.parse(lambda *a, **k: False) def check_schema_exists(self, database: str, schema: str) -> bool: From 2145d1f4edcbc385724eb24bdce85669131f950b Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Wed, 27 Nov 2019 08:22:37 -0700 Subject: [PATCH 105/860] Implement bq labels as a configurable option --- dbt/adapters/bigquery/impl.py | 2 +- dbt/include/bigquery/macros/adapters.sql | 35 +++++++++++++++--------- 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 5f8027cd8..766d6a23e 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -49,7 +49,7 @@ class BigQueryAdapter(BaseAdapter): ConnectionManager = BigQueryConnectionManager AdapterSpecificConfigs = frozenset({"cluster_by", "partition_by", - "kms_key_name"}) + "kms_key_name", "labels"}) ### # Implementations of abstract methods diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 944127024..2a422185f 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -27,25 +27,32 @@ {%- endmacro -%} {% macro bigquery_table_options(persist_docs, temporary, kms_key_name) %} - {% set opts = {} %} + {% set opts = {} -%} - {% set description = get_relation_comment(persist_docs, model) %} + {%- set description = get_relation_comment(persist_docs, model) -%} {%- if description is not none -%} - {% do opts.update({'description': "'" ~ description ~ "'"}) %} - {% endif %} - {% if temporary %} + {%- do opts.update({'description': "'" ~ description ~ "'"}) -%} + {%- endif -%} + {%- if temporary -%} {% do opts.update({'expiration_timestamp': 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)'}) %} - {% endif %} - {% if kms_key_name %} - {% do opts.update({'kms_key_name': "'" ~ kms_key_name ~ "'"}) %} - {% endif %} + {%- endif -%} + {%- if kms_key_name -%} + {%- do opts.update({'kms_key_name': "'" ~ kms_key_name ~ "'"}) -%} + {%- endif -%} + {%- if labels -%} + {%- set label_list = [] -%} + {%- for key in labels -%} + {%- do label_list.append((key, labels[key])) -%} + {%- endfor -%} + {%- do opts.update({'labels': label_list}) -%} + {%- endif -%} {% set options -%} OPTIONS({% for opt_key, opt_val in opts.items() %} {{ opt_key }}={{ opt_val }}{{ "," if not loop.last }} {% endfor %}) {%- endset %} - {% do return(options) %} + {%- do return(options) -%} {%- endmacro -%} {% macro bigquery__create_table_as(temporary, relation, sql) -%} @@ -53,15 +60,17 @@ {%- set raw_cluster_by = config.get('cluster_by', none) -%} {%- set raw_persist_docs = config.get('persist_docs', {}) -%} {%- set raw_kms_key_name = config.get('kms_key_name', none) -%} - + {%- set raw_labels = config.get('labels', []) -%} create or replace table {{ relation }} {{ partition_by(raw_partition_by) }} {{ cluster_by(raw_cluster_by) }} - {{ bigquery_table_options(persist_docs=raw_persist_docs, temporary=temporary, kms_key_name=raw_kms_key_name) }} + {{ bigquery_table_options( + persist_docs=raw_persist_docs, temporary=temporary, kms_key_name=raw_kms_key_name, + labels=raw_labels) }} as ( {{ sql }} ); -{% endmacro %} +{%- endmacro -%} {% macro bigquery__create_view_as(relation, sql) -%} From a270636ac5bd80ebbb1b05e6f025da5b81ecdb2d Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Wed, 27 Nov 2019 08:28:02 -0700 Subject: [PATCH 106/860] cleanup --- dbt/include/bigquery/macros/adapters.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 2a422185f..a6e8b38fe 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -65,8 +65,8 @@ {{ partition_by(raw_partition_by) }} {{ cluster_by(raw_cluster_by) }} {{ bigquery_table_options( - persist_docs=raw_persist_docs, temporary=temporary, kms_key_name=raw_kms_key_name, - labels=raw_labels) }} + persist_docs=raw_persist_docs, temporary=temporary, kms_key_name=raw_kms_key_name, + labels=raw_labels) }} as ( {{ sql }} ); From ce1246c1085054bf445f055f2aeb74e7e94df754 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Wed, 27 Nov 2019 12:05:09 -0700 Subject: [PATCH 107/860] Fix --- dbt/adapters/bigquery/connections.py | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 3cbe451cf..8f5d17b02 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -226,20 +226,6 @@ def execute(self, sql, auto_begin=False, fetch=None): table = client.get_table(query_job.destination) status = 'CREATE TABLE ({})'.format(table.num_rows) - elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: - elif query_job.statement_type == 'CREATE_TABLE_AS_SELECT': - conn = self.get_thread_connection() - client = conn.handle - table = client.get_table(query_job.destination) - status = 'CREATE TABLE ({})'.format(table.num_rows) - - elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: - elif query_job.statement_type == 'CREATE_TABLE_AS_SELECT': - conn = self.get_thread_connection() - client = conn.handle - table = client.get_table(query_job.destination) - status = 'CREATE TABLE ({})'.format(table.num_rows) - elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: status = '{} ({})'.format( query_job.statement_type, From 55da1b2b29e54c8270aa444a898f2884cb28aa86 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 2 Dec 2019 10:45:26 -0700 Subject: [PATCH 108/860] Add support for sql as a header to create or replace --- dbt/include/bigquery/macros/adapters.sql | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 944127024..2f4c01427 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -26,6 +26,10 @@ {%- endmacro -%} +{% macro sql_header(config) -%} + {{ config.set('sql_header', caller()) }} +{%- endmacro %} + {% macro bigquery_table_options(persist_docs, temporary, kms_key_name) %} {% set opts = {} %} @@ -53,6 +57,9 @@ {%- set raw_cluster_by = config.get('cluster_by', none) -%} {%- set raw_persist_docs = config.get('persist_docs', {}) -%} {%- set raw_kms_key_name = config.get('kms_key_name', none) -%} + {%- set sql_header = config.get('sql_header', none) -%} + + {{ sql_header if sql_header is not none }} create or replace table {{ relation }} {{ partition_by(raw_partition_by) }} From 21a3294b53a4f4441999791b9c29838b3023d1b3 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 2 Dec 2019 15:42:45 -0700 Subject: [PATCH 109/860] Add integration test and implement suggestions --- dbt/include/bigquery/macros/adapters.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index a6e8b38fe..78d739d75 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -26,7 +26,7 @@ {%- endmacro -%} -{% macro bigquery_table_options(persist_docs, temporary, kms_key_name) %} +{% macro bigquery_table_options(persist_docs, temporary, kms_key_name, labels) %} {% set opts = {} -%} {%- set description = get_relation_comment(persist_docs, model) -%} @@ -41,8 +41,8 @@ {%- endif -%} {%- if labels -%} {%- set label_list = [] -%} - {%- for key in labels -%} - {%- do label_list.append((key, labels[key])) -%} + {%- for label, value in labels.items() -%} + {%- do label_list.append((label, value)) -%} {%- endfor -%} {%- do opts.update({'labels': label_list}) -%} {%- endif -%} From fbda2bf5587c46c28ead42d33092e75a30fff61b Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Tue, 3 Dec 2019 12:57:32 -0700 Subject: [PATCH 110/860] Fix integration test and enable labels for views as well --- dbt/include/bigquery/macros/adapters.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 78d739d75..2dcb02001 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -75,9 +75,10 @@ {% macro bigquery__create_view_as(relation, sql) -%} {%- set raw_persist_docs = config.get('persist_docs', {}) -%} + {%- set raw_labels = config.get('labels', []) -%} create or replace view {{ relation }} - {{ bigquery_table_options(persist_docs=raw_persist_docs, temporary=false) }} + {{ bigquery_table_options(persist_docs=raw_persist_docs, temporary=false, labels=raw_labels) }} as ( {{ sql }} ); From f1d1676190bc771e1ac23e06115c05ee343abdd5 Mon Sep 17 00:00:00 2001 From: Kurt Date: Tue, 3 Dec 2019 15:02:05 -0500 Subject: [PATCH 111/860] Update plugins/bigquery/dbt/adapters/bigquery/connections.py Co-Authored-By: Drew Banin --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 8f5d17b02..09bef6c34 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -195,7 +195,7 @@ def raw_execute(self, sql, fetch=False): job_params = {'use_legacy_sql': False} priority = conn.credentials.get('priority', 'interactive') - if priority == 'batch': + if priority == Priority.Batch: job_params['priority'] = google.cloud.bigquery.QueryPriority.BATCH else: job_params[ From 33cc14ddeb06361f15b5af265dec831f427d7b63 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Tue, 3 Dec 2019 15:19:58 -0700 Subject: [PATCH 112/860] Use client.create_dataset and client.delete_dataset --- dbt/adapters/bigquery/connections.py | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 8f5d17b02..d782eb662 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -297,28 +297,19 @@ def drop_dataset(self, database, schema): dataset = self.dataset(database, schema, conn) client = conn.handle - def _drop_tables_then_dataset(): - for table in client.list_tables(dataset): - client.delete_table(table.reference) - client.delete_dataset(dataset) + fn = lambda: client.delete_dataset( + dataset, delete_contents=True, not_found_ok=True) self._retry_and_handle( - msg='drop dataset', conn=conn, fn=_drop_tables_then_dataset) + msg='drop dataset', conn=conn, fn=fn) def create_dataset(self, database, schema): conn = self.get_thread_connection() client = conn.handle dataset = self.dataset(database, schema, conn) - # Emulate 'create schema if not exists ...' - try: - client.get_dataset(dataset) - return - except google.api_core.exceptions.NotFound: - pass - - fn = lambda: client.create_dataset(dataset) + fn = lambda: client.create_dataset(dataset, exist_ok=True) self._retry_and_handle(msg='create dataset', conn=conn, fn=fn) def _query_and_results(self, client, sql, conn, job_params): From ec740f336d26ae08fde9776f68866ffc7e46f3b4 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Tue, 3 Dec 2019 16:01:52 -0700 Subject: [PATCH 113/860] Make sql_header universal accross adapters --- dbt/include/bigquery/macros/adapters.sql | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 2f4c01427..1042665b3 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -26,10 +26,6 @@ {%- endmacro -%} -{% macro sql_header(config) -%} - {{ config.set('sql_header', caller()) }} -{%- endmacro %} - {% macro bigquery_table_options(persist_docs, temporary, kms_key_name) %} {% set opts = {} %} @@ -73,6 +69,9 @@ {% macro bigquery__create_view_as(relation, sql) -%} {%- set raw_persist_docs = config.get('persist_docs', {}) -%} + {%- set sql_header = config.get('sql_header', none) -%} + + {{ sql_header if sql_header is not none }} create or replace view {{ relation }} {{ bigquery_table_options(persist_docs=raw_persist_docs, temporary=false) }} From 237faaad36b005f36cc3e262c8c18de9b153387b Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 4 Dec 2019 14:23:53 -0700 Subject: [PATCH 114/860] fail more gracefully when setuptools is missing --- setup.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/setup.py b/setup.py index 1a14db7b8..3dd31af2c 100644 --- a/setup.py +++ b/setup.py @@ -1,7 +1,17 @@ #!/usr/bin/env python -from setuptools import find_namespace_packages -from setuptools import setup import os +import sys + +from setuptools import setup +try: + from setuptools import find_namespace_packages +except ImportError: + # the user has a downlevel version of setuptools. + print('Error: dbt requires v40.1.0 or higher of setuptools.') + print('Please upgrade setuptools with "pip install --upgrade setuptools" ' + 'and try again') + sys.exit(1) + package_name = "dbt-bigquery" package_version = "0.15.0" From 89fbcaa226b2dae9b6e40e10a8408d6dd0a8723f Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 5 Dec 2019 11:05:34 -0700 Subject: [PATCH 115/860] Restore drop_schema to contexts Made a few related fixes to caching behavior w.r.t schemas Added a test --- dbt/adapters/bigquery/impl.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 5f8027cd8..05e22772c 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -201,6 +201,7 @@ def create_schema(self, database: str, schema: str) -> None: def drop_schema(self, database: str, schema: str) -> None: logger.debug('Dropping schema "{}.{}".', database, schema) self.connections.drop_dataset(database, schema) + self.cache.drop_schema(database, schema) @classmethod def quote(cls, identifier: str) -> str: From 0a2d8ab59d49e1967a3d4e06bc1d7fdf857aed69 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 5 Dec 2019 14:20:24 -0700 Subject: [PATCH 116/860] PR feedback --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 3dd31af2c..578529010 100644 --- a/setup.py +++ b/setup.py @@ -7,7 +7,7 @@ from setuptools import find_namespace_packages except ImportError: # the user has a downlevel version of setuptools. - print('Error: dbt requires v40.1.0 or higher of setuptools.') + print('Error: dbt requires setuptools v40.1.0 or higher.') print('Please upgrade setuptools with "pip install --upgrade setuptools" ' 'and try again') sys.exit(1) From a78e0a1700440dbb40e4b1e7099d52877d16f0b8 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 9 Dec 2019 11:56:24 -0700 Subject: [PATCH 117/860] Avoid overlapping validity timestamps, tests --- dbt/include/bigquery/macros/adapters.sql | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 944127024..b683bbd26 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -103,6 +103,12 @@ {%- endmacro %} +{% macro bigquery__snapshot_string_as_time(timestamp) -%} + {%- set result = 'TIMESTAMP("' ~ timestamp ~ '")' -%} + {{ return(result) }} +{%- endmacro %} + + {% macro bigquery__list_schemas(database) -%} {{ return(adapter.list_schemas()) }} {% endmacro %} From 9341bbc80c646a331750a62f12d2f6de3a464377 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 9 Dec 2019 15:10:38 -0700 Subject: [PATCH 118/860] Still get table ref --- dbt/adapters/bigquery/connections.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 4bd12a677..f2f30b285 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -264,6 +264,7 @@ def create_table(self, database, schema, table_name, sql): conn = self.get_thread_connection() client = conn.handle + table_ref = self.table_ref(database, schema, table_name, conn) job_params = {'destination': table_ref, 'write_disposition': 'WRITE_TRUNCATE'} From c213978dcd95a37a29d03b00fdabbc1393bca2ab Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 9 Dec 2019 16:47:50 -0700 Subject: [PATCH 119/860] Fix formatting --- dbt/adapters/bigquery/connections.py | 30 +++++++++++++++++----------- dbt/adapters/bigquery/impl.py | 4 ++-- 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index f2f30b285..d1f35ca76 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -177,8 +177,8 @@ def get_timeout(cls, conn): @classmethod def get_retries(cls, conn): - credentials = conn['credentials'] - return credentials.get('retries', cls.RETRIES) + credentials = conn.credentials + return credentials.retries @classmethod def get_table_from_response(cls, resp): @@ -194,14 +194,15 @@ def raw_execute(self, sql, fetch=False): job_params = {'use_legacy_sql': False} - priority = conn.credentials.get('priority', 'interactive') + priority = conn.credentials.priority if priority == Priority.Batch: job_params['priority'] = google.cloud.bigquery.QueryPriority.BATCH else: job_params[ - 'priority'] = google.cloud.bigquery.QueryPriority.INTERACTIVE + 'priority'] = google.cloud.bigquery.QueryPriority.INTERACTIVE - fn = lambda: self._query_and_results(client, sql, conn, job_params) + def fn(): + self._query_and_results(client, sql, conn, job_params) query_job, iterator = self._retry_and_handle(msg=sql, conn=conn, fn=fn) @@ -250,7 +251,8 @@ def create_bigquery_table(self, database, schema, table_name, callback, view = google.cloud.bigquery.Table(view_ref) callback(view) - fn = lambda: client.create_table(view) + def fn(): + client.create_table(view) self._retry_and_handle(msg=sql, conn=conn, fn=fn) def create_view(self, database, schema, table_name, sql): @@ -268,7 +270,8 @@ def create_table(self, database, schema, table_name, sql): job_params = {'destination': table_ref, 'write_disposition': 'WRITE_TRUNCATE'} - fn = lambda: self._query_and_results(client, sql, conn, job_params) + def fn(): + self._query_and_results(client, sql, conn, job_params) self._retry_and_handle(msg=sql, conn=conn, fn=fn) def create_date_partitioned_table(self, database, schema, table_name): @@ -298,9 +301,9 @@ def drop_dataset(self, database, schema): dataset = self.dataset(database, schema, conn) client = conn.handle - fn = lambda: client.delete_dataset( - dataset, delete_contents=True, not_found_ok=True) - + def fn(): + client.delete_dataset( + dataset, delete_contents=True, not_found_ok=True) self._retry_and_handle( msg='drop dataset', conn=conn, fn=fn) @@ -310,7 +313,8 @@ def create_dataset(self, database, schema): client = conn.handle dataset = self.dataset(database, schema, conn) - fn = lambda: client.create_dataset(dataset, exist_ok=True) + def fn(): + client.create_dataset(dataset, exist_ok=True) self._retry_and_handle(msg='create dataset', conn=conn, fn=fn) def _query_and_results(self, client, sql, conn, job_params): @@ -337,6 +341,7 @@ def _retry_generator(self): initial=self.DEFAULT_INITIAL_DELAY, maximum=self.DEFAULT_MAXIMUM_DELAY) + class _ErrorCounter(object): """Counts errors seen up to a threshold then raises the next error.""" @@ -347,7 +352,7 @@ def __init__(self, retries): def count_error(self, error): if self.retries == 0: return False # Don't log - self.error_count +=1 + self.error_count += 1 if _is_retryable(error) and self.error_count <= self.retries: logger.warning( 'Retry attempt %s of %s after error: %s', @@ -359,6 +364,7 @@ def count_error(self, error): self.error_count - 1, repr(error)) return False + def _is_retryable(error): """Return true for 500 level (retryable) errors.""" return isinstance(error, google.cloud.exceptions.ServerError) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 1dc2716b6..dc462e5a4 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -98,9 +98,9 @@ def query_schemas(): all_datasets = client.list_datasets(project=database, max_results=10000) return [ds.dataset_id for ds in all_datasets] - + return self.connections._retry_and_handle( - msg='list dataset', conn=conn, fn=query_schemas) + msg='list dataset', conn=conn, fn=query_schemas) @available.parse(lambda *a, **k: False) def check_schema_exists(self, database: str, schema: str) -> bool: From aaa97fdcdc43e83937b158ac0399e0a14e49e073 Mon Sep 17 00:00:00 2001 From: Kurt Date: Tue, 10 Dec 2019 12:55:56 -0500 Subject: [PATCH 120/860] Update plugins/bigquery/dbt/adapters/bigquery/connections.py Co-Authored-By: Jacob Beck --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index d1f35ca76..082a18bdf 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -201,7 +201,7 @@ def raw_execute(self, sql, fetch=False): job_params[ 'priority'] = google.cloud.bigquery.QueryPriority.INTERACTIVE - def fn(): + def fn(): self._query_and_results(client, sql, conn, job_params) query_job, iterator = self._retry_and_handle(msg=sql, conn=conn, fn=fn) From 50772609834324c6d5e273f2af0bb04c16e05a51 Mon Sep 17 00:00:00 2001 From: Kurt Date: Tue, 10 Dec 2019 12:56:24 -0500 Subject: [PATCH 121/860] Update plugins/bigquery/dbt/adapters/bigquery/connections.py Co-Authored-By: Jacob Beck --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 082a18bdf..09a81e39b 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -306,7 +306,7 @@ def fn(): dataset, delete_contents=True, not_found_ok=True) self._retry_and_handle( - msg='drop dataset', conn=conn, fn=fn) + msg='drop dataset', conn=conn, fn=fn) def create_dataset(self, database, schema): conn = self.get_thread_connection() From 1c5a8d9bc5dd1eb6bdc3ea4e0b5fc9fa642e9f62 Mon Sep 17 00:00:00 2001 From: Kurt Date: Tue, 10 Dec 2019 12:56:33 -0500 Subject: [PATCH 122/860] Update plugins/bigquery/dbt/adapters/bigquery/connections.py Co-Authored-By: Jacob Beck --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 09a81e39b..e7f6286e8 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -270,7 +270,7 @@ def create_table(self, database, schema, table_name, sql): job_params = {'destination': table_ref, 'write_disposition': 'WRITE_TRUNCATE'} - def fn(): + def fn(): self._query_and_results(client, sql, conn, job_params) self._retry_and_handle(msg=sql, conn=conn, fn=fn) From a2c52534f2eb24e6f88d5a20caae61a2857644ad Mon Sep 17 00:00:00 2001 From: Kurt Date: Wed, 11 Dec 2019 12:47:41 -0500 Subject: [PATCH 123/860] Update plugins/bigquery/dbt/adapters/bigquery/connections.py Co-Authored-By: Jacob Beck --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index e7f6286e8..fa817a389 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -314,7 +314,7 @@ def create_dataset(self, database, schema): dataset = self.dataset(database, schema, conn) def fn(): - client.create_dataset(dataset, exist_ok=True) + client.create_dataset(dataset, exists_ok=True) self._retry_and_handle(msg='create dataset', conn=conn, fn=fn) def _query_and_results(self, client, sql, conn, job_params): From 508fae404c57c3d01b5673cbc0e6816334c3930a Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Wed, 11 Dec 2019 12:57:32 -0700 Subject: [PATCH 124/860] Return function results --- dbt/adapters/bigquery/connections.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index e7f6286e8..a0427117c 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -202,7 +202,7 @@ def raw_execute(self, sql, fetch=False): 'priority'] = google.cloud.bigquery.QueryPriority.INTERACTIVE def fn(): - self._query_and_results(client, sql, conn, job_params) + return self._query_and_results(client, sql, conn, job_params) query_job, iterator = self._retry_and_handle(msg=sql, conn=conn, fn=fn) @@ -252,7 +252,7 @@ def create_bigquery_table(self, database, schema, table_name, callback, callback(view) def fn(): - client.create_table(view) + return client.create_table(view) self._retry_and_handle(msg=sql, conn=conn, fn=fn) def create_view(self, database, schema, table_name, sql): @@ -271,7 +271,7 @@ def create_table(self, database, schema, table_name, sql): 'write_disposition': 'WRITE_TRUNCATE'} def fn(): - self._query_and_results(client, sql, conn, job_params) + return self._query_and_results(client, sql, conn, job_params) self._retry_and_handle(msg=sql, conn=conn, fn=fn) def create_date_partitioned_table(self, database, schema, table_name): @@ -302,7 +302,7 @@ def drop_dataset(self, database, schema): client = conn.handle def fn(): - client.delete_dataset( + return client.delete_dataset( dataset, delete_contents=True, not_found_ok=True) self._retry_and_handle( @@ -314,7 +314,7 @@ def create_dataset(self, database, schema): dataset = self.dataset(database, schema, conn) def fn(): - client.create_dataset(dataset, exist_ok=True) + return client.create_dataset(dataset, exist_ok=True) self._retry_and_handle(msg='create dataset', conn=conn, fn=fn) def _query_and_results(self, client, sql, conn, job_params): From 8acd013021c5d5548805b25d8e18f497af8c909d Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Wed, 11 Dec 2019 13:18:51 -0700 Subject: [PATCH 125/860] clean up merge conflict --- dbt/adapters/bigquery/connections.py | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 4c8b20110..08ee91ee5 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -314,11 +314,7 @@ def create_dataset(self, database, schema): dataset = self.dataset(database, schema, conn) def fn(): -<<<<<<< HEAD - return client.create_dataset(dataset, exist_ok=True) -======= - client.create_dataset(dataset, exists_ok=True) ->>>>>>> c5c7932ee738cadfd9307f0d07253851cbfeb2aa + return client.create_dataset(dataset, exists_ok=True) self._retry_and_handle(msg='create dataset', conn=conn, fn=fn) def _query_and_results(self, client, sql, conn, job_params): From 4a5b9b2ee637d8262c3ac5a8d6c9c75520da06d5 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 12 Dec 2019 15:49:59 -0700 Subject: [PATCH 126/860] In bigquery, filter out missing schemas before querying the catalog --- dbt/adapters/bigquery/impl.py | 29 ++++++++++++++++++++++--- dbt/include/bigquery/macros/catalog.sql | 20 ++++++++++++----- 2 files changed, 40 insertions(+), 9 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 05e22772c..689eb9d70 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -13,6 +13,7 @@ from dbt.adapters.bigquery import BigQueryColumn from dbt.adapters.bigquery import BigQueryConnectionManager from dbt.contracts.connection import Connection +from dbt.contracts.graph.manifest import Manifest from dbt.logger import GLOBAL_LOGGER as logger from dbt.utils import filter_null_values @@ -479,10 +480,32 @@ def load_dataframe(self, database, schema, table_name, agate_table, with self.connections.exception_handler("LOAD TABLE"): self.poll_until_job_completes(job, timeout) - def _catalog_filter_table(self, table, manifest): - # BigQuery doesn't allow ":" chars in column names -- remap them here. + @classmethod + def _catalog_filter_table( + cls, table: agate.Table, manifest: Manifest + ) -> agate.Table: table = table.rename(column_names={ col.name: col.name.replace('__', ':') for col in table.columns }) - return super()._catalog_filter_table(table, manifest) + + def _get_catalog_information_schemas( + self, manifest: Manifest + ) -> List[BigQueryInformationSchema]: + + candidates = super()._get_catalog_information_schemas(manifest) + information_schemas = [] + db_schemas = {} + for candidate in candidates: + database = candidate.database + if database not in db_schemas: + db_schemas[database] = set(self.list_schemas(database)) + if candidate.schema in db_schemas[database]: + information_schemas.append(candidate) + else: + logger.debug( + 'Skipping catalog for {}.{} - schema does not exist' + .format(database, candidate.schema) + ) + + return information_schemas diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql index a20d9865e..bc013b2bd 100644 --- a/dbt/include/bigquery/macros/catalog.sql +++ b/dbt/include/bigquery/macros/catalog.sql @@ -1,8 +1,13 @@ {% macro bigquery__get_catalog(information_schemas) -%} - {%- call statement('catalog', fetch_result=True) -%} - {% for information_schema in information_schemas %} + {%- if (information_schemas | length) == 0 -%} + {# Hopefully nothing cares about the columns we return when there are no rows #} + {%- set query = "select 1 as id limit 0" -%} + {%- else -%} + + {%- set query -%} + {%- for information_schema in information_schemas -%} ( with schemas as ( @@ -214,8 +219,11 @@ ) {% if not loop.last %} union all {% endif %} - {% endfor %} - {%- endcall -%} - {{ return(load_result('catalog').table) }} + {%- endfor -%} + {%- endset -%} + + {%- endif -%} + + {{ return(run_query(query)) }} -{% endmacro %} +{%- endmacro %} From 42c4a94dfe91481975ceb7da327805512034e184 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 13 Dec 2019 11:04:52 -0700 Subject: [PATCH 127/860] put an upper bound on bigquery, add six --- setup.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 578529010..eb723820e 100644 --- a/setup.py +++ b/setup.py @@ -40,7 +40,12 @@ }, install_requires=[ 'dbt-core=={}'.format(package_version), - 'google-cloud-bigquery>=1.15.0,<2', + 'google-cloud-core>=1,<=1.1.0', + 'google-cloud-bigquery>=1.15.0,<1.24.0', + # hidden secret dependency: bq requires this but only documents 1.10.0 + # through its dependency chain. + # see https://github.com/googleapis/google-cloud-python/issues/9965 + 'six>=1.13.0', ], zip_safe=False, classifiers=[ From 9077547bdd15342e92137655265ca99c3ed156ac Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Wed, 18 Dec 2019 15:37:30 -0500 Subject: [PATCH 128/860] =?UTF-8?q?Bump=20version:=200.15.0=20=E2=86=92=20?= =?UTF-8?q?0.15.1rc1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index eb723820e..a54ef6b1a 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.15.0" +package_version = "0.15.1rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 1af6959f7965b7a707bcef29f83a3e879d495585 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 19 Dec 2019 10:22:10 -0700 Subject: [PATCH 129/860] fixes --- dbt/adapters/bigquery/connections.py | 24 +++++++++++++++--------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 08ee91ee5..35146bee2 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -176,9 +176,12 @@ def get_timeout(cls, conn): return credentials.timeout_seconds @classmethod - def get_retries(cls, conn): + def get_retries(cls, conn) -> int: credentials = conn.credentials - return credentials.retries + if credentials.retries is not None: + return credentials.retries + else: + return 1 @classmethod def get_table_from_response(cls, resp): @@ -270,8 +273,11 @@ def create_table(self, database, schema, table_name, sql): job_params = {'destination': table_ref, 'write_disposition': 'WRITE_TRUNCATE'} + timeout = self.get_timeout(conn) + def fn(): - return self._query_and_results(client, sql, conn, job_params) + return self._query_and_results(client, sql, conn, job_params, + timeout=timeout) self._retry_and_handle(msg=sql, conn=conn, fn=fn) def create_date_partitioned_table(self, database, schema, table_name): @@ -317,12 +323,12 @@ def fn(): return client.create_dataset(dataset, exists_ok=True) self._retry_and_handle(msg='create dataset', conn=conn, fn=fn) - def _query_and_results(self, client, sql, conn, job_params): + def _query_and_results(self, client, sql, conn, job_params, timeout=None): """Query the client and wait for results.""" # Cannot reuse job_config if destination is set and ddl is used job_config = google.cloud.bigquery.QueryJobConfig(**job_params) query_job = client.query(sql, job_config=job_config) - iterator = query_job.result(timeout=self.get_timeout(conn)) + iterator = query_job.result(timeout=timeout) return query_job, iterator @@ -354,13 +360,13 @@ def count_error(self, error): return False # Don't log self.error_count += 1 if _is_retryable(error) and self.error_count <= self.retries: - logger.warning( - 'Retry attempt %s of %s after error: %s', + logger.debug( + 'Retry attempt {} of {} after error: {}', self.error_count, self.retries, repr(error)) return True else: - logger.warning( - 'Not Retrying after %s previous attempts. Error: %s', + logger.debug( + 'Not Retrying after {} previous attempts. Error: {}', self.error_count - 1, repr(error)) return False From bc2dde12881176f9c9601aca7ee5e2b8619420fa Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Mon, 30 Dec 2019 13:44:38 -0500 Subject: [PATCH 130/860] =?UTF-8?q?Bump=20version:=200.15.1rc1=20=E2=86=92?= =?UTF-8?q?=200.15.1rc2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index a54ef6b1a..27d79ec5f 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.15.1rc1" +package_version = "0.15.1rc2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 1f2a0bfed4289d066a0d3b7b73861e53f3e7ec0d Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 13 Jan 2020 14:23:41 -0700 Subject: [PATCH 131/860] add "is_number" and "is_float" Column methods Split out snowflake column type added column type test integration tests --- dbt/adapters/bigquery/column.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py index 8c8a442b4..305bae77a 100644 --- a/dbt/adapters/bigquery/column.py +++ b/dbt/adapters/bigquery/column.py @@ -99,8 +99,15 @@ def data_type(self) -> str: def is_string(self) -> bool: return self.dtype.lower() == 'string' + def is_integer(self) -> bool: + # snowflake technicality: These are all synonyms with NUMBER(38, 0) + return self.dtype.lower() == 'int64' + def is_numeric(self) -> bool: - return False + return self.dtype.lower() == 'numeric' + + def is_float(self): + return self.dtype.lower() == 'float64' def can_expand_to(self: Self, other_column: Self) -> bool: """returns True if both columns are strings""" From 0e9ed926d9b23615afcc9491b180f7f1d5eb2278 Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Thu, 16 Jan 2020 08:46:19 -0500 Subject: [PATCH 132/860] =?UTF-8?q?Bump=20version:=200.15.1rc2=20=E2=86=92?= =?UTF-8?q?=200.15.1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 27d79ec5f..f5c6358dd 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.15.1rc2" +package_version = "0.15.1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From f103b228247d4351e78b46354072e16dce61e817 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 17 Jan 2020 09:31:00 -0700 Subject: [PATCH 133/860] =?UTF-8?q?Bump=20version:=200.15.1=20=E2=86=92=20?= =?UTF-8?q?0.15.2a1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index f5c6358dd..b146a0ad4 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.15.1" +package_version = "0.15.2a1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From b14a0c78f274428c89dfe6c2631b6c25b3bcfb38 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 24 Jan 2020 08:27:19 -0700 Subject: [PATCH 134/860] PR feedback --- dbt/adapters/bigquery/column.py | 1 - 1 file changed, 1 deletion(-) diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py index 305bae77a..6c8a70df1 100644 --- a/dbt/adapters/bigquery/column.py +++ b/dbt/adapters/bigquery/column.py @@ -100,7 +100,6 @@ def is_string(self) -> bool: return self.dtype.lower() == 'string' def is_integer(self) -> bool: - # snowflake technicality: These are all synonyms with NUMBER(38, 0) return self.dtype.lower() == 'int64' def is_numeric(self) -> bool: From 0c176b3a991694f88f3fcac5fecb499fa268ff44 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 9 Jan 2020 13:51:55 -0700 Subject: [PATCH 135/860] migrate other adapters to use the snowflake technique --- dbt/adapters/bigquery/impl.py | 22 +- dbt/include/bigquery/macros/catalog.sql | 360 ++++++++++++------------ 2 files changed, 190 insertions(+), 192 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index de02f5f2d..1e58c73a5 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -1,4 +1,4 @@ -from typing import Dict, List, Optional, Any +from typing import Dict, List, Optional, Any, Set import dbt.deprecations import dbt.exceptions @@ -7,6 +7,7 @@ import dbt.clients.agate_helper from dbt.adapters.base import BaseAdapter, available, RelationType +from dbt.adapters.base.impl import SchemaSearchMap from dbt.adapters.bigquery.relation import ( BigQueryRelation, BigQueryInformationSchema ) @@ -492,23 +493,22 @@ def _catalog_filter_table( }) return super()._catalog_filter_table(table, manifest) - def _get_catalog_information_schemas( - self, manifest: Manifest - ) -> List[BigQueryInformationSchema]: + def _get_cache_schemas( + self, manifest: Manifest, exec_only: bool = False + ) -> SchemaSearchMap: + candidates = super()._get_cache_schemas(manifest, exec_only) + db_schemas: Dict[str, Set[str]] = {} + result = SchemaSearchMap() - candidates = super()._get_catalog_information_schemas(manifest) - information_schemas = [] - db_schemas = {} - for candidate in candidates: + for candidate, schemas in candidates.items(): database = candidate.database if database not in db_schemas: db_schemas[database] = set(self.list_schemas(database)) if candidate.schema in db_schemas[database]: - information_schemas.append(candidate) + result[candidate] = schemas else: logger.debug( 'Skipping catalog for {}.{} - schema does not exist' .format(database, candidate.schema) ) - - return information_schemas + return result diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql index bc013b2bd..1f468f0b3 100644 --- a/dbt/include/bigquery/macros/catalog.sql +++ b/dbt/include/bigquery/macros/catalog.sql @@ -1,225 +1,223 @@ -{% macro bigquery__get_catalog(information_schemas) -%} +{% macro bigquery__get_catalog(information_schema, schemas) -%} - {%- if (information_schemas | length) == 0 -%} + {%- if (schemas | length) == 0 -%} {# Hopefully nothing cares about the columns we return when there are no rows #} {%- set query = "select 1 as id limit 0" -%} {%- else -%} {%- set query -%} - {%- for information_schema in information_schemas -%} - ( - with schemas as ( + with schemas as ( - select - catalog_name as table_database, - schema_name as table_schema, - location - - from {{ information_schema.replace(information_schema_view='SCHEMATA') }} - - ), - - tables as ( - select - project_id as table_database, - dataset_id as table_schema, - table_id as original_table_name, - - concat(project_id, '.', dataset_id, '.', table_id) as relation_id, - - row_count, - size_bytes as size_bytes, - case - when type = 1 then 'table' - when type = 2 then 'view' - else 'external' - end as table_type, + select + catalog_name as table_database, + schema_name as table_schema, + location + + from {{ information_schema.replace(information_schema_view='SCHEMATA') }} + where ( + {%- for schema in schemas -%} + schema_name = '{{ schema }}'{%- if not loop.last %} or {% endif -%} + {%- endfor -%} + ) + ), - REGEXP_CONTAINS(table_id, '^.+[0-9]{8}$') and coalesce(type, 0) = 1 as is_date_shard, - REGEXP_EXTRACT(table_id, '^(.+)[0-9]{8}$') as shard_base_name, - REGEXP_EXTRACT(table_id, '^.+([0-9]{8})$') as shard_name + tables as ( + select + project_id as table_database, + dataset_id as table_schema, + table_id as original_table_name, - from {{ information_schema.replace(information_schema_view='__TABLES__') }} + concat(project_id, '.', dataset_id, '.', table_id) as relation_id, - ), + row_count, + size_bytes as size_bytes, + case + when type = 1 then 'table' + when type = 2 then 'view' + else 'external' + end as table_type, - extracted as ( + REGEXP_CONTAINS(table_id, '^.+[0-9]{8}$') and coalesce(type, 0) = 1 as is_date_shard, + REGEXP_EXTRACT(table_id, '^(.+)[0-9]{8}$') as shard_base_name, + REGEXP_EXTRACT(table_id, '^.+([0-9]{8})$') as shard_name - select *, - case - when is_date_shard then shard_base_name - else original_table_name - end as table_name + from {{ information_schema.replace(information_schema_view='__TABLES__') }} - from tables + ), - ), + extracted as ( - unsharded_tables as ( + select *, + case + when is_date_shard then shard_base_name + else original_table_name + end as table_name - select - table_database, - table_schema, - table_name, - coalesce(table_type, 'external') as table_type, - is_date_shard, + from tables - struct( - min(shard_name) as shard_min, - max(shard_name) as shard_max, - count(*) as shard_count - ) as table_shards, + ), - sum(size_bytes) as size_bytes, - sum(row_count) as row_count, + unsharded_tables as ( - max(relation_id) as relation_id + select + table_database, + table_schema, + table_name, + coalesce(table_type, 'external') as table_type, + is_date_shard, - from extracted - group by 1,2,3,4,5 + struct( + min(shard_name) as shard_min, + max(shard_name) as shard_max, + count(*) as shard_count + ) as table_shards, - ), + sum(size_bytes) as size_bytes, + sum(row_count) as row_count, - info_schema_columns as ( + max(relation_id) as relation_id - select - concat(table_catalog, '.', table_schema, '.', table_name) as relation_id, - table_catalog as table_database, - table_schema, - table_name, + from extracted + group by 1,2,3,4,5 - -- use the "real" column name from the paths query below - column_name as base_column_name, - ordinal_position as column_index, - cast(null as string) as column_comment, + ), - is_partitioning_column, - clustering_ordinal_position + info_schema_columns as ( - from {{ information_schema.replace(information_schema_view='COLUMNS') }} - where ordinal_position is not null + select + concat(table_catalog, '.', table_schema, '.', table_name) as relation_id, + table_catalog as table_database, + table_schema, + table_name, - ), + -- use the "real" column name from the paths query below + column_name as base_column_name, + ordinal_position as column_index, + cast(null as string) as column_comment, - info_schema_column_paths as ( + is_partitioning_column, + clustering_ordinal_position - select - concat(table_catalog, '.', table_schema, '.', table_name) as relation_id, - field_path as column_name, - data_type as column_type, - column_name as base_column_name + from {{ information_schema.replace(information_schema_view='COLUMNS') }} + where ordinal_position is not null - from {{ information_schema.replace(information_schema_view='COLUMN_FIELD_PATHS') }} - where data_type not like 'STRUCT%' + ), - ), + info_schema_column_paths as ( - columns as ( + select + concat(table_catalog, '.', table_schema, '.', table_name) as relation_id, + field_path as column_name, + data_type as column_type, + column_name as base_column_name - select * except (base_column_name) - from info_schema_columns - join info_schema_column_paths using (relation_id, base_column_name) + from {{ information_schema.replace(information_schema_view='COLUMN_FIELD_PATHS') }} + where data_type not like 'STRUCT%' - ), + ), - column_stats as ( + columns as ( - select - table_database, - table_schema, - table_name, - max(relation_id) as relation_id, - max(case when is_partitioning_column = 'YES' then 1 else 0 end) = 1 as is_partitioned, - max(case when is_partitioning_column = 'YES' then column_name else null end) as partition_column, - max(case when clustering_ordinal_position is not null then 1 else 0 end) = 1 as is_clustered, - array_to_string( - array_agg( - case - when clustering_ordinal_position is not null then column_name - else null - end ignore nulls - order by clustering_ordinal_position - ), ', ' - ) as clustering_columns + select * except (base_column_name) + from info_schema_columns + join info_schema_column_paths using (relation_id, base_column_name) - from columns - group by 1,2,3 + ), - ) + column_stats as ( select - unsharded_tables.table_database, - unsharded_tables.table_schema, - case - when is_date_shard then concat(unsharded_tables.table_name, '*') - else unsharded_tables.table_name - end as table_name, - unsharded_tables.table_type, - - -- coalesce name and type for External tables - these columns are not - -- present in the COLUMN_FIELD_PATHS resultset - coalesce(columns.column_name, '') as column_name, - -- invent a row number to account for nested fields -- BQ does - -- not treat these nested properties as independent fields - row_number() over ( - partition by relation_id - order by columns.column_index, columns.column_name - ) as column_index, - coalesce(columns.column_type, '') as column_type, - columns.column_comment, - - 'Location' as `stats__location__label`, - location as `stats__location__value`, - 'The geographic location of this table' as `stats__location__description`, - location is not null as `stats__location__include`, - - 'Shard count' as `stats__date_shards__label`, - table_shards.shard_count as `stats__date_shards__value`, - 'The number of date shards in this table' as `stats__date_shards__description`, - is_date_shard as `stats__date_shards__include`, - - 'Shard (min)' as `stats__date_shard_min__label`, - table_shards.shard_min as `stats__date_shard_min__value`, - 'The first date shard in this table' as `stats__date_shard_min__description`, - is_date_shard as `stats__date_shard_min__include`, - - 'Shard (max)' as `stats__date_shard_max__label`, - table_shards.shard_max as `stats__date_shard_max__value`, - 'The last date shard in this table' as `stats__date_shard_max__description`, - is_date_shard as `stats__date_shard_max__include`, - - '# Rows' as `stats__num_rows__label`, - row_count as `stats__num_rows__value`, - 'Approximate count of rows in this table' as `stats__num_rows__description`, - (unsharded_tables.table_type = 'table') as `stats__num_rows__include`, - - 'Approximate Size' as `stats__num_bytes__label`, - size_bytes as `stats__num_bytes__value`, - 'Approximate size of table as reported by BigQuery' as `stats__num_bytes__description`, - (unsharded_tables.table_type = 'table') as `stats__num_bytes__include`, - - 'Partitioned By' as `stats__partitioning_type__label`, - partition_column as `stats__partitioning_type__value`, - 'The partitioning column for this table' as `stats__partitioning_type__description`, - is_partitioned as `stats__partitioning_type__include`, - - 'Clustered By' as `stats__clustering_fields__label`, - clustering_columns as `stats__clustering_fields__value`, - 'The clustering columns for this table' as `stats__clustering_fields__description`, - is_clustered as `stats__clustering_fields__include` - - -- join using relation_id (an actual relation, not a shard prefix) to make - -- sure that column metadata is picked up through the join. This will only - -- return the column information for the "max" table in a date-sharded table set - from unsharded_tables - left join schemas using(table_database, table_schema) - left join columns using (relation_id) - left join column_stats using (relation_id) - ) - - {% if not loop.last %} union all {% endif %} - {%- endfor -%} + table_database, + table_schema, + table_name, + max(relation_id) as relation_id, + max(case when is_partitioning_column = 'YES' then 1 else 0 end) = 1 as is_partitioned, + max(case when is_partitioning_column = 'YES' then column_name else null end) as partition_column, + max(case when clustering_ordinal_position is not null then 1 else 0 end) = 1 as is_clustered, + array_to_string( + array_agg( + case + when clustering_ordinal_position is not null then column_name + else null + end ignore nulls + order by clustering_ordinal_position + ), ', ' + ) as clustering_columns + + from columns + group by 1,2,3 + + ) + + select + unsharded_tables.table_database, + unsharded_tables.table_schema, + case + when is_date_shard then concat(unsharded_tables.table_name, '*') + else unsharded_tables.table_name + end as table_name, + unsharded_tables.table_type, + + -- coalesce name and type for External tables - these columns are not + -- present in the COLUMN_FIELD_PATHS resultset + coalesce(columns.column_name, '') as column_name, + -- invent a row number to account for nested fields -- BQ does + -- not treat these nested properties as independent fields + row_number() over ( + partition by relation_id + order by columns.column_index, columns.column_name + ) as column_index, + coalesce(columns.column_type, '') as column_type, + columns.column_comment, + + 'Location' as `stats__location__label`, + location as `stats__location__value`, + 'The geographic location of this table' as `stats__location__description`, + location is not null as `stats__location__include`, + + 'Shard count' as `stats__date_shards__label`, + table_shards.shard_count as `stats__date_shards__value`, + 'The number of date shards in this table' as `stats__date_shards__description`, + is_date_shard as `stats__date_shards__include`, + + 'Shard (min)' as `stats__date_shard_min__label`, + table_shards.shard_min as `stats__date_shard_min__value`, + 'The first date shard in this table' as `stats__date_shard_min__description`, + is_date_shard as `stats__date_shard_min__include`, + + 'Shard (max)' as `stats__date_shard_max__label`, + table_shards.shard_max as `stats__date_shard_max__value`, + 'The last date shard in this table' as `stats__date_shard_max__description`, + is_date_shard as `stats__date_shard_max__include`, + + '# Rows' as `stats__num_rows__label`, + row_count as `stats__num_rows__value`, + 'Approximate count of rows in this table' as `stats__num_rows__description`, + (unsharded_tables.table_type = 'table') as `stats__num_rows__include`, + + 'Approximate Size' as `stats__num_bytes__label`, + size_bytes as `stats__num_bytes__value`, + 'Approximate size of table as reported by BigQuery' as `stats__num_bytes__description`, + (unsharded_tables.table_type = 'table') as `stats__num_bytes__include`, + + 'Partitioned By' as `stats__partitioning_type__label`, + partition_column as `stats__partitioning_type__value`, + 'The partitioning column for this table' as `stats__partitioning_type__description`, + is_partitioned as `stats__partitioning_type__include`, + + 'Clustered By' as `stats__clustering_fields__label`, + clustering_columns as `stats__clustering_fields__value`, + 'The clustering columns for this table' as `stats__clustering_fields__description`, + is_clustered as `stats__clustering_fields__include` + + -- join using relation_id (an actual relation, not a shard prefix) to make + -- sure that column metadata is picked up through the join. This will only + -- return the column information for the "max" table in a date-sharded table set + from unsharded_tables + left join schemas using(table_database, table_schema) + left join columns using (relation_id) + left join column_stats using (relation_id) {%- endset -%} {%- endif -%} From 609744cd0568e4fc53653818b3b64f2e71e58070 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 31 Jan 2020 11:10:45 -0700 Subject: [PATCH 136/860] PR feedback w/ improved catalog results behavior --- dbt/include/bigquery/macros/catalog.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql index 1f468f0b3..d41b03604 100644 --- a/dbt/include/bigquery/macros/catalog.sql +++ b/dbt/include/bigquery/macros/catalog.sql @@ -17,7 +17,7 @@ from {{ information_schema.replace(information_schema_view='SCHEMATA') }} where ( {%- for schema in schemas -%} - schema_name = '{{ schema }}'{%- if not loop.last %} or {% endif -%} + upper(schema_name) = upper('{{ schema }}'){%- if not loop.last %} or {% endif -%} {%- endfor -%} ) ), From da7994948cc4743a3ade5014ad5386837d8b0d6a Mon Sep 17 00:00:00 2001 From: Connor McArthur Date: Sun, 2 Feb 2020 14:16:13 -0500 Subject: [PATCH 137/860] =?UTF-8?q?Bump=20version:=200.15.2a1=20=E2=86=92?= =?UTF-8?q?=200.15.2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index b146a0ad4..628a8d927 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.15.2a1" +package_version = "0.15.2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 46d679d6fa59d3c70662feb0daddb0dca7708c8b Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 4 Feb 2020 19:28:34 -0700 Subject: [PATCH 138/860] =?UTF-8?q?Bump=20version:=200.15.2=20=E2=86=92=20?= =?UTF-8?q?0.16.0a2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 628a8d927..de9125414 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.15.2" +package_version = "0.16.0a2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 1a1570ded090debb4843ced6e264f1a3c5bf22a0 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 10 Feb 2020 15:17:01 -0700 Subject: [PATCH 139/860] =?UTF-8?q?Bump=20version:=200.16.0a2=20=E2=86=92?= =?UTF-8?q?=200.16.0b1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index de9125414..04cc4c30a 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.16.0a2" +package_version = "0.16.0b1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From a6e08c8330501956b6377357a2c8c16c459a7d92 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 12 Feb 2020 07:58:34 -0700 Subject: [PATCH 140/860] rev boto3/botocore/snowflake-connector-python/google libraries --- setup.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/setup.py b/setup.py index 04cc4c30a..5d1c45662 100644 --- a/setup.py +++ b/setup.py @@ -40,10 +40,10 @@ }, install_requires=[ 'dbt-core=={}'.format(package_version), - 'google-cloud-core>=1,<=1.1.0', - 'google-cloud-bigquery>=1.15.0,<1.24.0', - # hidden secret dependency: bq requires this but only documents 1.10.0 - # through its dependency chain. + 'google-cloud-core>=1,<=1.3.0', + 'google-cloud-bigquery>=1.15.0,<1.25.0', + # hidden secret dependency: bq 1.23.0 requires this but only documents + # 1.10.0 through its dependency chain. # see https://github.com/googleapis/google-cloud-python/issues/9965 'six>=1.13.0', ], From d9d62b03716ed3f8098a9a4fb3e4536fad9e9597 Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Tue, 3 Dec 2019 09:33:33 -0500 Subject: [PATCH 141/860] Implement partition-aware BigQuery merge statements --- dbt/adapters/bigquery/impl.py | 119 ++++++++++++++++-- dbt/include/bigquery/macros/adapters.sql | 67 ++++++---- .../macros/materializations/incremental.sql | 69 ++++++++-- .../macros/materializations/merge.sql | 3 - .../macros/materializations/table.sql | 7 ++ 5 files changed, 225 insertions(+), 40 deletions(-) delete mode 100644 dbt/include/bigquery/macros/materializations/merge.sql diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 1e58c73a5..c304bf9d1 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -5,6 +5,7 @@ import dbt.flags as flags import dbt.clients.gcloud import dbt.clients.agate_helper +import dbt.links from dbt.adapters.base import BaseAdapter, available, RelationType from dbt.adapters.base.impl import SchemaSearchMap @@ -26,6 +27,15 @@ import time import agate +import re + + +BQ_INTEGER_RANGE_NOT_SUPPORTED = f""" +BigQuery integer range partitioning is only supported by the +`partition_by` config, which accepts a dictionary. + +See: {dbt.links.BigQueryNewPartitionBy} +""" def _stub_relation(*args, **kwargs): @@ -413,6 +423,47 @@ def execute_model(self, model, materialization, sql_override=None, return res + def _partitions_match(self, table, conf_partition): + """ + Check if the actual and configured partitions for a table are a match. + BigQuery tables can be replaced if: + - Both tables are not partitioned, OR + - Both tables are partitioned using the exact same configs + + If there is a mismatch, then the table cannot be replaced directly. + """ + if table.partitioning_type is None and conf_partition is None: + return True + + elif conf_partition is None or table.partitioning_type is None: + return False + + elif table.partitioning_type == 'DAY': + return table.time_partitioning.field == conf_partition.get('field') + + elif table.partitioning_type == 'RANGE': # TODO + dest_part = table.range_partition.range_ + conf_part = conf_partition.get('range') + + return dest_part.field == conf_part.get('field') \ + and dest_part.start == conf_part.get('start') \ + and dest_part.end == conf_part.get('end') \ + and dest_part.interval == conf_part.get('interval') + + else: + return False + + def _clusters_match(self, table, conf_cluster): + """ + Check if the actual and configured clustering columns for a table + are a match. BigQuery tables can be replaced if clustering columns + match exactly. + """ + if isinstance(conf_cluster, str): + conf_cluster = [conf_cluster] + + return table.clustering_fields == conf_cluster + @available.parse(lambda *a, **k: True) def is_replaceable(self, relation, conf_partition, conf_cluster): """ @@ -422,6 +473,9 @@ def is_replaceable(self, relation, conf_partition, conf_cluster): partitioning spec. This method returns True if the given config spec is identical to that of the existing table. """ + if not relation: + return True + try: table = self.connections.get_bq_table( database=relation.database, @@ -431,17 +485,66 @@ def is_replaceable(self, relation, conf_partition, conf_cluster): except google.cloud.exceptions.NotFound: return True - table_partition = table.time_partitioning - if table_partition is not None: - table_partition = table_partition.field + return all(( + self._partitions_match(table, conf_partition), + self._clusters_match(table, conf_cluster) + )) - table_cluster = table.clustering_fields + @available + def parse_partition_by(self, raw_partition_by): + """ + dbt v0.16.0 expects `partition_by` to be a dictionary where previously + it was a string. Check the type of `partition_by`, raise error + or warning if string, and attempt to convert to dict. + """ + + if isinstance(raw_partition_by, dict): + if raw_partition_by.get('field'): + if raw_partition_by.get('data_type'): + return raw_partition_by + else: # assume date type as default + return dict(raw_partition_by, data_type='date') + else: + dbt.exceptions.raise_compiler_error( + 'Config `partition_by` is missing required item `field`' + ) - if isinstance(conf_cluster, str): - conf_cluster = [conf_cluster] + elif isinstance(raw_partition_by, str): + raw_partition_by = raw_partition_by.strip() + if 'range_bucket' in raw_partition_by.lower(): + dbt.exceptions.raise_compiler_error( + BQ_INTEGER_RANGE_NOT_SUPPORTED + ) + + elif raw_partition_by.lower().startswith('date('): + matches = re.match(r'date\((.+)\)', raw_partition_by, + re.IGNORECASE) + if not matches: + dbt.exceptions.raise_compiler_error( + f"Specified partition_by '{raw_partition_by}' " + "is not parseable") - return table_partition == conf_partition \ - and table_cluster == conf_cluster + partition_by = matches.group(1) + data_type = 'timestamp' + + else: + partition_by = raw_partition_by + data_type = 'date' + + inferred_partition_by = { + 'field': partition_by, + 'data_type': data_type + } + + dbt.deprecations.warn( + 'bq-partition-by-string', + raw_partition_by=raw_partition_by, + inferred_partition_by=inferred_partition_by + ) + + return inferred_partition_by + else: + return None @available.parse_none def alter_table_add_columns(self, relation, columns): diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index d67b3f47a..c1522a22e 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -1,25 +1,39 @@ -{% macro partition_by(raw_partition_by) %} - {%- if raw_partition_by is none -%} - {{ return('') }} - {% endif %} - - {% set partition_by_clause %} - partition by {{ raw_partition_by }} - {%- endset -%} +{% macro pprint_partition_field(partition_by_dict, alias = '') %} + {%- set partition_col_exp -%} + {%- if alias -%} {{alias}}.{{partition_by_dict.field}} + {%- else -%} {{partition_by_dict.field}} + {%- endif -%} + {%- endset -%} + {%- if partition_by_dict.data_type in ('timestamp','datetime') -%} + date({{partition_col_exp}}) + {%- else -%} + {{partition_col_exp}} + {%- endif -%} +{% endmacro %} - {{ return(partition_by_clause) }} +{% macro partition_by(partition_by_dict) %} + {%- if partition_by_dict is not none -%} + {%- set partition_by_type = partition_by_dict.data_type|trim|lower -%} + {%- if partition_by_type in ('date','timestamp','datetime') -%} + partition by {{pprint_partition_field(partition_by_dict)}} + {%- elif partition_by_type in ('int64') -%} + {%- set pbr = partition_by_dict.range -%} + partition by range_bucket( + {{partition_by_dict.field}}, + generate_array({{pbr.start}}, {{pbr.end}}, {{pbr.interval}}) + ) + {%- endif -%} + {%- endif -%} {%- endmacro -%} - {% macro cluster_by(raw_cluster_by) %} {%- if raw_cluster_by is not none -%} - cluster by - {% if raw_cluster_by is string -%} + cluster by {% if raw_cluster_by is string -%} {% set raw_cluster_by = [raw_cluster_by] %} {%- endif -%} {%- for cluster in raw_cluster_by -%} {{ cluster }} - {%- if not loop.last -%},{%- endif -%} + {%- if not loop.last -%}, {% endif -%} {%- endfor -%} {% endif %} @@ -33,9 +47,6 @@ {%- if description is not none -%} {%- do opts.update({'description': "'" ~ description ~ "'"}) -%} {%- endif -%} - {%- if temporary -%} - {% do opts.update({'expiration_timestamp': 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)'}) %} - {%- endif -%} {%- if kms_key_name -%} {%- do opts.update({'kms_key_name': "'" ~ kms_key_name ~ "'"}) -%} {%- endif -%} @@ -63,17 +74,24 @@ {%- set raw_labels = config.get('labels', []) -%} {%- set sql_header = config.get('sql_header', none) -%} + {%- set partition_by_dict = adapter.parse_partition_by(raw_partition_by) -%} + {{ sql_header if sql_header is not none }} - create or replace table {{ relation }} - {{ partition_by(raw_partition_by) }} + create or replace {% if temporary -%}temp{%- endif %} table + {{ relation.include(database=(not is_scripting), schema=(not is_scripting)) }} + {{ partition_by(partition_by_dict) }} {{ cluster_by(raw_cluster_by) }} - {{ bigquery_table_options( - persist_docs=raw_persist_docs, temporary=temporary, kms_key_name=raw_kms_key_name, - labels=raw_labels) }} + {%- if not temporary -%} + {{ bigquery_table_options( + persist_docs=raw_persist_docs, + kms_key_name=raw_kms_key_name, + labels=raw_labels) }} + {%- endif %} as ( {{ sql }} ); + {%- endmacro -%} @@ -134,3 +152,10 @@ {% macro bigquery__check_schema_exists(information_schema, schema) %} {{ return(adapter.check_schema_exists(information_schema.database, schema)) }} {% endmacro %} + + +{% macro bigquery__make_temp_relation(base_relation, suffix) %} + {% set tmp_identifier = base_relation.identifier ~ suffix %} + {% set tmp_relation = api.Relation.create(identifier=tmp_identifier) -%} + {% do return(tmp_relation) %} +{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index d2e62e4f8..873f10382 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -1,4 +1,41 @@ +{% macro bq_partition_merge(tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns) %} + {%- set array_datatype = + 'date' if partition_by.data_type in ('timestamp, datetime') + else partition_by.data_type -%} + + {% set predicate -%} + {{ pprint_partition_field( + partition_by, + alias = 'DBT_INTERNAL_DEST') + }} in unnest(partitions_for_upsert) + {%- endset %} + + {%- set source_sql -%} + ( + select * from {{tmp_relation.identifier}} + ) + {%- endset -%} + + -- generated script to merge partitions into {{ target_relation }} + declare partitions_for_upsert array<{{array_datatype}}>; + + -- 1. create a temp table + {{ create_table_as(True, tmp_relation, sql) }} + + -- 2. define partitions to update + set (partitions_for_upsert) = ( + select as struct + array_agg(distinct {{pprint_partition_field(partition_by)}}) + from {{tmp_relation.identifier}} + ); + + -- 3. run the merge statement + {{ get_merge_sql(target_relation, source_sql, unique_key, dest_columns, [predicate]) }} + +{% endmacro %} + + {% materialization incremental, adapter='bigquery' -%} {%- set unique_key = config.get('unique_key') -%} @@ -8,7 +45,8 @@ {%- set existing_relation = load_relation(this) %} {%- set tmp_relation = make_temp_relation(this) %} - {%- set partition_by = config.get('partition_by', none) -%} + {%- set raw_partition_by = config.get('partition_by', none) -%} + {%- set partition_by = adapter.parse_partition_by(raw_partition_by) -%} {%- set cluster_by = config.get('cluster_by', none) -%} {{ run_hooks(pre_hooks) }} @@ -29,13 +67,28 @@ {% else %} {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %} - {#-- wrap sql in parens to make it a subquery --#} - {% set source_sql -%} - ( - {{ sql }} - ) - {%- endset -%} - {% set build_sql = get_merge_sql(target_relation, source_sql, unique_key, dest_columns) %} + {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#} + {% if partition_by %} + {% set build_sql = bq_partition_merge( + tmp_relation, + target_relation, + sql, + unique_key, + partition_by, + dest_columns) %} + + {% else %} + {#-- wrap sql in parens to make it a subquery --#} + {%- set source_sql -%} + ( + {{sql}} + ) + {%- endset -%} + + {% set build_sql = get_merge_sql(target_relation, source_sql, unique_key, dest_columns) %} + + {% endif %} + {% endif %} {%- call statement('main') -%} diff --git a/dbt/include/bigquery/macros/materializations/merge.sql b/dbt/include/bigquery/macros/materializations/merge.sql deleted file mode 100644 index 8e8f42a35..000000000 --- a/dbt/include/bigquery/macros/materializations/merge.sql +++ /dev/null @@ -1,3 +0,0 @@ -{% macro bigquery__get_merge_sql(target, source, unique_key, dest_columns) %} - {{ common_get_merge_sql(target, source, unique_key, dest_columns) }} -{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql index 8ac3956ff..ea9b7e4d3 100644 --- a/dbt/include/bigquery/macros/materializations/table.sql +++ b/dbt/include/bigquery/macros/materializations/table.sql @@ -63,6 +63,13 @@ {%- set should_create = (old_relation is none or exists_not_as_table) -%} {{ make_date_partitioned_table(model, target_relation, partitions, should_create, verbose) }} {% else %} + {%- set raw_partition_by = config.get('partition_by', none) -%} + {%- set partition_by = adapter.parse_partition_by(raw_partition_by) -%} + {%- set cluster_by = config.get('cluster_by', none) -%} + {% if not adapter.is_replaceable(old_relation, partition_by, cluster_by) %} + {% do log("Hard refreshing " ~ old_relation ~ " because it is not replaceable") %} + {% do adapter.drop_relation(old_relation) %} + {% endif %} {% call statement('main') -%} {{ create_table_as(False, target_relation, sql) }} {% endcall -%} From 4923698d9d87bc7281525695b25465842b627f48 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Sat, 15 Feb 2020 16:31:56 -0500 Subject: [PATCH 142/860] report bytes billed for scripts, add _dbt_max_partition field --- dbt/adapters/bigquery/connections.py | 6 ++++++ dbt/include/bigquery/macros/adapters.sql | 2 +- .../macros/materializations/incremental.sql | 15 ++++++++++----- 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 35146bee2..261b050ac 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -11,6 +11,7 @@ import dbt.clients.agate_helper import dbt.exceptions +import dbt.utils from dbt.adapters.base import BaseConnectionManager, Credentials from dbt.logger import GLOBAL_LOGGER as logger @@ -230,6 +231,10 @@ def execute(self, sql, auto_begin=False, fetch=None): table = client.get_table(query_job.destination) status = 'CREATE TABLE ({})'.format(table.num_rows) + elif query_job.statement_type == 'SCRIPT': + billed = query_job.total_bytes_billed + status = 'SCRIPT ({} billed)'.format(dbt.utils.format_bytes(billed)) + elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: status = '{} ({})'.format( query_job.statement_type, @@ -237,6 +242,7 @@ def execute(self, sql, auto_begin=False, fetch=None): ) else: + import ipdb; ipdb.set_trace() status = 'OK' return status, res diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index c1522a22e..e4c34f191 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -82,7 +82,7 @@ {{ relation.include(database=(not is_scripting), schema=(not is_scripting)) }} {{ partition_by(partition_by_dict) }} {{ cluster_by(raw_cluster_by) }} - {%- if not temporary -%} + {%- if not temporary %} {{ bigquery_table_options( persist_docs=raw_persist_docs, kms_key_name=raw_kms_key_name, diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 873f10382..9ca640fcb 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -1,6 +1,6 @@ {% macro bq_partition_merge(tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns) %} - {%- set array_datatype = + {%- set partition_type = 'date' if partition_by.data_type in ('timestamp, datetime') else partition_by.data_type -%} @@ -8,23 +8,28 @@ {{ pprint_partition_field( partition_by, alias = 'DBT_INTERNAL_DEST') - }} in unnest(partitions_for_upsert) + }} in unnest(dbt_partitions_for_upsert) {%- endset %} {%- set source_sql -%} ( - select * from {{tmp_relation.identifier}} + select * from {{tmp_relation}} ) {%- endset -%} -- generated script to merge partitions into {{ target_relation }} - declare partitions_for_upsert array<{{array_datatype}}>; + declare dbt_partitions_for_upsert array<{{ partition_type }}>; + declare _dbt_max_partition {{ partition_by.data_type }}; + + set _dbt_max_partition = ( + select max({{ partition_by.field }}) from {{ this }} + ); -- 1. create a temp table {{ create_table_as(True, tmp_relation, sql) }} -- 2. define partitions to update - set (partitions_for_upsert) = ( + set (dbt_partitions_for_upsert) = ( select as struct array_agg(distinct {{pprint_partition_field(partition_by)}}) from {{tmp_relation.identifier}} From 0f8adc05621f7e1301b2fda0eaa7b0bf2219cc91 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Sat, 15 Feb 2020 18:22:00 -0500 Subject: [PATCH 143/860] add tests --- dbt/adapters/bigquery/connections.py | 5 ++-- dbt/adapters/bigquery/impl.py | 34 ++++++++++++++-------------- 2 files changed, 19 insertions(+), 20 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 261b050ac..b8061512f 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -232,8 +232,8 @@ def execute(self, sql, auto_begin=False, fetch=None): status = 'CREATE TABLE ({})'.format(table.num_rows) elif query_job.statement_type == 'SCRIPT': - billed = query_job.total_bytes_billed - status = 'SCRIPT ({} billed)'.format(dbt.utils.format_bytes(billed)) + processed = dbt.utils.format_bytes(query_job.total_bytes_processed) + status = f'SCRIPT ({processed} processed)' elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: status = '{} ({})'.format( @@ -242,7 +242,6 @@ def execute(self, sql, auto_begin=False, fetch=None): ) else: - import ipdb; ipdb.set_trace() status = 'OK' return status, res diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index c304bf9d1..61803fdfa 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -403,7 +403,7 @@ def execute_model(self, model, materialization, sql_override=None, if flags.STRICT_MODE: connection = self.connections.get_thread_connection() if not isinstance(connection, Connection): - raise dbt.exceptions.CompilerException( + dbt.exceptions.raise_compiler_error( f'Got {connection} - not a Connection!' ) model_uid = model.get('unique_id') @@ -423,29 +423,29 @@ def execute_model(self, model, materialization, sql_override=None, return res - def _partitions_match(self, table, conf_partition): + def _partitions_match(self, table, conf_partition: Dict[str, Any]): """ Check if the actual and configured partitions for a table are a match. BigQuery tables can be replaced if: - Both tables are not partitioned, OR - Both tables are partitioned using the exact same configs - + If there is a mismatch, then the table cannot be replaced directly. """ - if table.partitioning_type is None and conf_partition is None: - return True + is_partitioned = (table.range_partitioning or table.time_partitioning) - elif conf_partition is None or table.partitioning_type is None: - return False + if not is_partitioned and not conf_partition: + return True - elif table.partitioning_type == 'DAY': - return table.time_partitioning.field == conf_partition.get('field') + if table.time_partitioning is not None: + table_field = table.time_partitioning.field + return table_field == conf_partition.get('field') - elif table.partitioning_type == 'RANGE': # TODO + elif table.range_partitioning is not None: dest_part = table.range_partition.range_ - conf_part = conf_partition.get('range') + conf_part = conf_partition.get('range', {}) - return dest_part.field == conf_part.get('field') \ + return dest_part.field == conf_partition.get('field') \ and dest_part.start == conf_part.get('start') \ and dest_part.end == conf_part.get('end') \ and dest_part.interval == conf_part.get('interval') @@ -465,7 +465,7 @@ def _clusters_match(self, table, conf_cluster): return table.clustering_fields == conf_cluster @available.parse(lambda *a, **k: True) - def is_replaceable(self, relation, conf_partition, conf_cluster): + def is_replaceable(self, relation, conf_partition: dict, conf_cluster): """ Check if a given partition and clustering column spec for a table can replace an existing relation in the database. BigQuery does not @@ -491,13 +491,13 @@ def is_replaceable(self, relation, conf_partition, conf_cluster): )) @available - def parse_partition_by(self, raw_partition_by): + def parse_partition_by(self, raw_partition_by: Any): """ dbt v0.16.0 expects `partition_by` to be a dictionary where previously it was a string. Check the type of `partition_by`, raise error or warning if string, and attempt to convert to dict. """ - + if isinstance(raw_partition_by, dict): if raw_partition_by.get('field'): if raw_partition_by.get('data_type'): @@ -521,8 +521,8 @@ def parse_partition_by(self, raw_partition_by): re.IGNORECASE) if not matches: dbt.exceptions.raise_compiler_error( - f"Specified partition_by '{raw_partition_by}' " - "is not parseable") + f"Specified partition_by '{raw_partition_by}' " + "is not parseable") partition_by = matches.group(1) data_type = 'timestamp' From 46625a376ec1b50397befa4136c0ab41516aad22 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Sun, 16 Feb 2020 17:55:30 -0500 Subject: [PATCH 144/860] use not-real temp tables on bigquery, fix snapshots --- dbt/include/bigquery/macros/adapters.sql | 26 +++++++------------ .../macros/materializations/incremental.sql | 13 ++++++---- 2 files changed, 18 insertions(+), 21 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index e4c34f191..cacb058f1 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -47,6 +47,9 @@ {%- if description is not none -%} {%- do opts.update({'description': "'" ~ description ~ "'"}) -%} {%- endif -%} + {%- if temporary -%} + {% do opts.update({'expiration_timestamp': 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)'}) %} + {%- endif -%} {%- if kms_key_name -%} {%- do opts.update({'kms_key_name': "'" ~ kms_key_name ~ "'"}) -%} {%- endif -%} @@ -78,23 +81,21 @@ {{ sql_header if sql_header is not none }} - create or replace {% if temporary -%}temp{%- endif %} table - {{ relation.include(database=(not is_scripting), schema=(not is_scripting)) }} + create or replace table {{ relation }} {{ partition_by(partition_by_dict) }} {{ cluster_by(raw_cluster_by) }} - {%- if not temporary %} - {{ bigquery_table_options( - persist_docs=raw_persist_docs, - kms_key_name=raw_kms_key_name, - labels=raw_labels) }} - {%- endif %} + {{ bigquery_table_options( + persist_docs=raw_persist_docs, + temporary=temporary, + kms_key_name=raw_kms_key_name, + labels=raw_labels + ) }} as ( {{ sql }} ); {%- endmacro -%} - {% macro bigquery__create_view_as(relation, sql) -%} {%- set raw_persist_docs = config.get('persist_docs', {}) -%} {%- set raw_labels = config.get('labels', []) -%} @@ -152,10 +153,3 @@ {% macro bigquery__check_schema_exists(information_schema, schema) %} {{ return(adapter.check_schema_exists(information_schema.database, schema)) }} {% endmacro %} - - -{% macro bigquery__make_temp_relation(base_relation, suffix) %} - {% set tmp_identifier = base_relation.identifier ~ suffix %} - {% set tmp_relation = api.Relation.create(identifier=tmp_identifier) -%} - {% do return(tmp_relation) %} -{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 9ca640fcb..f529415d4 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -1,6 +1,6 @@ {% macro bq_partition_merge(tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns) %} - {%- set partition_type = + {%- set partition_type = 'date' if partition_by.data_type in ('timestamp, datetime') else partition_by.data_type -%} @@ -13,14 +13,14 @@ {%- set source_sql -%} ( - select * from {{tmp_relation}} + select * from {{ tmp_relation }} ) {%- endset -%} -- generated script to merge partitions into {{ target_relation }} declare dbt_partitions_for_upsert array<{{ partition_type }}>; declare _dbt_max_partition {{ partition_by.data_type }}; - + set _dbt_max_partition = ( select max({{ partition_by.field }}) from {{ this }} ); @@ -31,13 +31,16 @@ -- 2. define partitions to update set (dbt_partitions_for_upsert) = ( select as struct - array_agg(distinct {{pprint_partition_field(partition_by)}}) - from {{tmp_relation.identifier}} + array_agg(distinct {{ pprint_partition_field(partition_by) }}) + from {{ tmp_relation }} ); -- 3. run the merge statement {{ get_merge_sql(target_relation, source_sql, unique_key, dest_columns, [predicate]) }} + -- 4. clean up the temp table + drop table if exists {{ tmp_relation }} + {% endmacro %} From b39759db4aaa9bce49530aa81ece456844c3b403 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 18 Feb 2020 08:42:19 -0700 Subject: [PATCH 145/860] add a user agent with version to the bigquery client info Updated unit tests --- dbt/adapters/bigquery/connections.py | 43 ++++++++++++++++------------ 1 file changed, 25 insertions(+), 18 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 35146bee2..82fd74ead 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -3,16 +3,18 @@ from typing import Optional, Any, Dict import google.auth -import google.api_core -import google.oauth2 -import google.cloud.exceptions import google.cloud.bigquery -from google.api_core import retry +import google.cloud.exceptions +from google.api_core import retry, client_info +from google.oauth2 import service_account -import dbt.clients.agate_helper -import dbt.exceptions +from dbt.clients import agate_helper, gcloud +from dbt.exceptions import ( + FailedToConnectException, RuntimeException, DatabaseException +) from dbt.adapters.base import BaseConnectionManager, Credentials from dbt.logger import GLOBAL_LOGGER as logger +from dbt.version import __version__ as dbt_version from hologram.helpers import StrEnum @@ -70,7 +72,7 @@ def handle_error(cls, error, message, sql): error_msg = "\n".join( [item['message'] for item in error.errors]) - raise dbt.exceptions.DatabaseException(error_msg) from error + raise DatabaseException(error_msg) from error def clear_transaction(self): pass @@ -91,12 +93,12 @@ def exception_handler(self, sql): except Exception as e: logger.debug("Unhandled error while running:\n{}".format(sql)) logger.debug(e) - if isinstance(e, dbt.exceptions.RuntimeException): + if isinstance(e, RuntimeException): # during a sql query, an internal to dbt exception was raised. # this sounds a lot like a signal handler and probably has # useful information, so raise it without modification. raise - raise dbt.exceptions.RuntimeException(str(e)) from e + raise RuntimeException(str(e)) from e def cancel_open(self) -> None: pass @@ -116,7 +118,7 @@ def commit(self): @classmethod def get_bigquery_credentials(cls, profile_credentials): method = profile_credentials.method - creds = google.oauth2.service_account.Credentials + creds = service_account.Credentials if method == BigQueryConnectionMethod.OAUTH: credentials, project_id = google.auth.default(scopes=cls.SCOPE) @@ -131,15 +133,21 @@ def get_bigquery_credentials(cls, profile_credentials): return creds.from_service_account_info(details, scopes=cls.SCOPE) error = ('Invalid `method` in profile: "{}"'.format(method)) - raise dbt.exceptions.FailedToConnectException(error) + raise FailedToConnectException(error) @classmethod def get_bigquery_client(cls, profile_credentials): database = profile_credentials.database creds = cls.get_bigquery_credentials(profile_credentials) location = getattr(profile_credentials, 'location', None) - return google.cloud.bigquery.Client(database, creds, - location=location) + + info = client_info.ClientInfo(user_agent=f'dbt-{dbt_version}') + return google.cloud.bigquery.Client( + database, + creds, + location=location, + client_info=info, + ) @classmethod def open(cls, connection): @@ -152,7 +160,7 @@ def open(cls, connection): except google.auth.exceptions.DefaultCredentialsError: logger.info("Please log into GCP to continue") - dbt.clients.gcloud.setup_default_credentials() + gcloud.setup_default_credentials() handle = cls.get_bigquery_client(connection.credentials) @@ -164,7 +172,7 @@ def open(cls, connection): connection.handle = None connection.state = 'fail' - raise dbt.exceptions.FailedToConnectException(str(e)) + raise FailedToConnectException(str(e)) connection.handle = handle connection.state = 'open' @@ -186,8 +194,7 @@ def get_retries(cls, conn) -> int: @classmethod def get_table_from_response(cls, resp): column_names = [field.name for field in resp.schema] - return dbt.clients.agate_helper.table_from_data_flat(resp, - column_names) + return agate_helper.table_from_data_flat(resp, column_names) def raw_execute(self, sql, fetch=False): conn = self.get_thread_connection() @@ -219,7 +226,7 @@ def execute(self, sql, auto_begin=False, fetch=None): if fetch: res = self.get_table_from_response(iterator) else: - res = dbt.clients.agate_helper.empty_table() + res = agate_helper.empty_table() if query_job.statement_type == 'CREATE_VIEW': status = 'CREATE VIEW' From 6c2780b3236dc7eaa2c5285008453ede5570cdaf Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 18 Feb 2020 10:42:48 -0700 Subject: [PATCH 146/860] =?UTF-8?q?Bump=20version:=200.15.2=20=E2=86=92=20?= =?UTF-8?q?0.15.3a1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 628a8d927..527eb149b 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.15.2" +package_version = "0.15.3a1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 6594699749d5ddb7cd605c28846b07b1a4487aa0 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 19 Feb 2020 12:38:01 -0700 Subject: [PATCH 147/860] =?UTF-8?q?Bump=20version:=200.15.3a1=20=E2=86=92?= =?UTF-8?q?=200.15.3rc1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 527eb149b..c9bd7529c 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.15.3a1" +package_version = "0.15.3rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 5fc5b797c86ecbc3494c79bbb578578956cd3cc2 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Wed, 19 Feb 2020 18:07:29 -0500 Subject: [PATCH 148/860] quiet bq error logs; refactor to use dataclasses --- dbt/adapters/bigquery/connections.py | 23 ++-- dbt/adapters/bigquery/impl.py | 117 +++++++++++------- .../macros/materializations/incremental.sql | 4 +- 3 files changed, 79 insertions(+), 65 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index d021f27db..98b0d8397 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -67,13 +67,9 @@ class BigQueryConnectionManager(BaseConnectionManager): DEFAULT_MAXIMUM_DELAY = 1.0 # Seconds @classmethod - def handle_error(cls, error, message, sql): - logger.debug(message.format(sql=sql)) - logger.debug(str(error)) - error_msg = "\n".join( - [item['message'] for item in error.errors]) - - raise DatabaseException(error_msg) from error + def handle_error(cls, error, message): + error_msg = "\n".join([item['message'] for item in error.errors]) + raise DatabaseException(error_msg) def clear_transaction(self): pass @@ -84,12 +80,12 @@ def exception_handler(self, sql): yield except google.cloud.exceptions.BadRequest as e: - message = "Bad request while running:\n{sql}" - self.handle_error(e, message, sql) + message = "Bad request while running query" + self.handle_error(e, message) except google.cloud.exceptions.Forbidden as e: - message = "Access denied while running:\n{sql}" - self.handle_error(e, message, sql) + message = "Access denied while running query" + self.handle_error(e, message) except Exception as e: logger.debug("Unhandled error while running:\n{}".format(sql)) @@ -99,7 +95,7 @@ def exception_handler(self, sql): # this sounds a lot like a signal handler and probably has # useful information, so raise it without modification. raise - raise RuntimeException(str(e)) from e + raise RuntimeException(str(e)) def cancel_open(self) -> None: pass @@ -377,9 +373,6 @@ def count_error(self, error): self.error_count, self.retries, repr(error)) return True else: - logger.debug( - 'Not Retrying after {} previous attempts. Error: {}', - self.error_count - 1, repr(error)) return False diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 61803fdfa..f017b113c 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -1,3 +1,4 @@ +from dataclasses import dataclass from typing import Dict, List, Optional, Any, Set import dbt.deprecations @@ -38,6 +39,73 @@ """ +@dataclass +class PartitionConfig(Dict): + field: str + data_type: str + + @classmethod + def _parse(cls, raw_partition_by) -> Optional['PartitionConfig']: + if isinstance(raw_partition_by, dict): + if raw_partition_by.get('field'): + if raw_partition_by.get('data_type'): + return cls(**raw_partition_by) + else: # assume date type as default + return cls(**raw_partition_by, data_type='date') + else: + dbt.exceptions.raise_compiler_error( + 'Config `partition_by` is missing required item `field`' + ) + + elif isinstance(raw_partition_by, str): + raw_partition_by = raw_partition_by.strip() + if 'range_bucket' in raw_partition_by.lower(): + dbt.exceptions.raise_compiler_error( + BQ_INTEGER_RANGE_NOT_SUPPORTED + ) + + elif raw_partition_by.lower().startswith('date('): + matches = re.match(r'date\((.+)\)', raw_partition_by, + re.IGNORECASE) + if not matches: + dbt.exceptions.raise_compiler_error( + f"Specified partition_by '{raw_partition_by}' " + "is not parseable") + + partition_by = matches.group(1) + data_type = 'timestamp' + + else: + partition_by = raw_partition_by + data_type = 'date' + + inferred_partition_by = { + 'field': partition_by, + 'data_type': data_type + } + + dbt.deprecations.warn( + 'bq-partition-by-string', + raw_partition_by=raw_partition_by, + inferred_partition_by=inferred_partition_by + ) + + return cls(**inferred_partition_by) + else: + return None + + @classmethod + def parse(cls, raw_partition_by) -> Optional['PartitionConfig']: + try: + return cls._parse(raw_partition_by) + except TypeError: + dbt.exceptions.raise_compiler_error( + f'Invalid partition_by config:\n' + f' Got: {raw_partition_by}\n' + f' Expected a dictionary with "field" and "data_type" keys' + ) + + def _stub_relation(*args, **kwargs): return BigQueryRelation.create( database='', @@ -497,54 +565,7 @@ def parse_partition_by(self, raw_partition_by: Any): it was a string. Check the type of `partition_by`, raise error or warning if string, and attempt to convert to dict. """ - - if isinstance(raw_partition_by, dict): - if raw_partition_by.get('field'): - if raw_partition_by.get('data_type'): - return raw_partition_by - else: # assume date type as default - return dict(raw_partition_by, data_type='date') - else: - dbt.exceptions.raise_compiler_error( - 'Config `partition_by` is missing required item `field`' - ) - - elif isinstance(raw_partition_by, str): - raw_partition_by = raw_partition_by.strip() - if 'range_bucket' in raw_partition_by.lower(): - dbt.exceptions.raise_compiler_error( - BQ_INTEGER_RANGE_NOT_SUPPORTED - ) - - elif raw_partition_by.lower().startswith('date('): - matches = re.match(r'date\((.+)\)', raw_partition_by, - re.IGNORECASE) - if not matches: - dbt.exceptions.raise_compiler_error( - f"Specified partition_by '{raw_partition_by}' " - "is not parseable") - - partition_by = matches.group(1) - data_type = 'timestamp' - - else: - partition_by = raw_partition_by - data_type = 'date' - - inferred_partition_by = { - 'field': partition_by, - 'data_type': data_type - } - - dbt.deprecations.warn( - 'bq-partition-by-string', - raw_partition_by=raw_partition_by, - inferred_partition_by=inferred_partition_by - ) - - return inferred_partition_by - else: - return None + return PartitionConfig.parse(raw_partition_by) @available.parse_none def alter_table_add_columns(self, relation, columns): diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index f529415d4..8c0d83697 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -36,7 +36,7 @@ ); -- 3. run the merge statement - {{ get_merge_sql(target_relation, source_sql, unique_key, dest_columns, [predicate]) }} + {{ get_merge_sql(target_relation, source_sql, unique_key, dest_columns, [predicate]) }}; -- 4. clean up the temp table drop table if exists {{ tmp_relation }} @@ -76,7 +76,7 @@ {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %} {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#} - {% if partition_by %} + {% if partition_by is not none %} {% set build_sql = bq_partition_merge( tmp_relation, target_relation, From 9d013439ddbefcc6305a19df1809a58a8b8efeb1 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Wed, 19 Feb 2020 21:11:42 -0500 Subject: [PATCH 149/860] fix tests --- dbt/adapters/bigquery/impl.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index f017b113c..e19a731cf 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -1,5 +1,6 @@ from dataclasses import dataclass from typing import Dict, List, Optional, Any, Set +from hologram import JsonSchemaMixin import dbt.deprecations import dbt.exceptions @@ -40,7 +41,7 @@ @dataclass -class PartitionConfig(Dict): +class PartitionConfig(JsonSchemaMixin): field: str data_type: str From 595a5ce1f4f5a4094471535da37db68efd214be3 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Wed, 19 Feb 2020 21:33:09 -0500 Subject: [PATCH 150/860] push some gnarly logic out of jinja and into python --- dbt/adapters/bigquery/impl.py | 13 ++++++- dbt/include/bigquery/macros/adapters.sql | 35 ++++++------------- .../macros/materializations/incremental.sql | 7 ++-- 3 files changed, 25 insertions(+), 30 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index e19a731cf..007ebe5ba 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -41,9 +41,20 @@ @dataclass -class PartitionConfig(JsonSchemaMixin): +class PartitionConfig(Dict[str, Any], JsonSchemaMixin): field: str data_type: str + range: Optional[Dict[str, any]] = None + + def render(self, alias: Optional[str] = None): + column: str = self.field + if alias: + column = f'{alias}.{self.field}' + + if self.data_type in ('timestamp', 'datetime'): + return f'date({column})' + else: + return column @classmethod def _parse(cls, raw_partition_by) -> Optional['PartitionConfig']: diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index cacb058f1..a9d3218c5 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -1,28 +1,15 @@ -{% macro pprint_partition_field(partition_by_dict, alias = '') %} - {%- set partition_col_exp -%} - {%- if alias -%} {{alias}}.{{partition_by_dict.field}} - {%- else -%} {{partition_by_dict.field}} - {%- endif -%} - {%- endset -%} - {%- if partition_by_dict.data_type in ('timestamp','datetime') -%} - date({{partition_col_exp}}) - {%- else -%} - {{partition_col_exp}} - {%- endif -%} -{% endmacro %} -{% macro partition_by(partition_by_dict) %} - {%- if partition_by_dict is not none -%} - {%- set partition_by_type = partition_by_dict.data_type|trim|lower -%} - {%- if partition_by_type in ('date','timestamp','datetime') -%} - partition by {{pprint_partition_field(partition_by_dict)}} - {%- elif partition_by_type in ('int64') -%} - {%- set pbr = partition_by_dict.range -%} - partition by range_bucket( - {{partition_by_dict.field}}, - generate_array({{pbr.start}}, {{pbr.end}}, {{pbr.interval}}) - ) - {%- endif -%} +{% macro partition_by(partition_config) -%} + {%- if partition_config is none -%} + {% do return('') %} + {%- elif partition_config.data_type | lower in ('date','timestamp','datetime') -%} + partition by {{ partition_config.render() }} + {%- elif partition_config.data_type | lower in ('int64') -%} + {%- set range = partition_config.range -%} + partition by range_bucket( + {{ partition_config.field }}, + generate_array({{ range.start}}, {{ range.end }}, {{ range.interval }}) + ) {%- endif -%} {%- endmacro -%} diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 8c0d83697..66e1c48c6 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -5,10 +5,7 @@ else partition_by.data_type -%} {% set predicate -%} - {{ pprint_partition_field( - partition_by, - alias = 'DBT_INTERNAL_DEST') - }} in unnest(dbt_partitions_for_upsert) + {{ partition_by.render(alias='DBT_INTERNAL_DEST') }} in unnest(dbt_partitions_for_upsert) {%- endset %} {%- set source_sql -%} @@ -31,7 +28,7 @@ -- 2. define partitions to update set (dbt_partitions_for_upsert) = ( select as struct - array_agg(distinct {{ pprint_partition_field(partition_by) }}) + array_agg(distinct {{ partition_by.render() }}) from {{ tmp_relation }} ); From 5e7545d862f6cb19eec3b07d0f7af472c9fa2ce3 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Fri, 21 Feb 2020 12:49:53 -0500 Subject: [PATCH 151/860] wip --- dbt/adapters/bigquery/impl.py | 26 ++++++++++-------------- dbt/include/bigquery/macros/adapters.sql | 4 ++-- 2 files changed, 13 insertions(+), 17 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 007ebe5ba..b3869b40f 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -1,6 +1,6 @@ from dataclasses import dataclass from typing import Dict, List, Optional, Any, Set -from hologram import JsonSchemaMixin +from hologram import JsonSchemaMixin, ValidationError import dbt.deprecations import dbt.exceptions @@ -41,9 +41,9 @@ @dataclass -class PartitionConfig(Dict[str, Any], JsonSchemaMixin): +class PartitionConfig(JsonSchemaMixin): field: str - data_type: str + data_type: str = 'date' range: Optional[Dict[str, any]] = None def render(self, alias: Optional[str] = None): @@ -59,12 +59,9 @@ def render(self, alias: Optional[str] = None): @classmethod def _parse(cls, raw_partition_by) -> Optional['PartitionConfig']: if isinstance(raw_partition_by, dict): - if raw_partition_by.get('field'): - if raw_partition_by.get('data_type'): - return cls(**raw_partition_by) - else: # assume date type as default - return cls(**raw_partition_by, data_type='date') - else: + try: + return cls.from_dict(raw_partition_by) + except ValidationError: dbt.exceptions.raise_compiler_error( 'Config `partition_by` is missing required item `field`' ) @@ -91,18 +88,17 @@ def _parse(cls, raw_partition_by) -> Optional['PartitionConfig']: partition_by = raw_partition_by data_type = 'date' - inferred_partition_by = { - 'field': partition_by, - 'data_type': data_type - } + inferred_partition_by = cls( + field=partition_by, + data_type=data_type + ) dbt.deprecations.warn( 'bq-partition-by-string', raw_partition_by=raw_partition_by, inferred_partition_by=inferred_partition_by ) - - return cls(**inferred_partition_by) + return inferred_partition_by else: return None diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index a9d3218c5..290977d76 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -64,12 +64,12 @@ {%- set raw_labels = config.get('labels', []) -%} {%- set sql_header = config.get('sql_header', none) -%} - {%- set partition_by_dict = adapter.parse_partition_by(raw_partition_by) -%} + {%- set partition_config = adapter.parse_partition_by(raw_partition_by) -%} {{ sql_header if sql_header is not none }} create or replace table {{ relation }} - {{ partition_by(partition_by_dict) }} + {{ partition_by(partition_config.to_dict()) }} {{ cluster_by(raw_cluster_by) }} {{ bigquery_table_options( persist_docs=raw_persist_docs, From 9b2079c36b8716b55510a5e7cc9abd06ff3cb86f Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 21 Feb 2020 13:00:06 -0700 Subject: [PATCH 152/860] fix tests and types, enhance error message for bad partition configs --- dbt/adapters/bigquery/impl.py | 7 ++++--- dbt/include/bigquery/macros/adapters.sql | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index b3869b40f..a815cfa21 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -44,7 +44,7 @@ class PartitionConfig(JsonSchemaMixin): field: str data_type: str = 'date' - range: Optional[Dict[str, any]] = None + range: Optional[Dict[str, Any]] = None def render(self, alias: Optional[str] = None): column: str = self.field @@ -61,9 +61,10 @@ def _parse(cls, raw_partition_by) -> Optional['PartitionConfig']: if isinstance(raw_partition_by, dict): try: return cls.from_dict(raw_partition_by) - except ValidationError: + except ValidationError as exc: + msg = dbt.exceptions.validator_error_message(exc) dbt.exceptions.raise_compiler_error( - 'Config `partition_by` is missing required item `field`' + f'Could not parse partition config: {msg}' ) elif isinstance(raw_partition_by, str): diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 290977d76..a7530d43b 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -69,7 +69,7 @@ {{ sql_header if sql_header is not none }} create or replace table {{ relation }} - {{ partition_by(partition_config.to_dict()) }} + {{ partition_by(partition_config) }} {{ cluster_by(raw_cluster_by) }} {{ bigquery_table_options( persist_docs=raw_persist_docs, From 89f30c7cde0ee1f554afff7dfcbcfc7f401d3c0e Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 21 Feb 2020 13:00:06 -0700 Subject: [PATCH 153/860] fix dict/PartitionConfig confusion Also, make tests run --- dbt/adapters/bigquery/impl.py | 30 ++++++++++++++++++------------ 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index a815cfa21..4c28f99fd 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -500,7 +500,9 @@ def execute_model(self, model, materialization, sql_override=None, return res - def _partitions_match(self, table, conf_partition: Dict[str, Any]): + def _partitions_match( + self, table, conf_partition: Optional[PartitionConfig] + ) -> bool: """ Check if the actual and configured partitions for a table are a match. BigQuery tables can be replaced if: @@ -513,24 +515,21 @@ def _partitions_match(self, table, conf_partition: Dict[str, Any]): if not is_partitioned and not conf_partition: return True - - if table.time_partitioning is not None: + elif conf_partition and table.time_partitioning is not None: table_field = table.time_partitioning.field - return table_field == conf_partition.get('field') - - elif table.range_partitioning is not None: + return table_field == conf_partition.field + elif conf_partition and table.range_partitioning is not None: dest_part = table.range_partition.range_ - conf_part = conf_partition.get('range', {}) + conf_part = conf_partition.range or {} - return dest_part.field == conf_partition.get('field') \ + return dest_part.field == conf_partition.field \ and dest_part.start == conf_part.get('start') \ and dest_part.end == conf_part.get('end') \ and dest_part.interval == conf_part.get('interval') - else: return False - def _clusters_match(self, table, conf_cluster): + def _clusters_match(self, table, conf_cluster) -> bool: """ Check if the actual and configured clustering columns for a table are a match. BigQuery tables can be replaced if clustering columns @@ -542,7 +541,12 @@ def _clusters_match(self, table, conf_cluster): return table.clustering_fields == conf_cluster @available.parse(lambda *a, **k: True) - def is_replaceable(self, relation, conf_partition: dict, conf_cluster): + def is_replaceable( + self, + relation, + conf_partition: Optional[PartitionConfig], + conf_cluster + ) -> bool: """ Check if a given partition and clustering column spec for a table can replace an existing relation in the database. BigQuery does not @@ -568,7 +572,9 @@ def is_replaceable(self, relation, conf_partition: dict, conf_cluster): )) @available - def parse_partition_by(self, raw_partition_by: Any): + def parse_partition_by( + self, raw_partition_by: Any + ) -> Optional[PartitionConfig]: """ dbt v0.16.0 expects `partition_by` to be a dictionary where previously it was a string. Check the type of `partition_by`, raise error From d80b62c2b15a64a661361648f2a8a246b6e10efe Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 24 Feb 2020 08:07:31 -0700 Subject: [PATCH 154/860] Update plugins/bigquery/dbt/adapters/bigquery/impl.py Co-Authored-By: Drew Banin --- dbt/adapters/bigquery/impl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 4c28f99fd..10a8380ca 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -97,7 +97,7 @@ def _parse(cls, raw_partition_by) -> Optional['PartitionConfig']: dbt.deprecations.warn( 'bq-partition-by-string', raw_partition_by=raw_partition_by, - inferred_partition_by=inferred_partition_by + inferred_partition_by=inferred_partition_by.to_dict() ) return inferred_partition_by else: From 8579088f1a0b76fbc3d4200d95aaa89e17cef103 Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Tue, 25 Feb 2020 01:33:00 -0500 Subject: [PATCH 155/860] Fixup range_partitioning comparison logic --- dbt/adapters/bigquery/impl.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 10a8380ca..f7f658142 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -519,13 +519,13 @@ def _partitions_match( table_field = table.time_partitioning.field return table_field == conf_partition.field elif conf_partition and table.range_partitioning is not None: - dest_part = table.range_partition.range_ + dest_part = table.range_partitioning conf_part = conf_partition.range or {} return dest_part.field == conf_partition.field \ - and dest_part.start == conf_part.get('start') \ - and dest_part.end == conf_part.get('end') \ - and dest_part.interval == conf_part.get('interval') + and dest_part.range_.start == conf_part.get('start') \ + and dest_part.range_.end == conf_part.get('end') \ + and dest_part.range_.interval == conf_part.get('interval') else: return False From 9f7a6112859b43534a98fdc0f611cf34b39c4128 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 26 Feb 2020 16:39:00 -0700 Subject: [PATCH 156/860] =?UTF-8?q?Bump=20version:=200.16.0b1=20=E2=86=92?= =?UTF-8?q?=200.16.0b2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 5d1c45662..1ac1e169a 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.16.0b1" +package_version = "0.16.0b2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From f04c10e03006ad0719bc3a27450d598a67724acb Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 26 Feb 2020 16:43:26 -0700 Subject: [PATCH 157/860] =?UTF-8?q?Bump=20version:=200.16.0b2=20=E2=86=92?= =?UTF-8?q?=200.16.0b3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 1ac1e169a..d5c7fbe7e 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.16.0b2" +package_version = "0.16.0b3" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 1f42589e7d0a929276eaf804467ba8c618089828 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 28 Feb 2020 11:26:15 -0700 Subject: [PATCH 158/860] refactor, fix flake8/mypy --- dbt/adapters/bigquery/impl.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index f7f658142..d277715ae 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -9,8 +9,9 @@ import dbt.clients.agate_helper import dbt.links -from dbt.adapters.base import BaseAdapter, available, RelationType -from dbt.adapters.base.impl import SchemaSearchMap +from dbt.adapters.base import ( + BaseAdapter, available, RelationType, SchemaSearchMap +) from dbt.adapters.bigquery.relation import ( BigQueryRelation, BigQueryInformationSchema ) From 0bed656af231ec33bc98439b29152b44d829016a Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 28 Feb 2020 12:04:31 -0700 Subject: [PATCH 159/860] strip out the database qutoes on bigquery for the API --- dbt/adapters/bigquery/impl.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index d277715ae..3ae6f40a5 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -180,6 +180,9 @@ def rename_relation( @available def list_schemas(self, database: str) -> List[str]: + # the database string we get here is potentially quoted. Strip that off + # for the API call. + database = database.strip('`') conn = self.connections.get_thread_connection() client = conn.handle From 87aaeb02bcc78960b7452f18ad5ddb78494862e2 Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Mon, 24 Feb 2020 17:28:36 -0500 Subject: [PATCH 160/860] Add insert_overwrite as incremental strategy on BQ --- .../macros/materializations/incremental.sql | 30 +++++++++++++++++-- 1 file changed, 28 insertions(+), 2 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 66e1c48c6..b550c0ca1 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -1,5 +1,21 @@ -{% macro bq_partition_merge(tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns) %} +{% macro dbt_bigquery_validate_get_incremental_strategy(config) %} + {#-- Find and validate the incremental strategy #} + {%- set strategy = config.get("incremental_strategy", default="merge") -%} + + {% set invalid_strategy_msg -%} + Invalid incremental strategy provided: {{ strategy }} + Expected one of: 'merge', 'insert_overwrite' + {%- endset %} + {% if strategy not in ['merge', 'insert_overwrite'] %} + {% do exceptions.raise_compiler_error(invalid_strategy_msg) %} + {% endif %} + + {% do return(strategy) %} +{% endmacro %} + + +{% macro bq_partition_merge(strategy, tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns) %} {%- set partition_type = 'date' if partition_by.data_type in ('timestamp, datetime') else partition_by.data_type -%} @@ -31,9 +47,15 @@ array_agg(distinct {{ partition_by.render() }}) from {{ tmp_relation }} ); - + -- 3. run the merge statement + {% if strategy == 'merge' %} {{ get_merge_sql(target_relation, source_sql, unique_key, dest_columns, [predicate]) }}; + {% elif strategy == 'insert_overwrite' %} + {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate]) }}; + {% else %} + {% do exceptions.raise_compiler_error('invalid strategy: ' ~ strategy) %} + {% endif %} -- 4. clean up the temp table drop table if exists {{ tmp_relation }} @@ -49,6 +71,9 @@ {%- set target_relation = this %} {%- set existing_relation = load_relation(this) %} {%- set tmp_relation = make_temp_relation(this) %} + + {#-- Validate early so we don't run SQL if the strategy is invalid --#} + {% set strategy = dbt_bigquery_validate_get_incremental_strategy(config) -%} {%- set raw_partition_by = config.get('partition_by', none) -%} {%- set partition_by = adapter.parse_partition_by(raw_partition_by) -%} @@ -75,6 +100,7 @@ {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#} {% if partition_by is not none %} {% set build_sql = bq_partition_merge( + strategy, tmp_relation, target_relation, sql, From 0f14823aec3a19b5527cd85acc03f4ba0b4e00db Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 4 Mar 2020 15:27:00 -0700 Subject: [PATCH 161/860] =?UTF-8?q?Bump=20version:=200.16.0b3=20=E2=86=92?= =?UTF-8?q?=200.16.0rc1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index d5c7fbe7e..2ffa7d6f6 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.16.0b3" +package_version = "0.16.0rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From df44bb697d1b24dea3c5b31862beb1f575a0002c Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 4 Mar 2020 16:39:43 -0700 Subject: [PATCH 162/860] =?UTF-8?q?Bump=20version:=200.16.0rc1=20=E2=86=92?= =?UTF-8?q?=200.16.0rc2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 2ffa7d6f6..809c1547c 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.16.0rc1" +package_version = "0.16.0rc2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From af0e75ba3dab9c2e08c15b049920f8250f8eef96 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 4 Mar 2020 14:20:05 -0700 Subject: [PATCH 163/860] =?UTF-8?q?Bump=20version:=200.16.0b3=20=E2=86=92?= =?UTF-8?q?=200.17.0a1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 809c1547c..b1080e75f 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.16.0rc2" +package_version = "0.17.0a1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 369b22662acc0f8f1bdc2017a01c6fdcdc3de775 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 11 Mar 2020 09:20:06 -0600 Subject: [PATCH 164/860] =?UTF-8?q?Bump=20version:=200.16.0rc2=20=E2=86=92?= =?UTF-8?q?=200.16.0rc3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 809c1547c..a91b36394 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.16.0rc2" +package_version = "0.16.0rc3" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From d9a779264a5e27f324766a9380baa5c088ea4665 Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Wed, 11 Mar 2020 22:21:11 -0400 Subject: [PATCH 165/860] Rework insert_overwrite incremental strategy --- .../macros/materializations/incremental.sql | 107 +++++++++++------- 1 file changed, 66 insertions(+), 41 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index b550c0ca1..28e57bb5a 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -15,51 +15,67 @@ {% endmacro %} -{% macro bq_partition_merge(strategy, tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns) %} +{% macro bq_insert_overwrite(tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns) %} {%- set partition_type = 'date' if partition_by.data_type in ('timestamp, datetime') else partition_by.data_type -%} - {% set predicate -%} - {{ partition_by.render(alias='DBT_INTERNAL_DEST') }} in unnest(dbt_partitions_for_upsert) - {%- endset %} - - {%- set source_sql -%} - ( - select * from {{ tmp_relation }} - ) - {%- endset -%} - - -- generated script to merge partitions into {{ target_relation }} - declare dbt_partitions_for_upsert array<{{ partition_type }}>; - declare _dbt_max_partition {{ partition_by.data_type }}; - - set _dbt_max_partition = ( - select max({{ partition_by.field }}) from {{ this }} - ); - - -- 1. create a temp table - {{ create_table_as(True, tmp_relation, sql) }} - - -- 2. define partitions to update - set (dbt_partitions_for_upsert) = ( - select as struct - array_agg(distinct {{ partition_by.render() }}) - from {{ tmp_relation }} - ); + {% if partitions is not none and partitions != [] %} {# static #} + + {% set predicate -%} + {{ partition_by.render(alias='DBT_INTERNAL_DEST') }} in ( + {% for partition in partitions %} + '{{partition}}' {{- ',' if not loop.last -}} + {% endfor %} + ) + {%- endset %} + + {%- set source_sql -%} + ( + {{sql}} + ) + {%- endset -%} + + {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate]) }} + + {% else %} {# dynamic #} + + {% set predicate -%} + {{ partition_by.render(alias='DBT_INTERNAL_DEST') }} in unnest(dbt_partitions_for_upsert) + {%- endset %} + + {%- set source_sql -%} + ( + select * from {{ tmp_relation }} + ) + {%- endset -%} + + -- generated script to merge partitions into {{ target_relation }} + declare dbt_partitions_for_upsert array<{{ partition_type }}>; + declare _dbt_max_partition {{ partition_by.data_type }}; + + set _dbt_max_partition = ( + select max({{ partition_by.field }}) from {{ this }} + ); + + -- 1. create a temp table + {{ create_table_as(True, tmp_relation, sql) }} + + -- 2. define partitions to update + set (dbt_partitions_for_upsert) = ( + select as struct + array_agg(distinct {{ partition_by.render() }}) + from {{ tmp_relation }} + ); + + -- 3. run the merge statement + {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate]) }}; + + -- 4. clean up the temp table + drop table if exists {{ tmp_relation }} - -- 3. run the merge statement - {% if strategy == 'merge' %} - {{ get_merge_sql(target_relation, source_sql, unique_key, dest_columns, [predicate]) }}; - {% elif strategy == 'insert_overwrite' %} - {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate]) }}; - {% else %} - {% do exceptions.raise_compiler_error('invalid strategy: ' ~ strategy) %} {% endif %} - -- 4. clean up the temp table - drop table if exists {{ tmp_relation }} - {% endmacro %} @@ -77,6 +93,7 @@ {%- set raw_partition_by = config.get('partition_by', none) -%} {%- set partition_by = adapter.parse_partition_by(raw_partition_by) -%} + {%- set partitions = config.get('partitions', none) -%} {%- set cluster_by = config.get('cluster_by', none) -%} {{ run_hooks(pre_hooks) }} @@ -98,14 +115,22 @@ {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %} {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#} - {% if partition_by is not none %} - {% set build_sql = bq_partition_merge( - strategy, + {% if strategy == 'insert_overwrite' %} + + {% set missing_partition_msg -%} + The 'insert_overwrite' strategy requires the `partition_by` config. + {%- endset %} + {% if partition_by is none %} + {% do exceptions.raise_compiler_error(missing_partition_msg) %} + {% endif %} + + {% set build_sql = bq_insert_overwrite( tmp_relation, target_relation, sql, unique_key, partition_by, + partitions, dest_columns) %} {% else %} From 5732bb2e9e5cffbe7d192cb4e134d8c4d9af0fef Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Thu, 12 Mar 2020 10:51:09 -0400 Subject: [PATCH 166/860] Rename dbt_partitions_for_upsert to dbt_partitions_for_replacement --- .../bigquery/macros/materializations/incremental.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 28e57bb5a..a7352086f 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -41,7 +41,7 @@ {% else %} {# dynamic #} {% set predicate -%} - {{ partition_by.render(alias='DBT_INTERNAL_DEST') }} in unnest(dbt_partitions_for_upsert) + {{ partition_by.render(alias='DBT_INTERNAL_DEST') }} in unnest(dbt_partitions_for_replacement) {%- endset %} {%- set source_sql -%} @@ -51,7 +51,7 @@ {%- endset -%} -- generated script to merge partitions into {{ target_relation }} - declare dbt_partitions_for_upsert array<{{ partition_type }}>; + declare dbt_partitions_for_replacement array<{{ partition_type }}>; declare _dbt_max_partition {{ partition_by.data_type }}; set _dbt_max_partition = ( @@ -62,7 +62,7 @@ {{ create_table_as(True, tmp_relation, sql) }} -- 2. define partitions to update - set (dbt_partitions_for_upsert) = ( + set (dbt_partitions_for_replacement) = ( select as struct array_agg(distinct {{ partition_by.render() }}) from {{ tmp_relation }} From 1055d052b3b785c563eacde290af6a6f4d64936b Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Mon, 16 Mar 2020 13:01:27 -0400 Subject: [PATCH 167/860] Unquote partition values in predicate --- dbt/include/bigquery/macros/materializations/incremental.sql | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index a7352086f..18a0c0bc3 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -24,9 +24,7 @@ {% set predicate -%} {{ partition_by.render(alias='DBT_INTERNAL_DEST') }} in ( - {% for partition in partitions %} - '{{partition}}' {{- ',' if not loop.last -}} - {% endfor %} + {{ partitions | join (', ') }} ) {%- endset %} From ab454dffc65685e015accb0cc6cc92ea2846c00b Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 18 Mar 2020 15:19:33 -0600 Subject: [PATCH 168/860] increase the lower bound to ensure we support "range_partitioning" --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index a91b36394..70978b86e 100644 --- a/setup.py +++ b/setup.py @@ -41,7 +41,7 @@ install_requires=[ 'dbt-core=={}'.format(package_version), 'google-cloud-core>=1,<=1.3.0', - 'google-cloud-bigquery>=1.15.0,<1.25.0', + 'google-cloud-bigquery>=1.22.0,<1.25.0', # hidden secret dependency: bq 1.23.0 requires this but only documents # 1.10.0 through its dependency chain. # see https://github.com/googleapis/google-cloud-python/issues/9965 From 727c5e2d809b6e676904527b422fdb9d44128520 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 20 Mar 2020 11:31:17 -0600 Subject: [PATCH 169/860] =?UTF-8?q?Bump=20version:=200.16.0rc3=20=E2=86=92?= =?UTF-8?q?=200.16.0rc4?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 70978b86e..2eea5f584 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.16.0rc3" +package_version = "0.16.0rc4" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From afa2c5fa114643a730f1700cfae0032a5b648c10 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 23 Mar 2020 07:59:37 -0600 Subject: [PATCH 170/860] =?UTF-8?q?Bump=20version:=200.16.0rc4=20=E2=86=92?= =?UTF-8?q?=200.16.0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 2eea5f584..1475c3f86 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.16.0rc4" +package_version = "0.16.0" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From a7431095e8eccd2ffb6885081feb4000a69bbf9f Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 6 Mar 2020 13:30:51 -0700 Subject: [PATCH 171/860] split test types into data/schema tests --- dbt/adapters/bigquery/impl.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 3ae6f40a5..906fbcf04 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -303,6 +303,12 @@ def drop_schema(self, database: str, schema: str) -> None: def quote(cls, identifier: str) -> str: return '`{}`'.format(identifier) + @classmethod + def unquote(cls, identifier: str) -> str: + if identifier.startswith('`') and identifier.endswith('`'): + return identifier[1:-1] + return identifier + @classmethod def convert_text_type(cls, agate_table: agate.Table, col_idx: int) -> str: return "string" From fd5010edd5020bd03840c52288a7a3c51e534723 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 25 Mar 2020 07:31:29 -0600 Subject: [PATCH 172/860] remove extra methods --- dbt/adapters/bigquery/impl.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 906fbcf04..3ae6f40a5 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -303,12 +303,6 @@ def drop_schema(self, database: str, schema: str) -> None: def quote(cls, identifier: str) -> str: return '`{}`'.format(identifier) - @classmethod - def unquote(cls, identifier: str) -> str: - if identifier.startswith('`') and identifier.endswith('`'): - return identifier[1:-1] - return identifier - @classmethod def convert_text_type(cls, agate_table: agate.Table, col_idx: int) -> str: return "string" From 109719f1d95cf62d7ee8056724d37883d72ffc3c Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 26 Mar 2020 10:56:56 -0600 Subject: [PATCH 173/860] =?UTF-8?q?Bump=20version:=200.16.0=20=E2=86=92=20?= =?UTF-8?q?0.16.1a1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 1475c3f86..9df0ae8f2 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.16.0" +package_version = "0.16.1a1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 24bdcdae63a6432de7725720de2ac8b672e5dd9f Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 26 Mar 2020 10:29:13 -0600 Subject: [PATCH 174/860] rev the bigquery minimum versions to a big number --- setup.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/setup.py b/setup.py index 9df0ae8f2..ef70fc1e3 100644 --- a/setup.py +++ b/setup.py @@ -40,12 +40,11 @@ }, install_requires=[ 'dbt-core=={}'.format(package_version), - 'google-cloud-core>=1,<=1.3.0', - 'google-cloud-bigquery>=1.22.0,<1.25.0', - # hidden secret dependency: bq 1.23.0 requires this but only documents - # 1.10.0 through its dependency chain. - # see https://github.com/googleapis/google-cloud-python/issues/9965 - 'six>=1.13.0', + 'google-cloud-core>=1.3.0,<1.4', + 'google-cloud-bigquery>=1.24.0,<1.25.0', + 'google-api-core>=1.16.0,<1.17.0', + 'googleapis-common-protos>=1.6.0,<1.7.0', + 'six>=1.14.0', ], zip_safe=False, classifiers=[ From 096cb6a297fce670a39c010b1fbe054f74cf8d36 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 31 Mar 2020 14:35:49 -0600 Subject: [PATCH 175/860] Add "partitions" to the adapter specific config And tests, of course --- dbt/adapters/bigquery/impl.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 3ae6f40a5..0ac599ffb 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -138,8 +138,9 @@ class BigQueryAdapter(BaseAdapter): Column = BigQueryColumn ConnectionManager = BigQueryConnectionManager - AdapterSpecificConfigs = frozenset({"cluster_by", "partition_by", - "kms_key_name", "labels"}) + AdapterSpecificConfigs = frozenset({ + "cluster_by", "partition_by", "kms_key_name", "labels", "partitions" + }) ### # Implementations of abstract methods From fa49868f3fb4f84731054bc48b99e35523b3eea3 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 3 Apr 2020 12:13:45 -0600 Subject: [PATCH 176/860] =?UTF-8?q?Bump=20version:=200.16.1a1=20=E2=86=92?= =?UTF-8?q?=200.16.1rc1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index ef70fc1e3..a4d50060d 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.16.1a1" +package_version = "0.16.1rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 369d4cb24cb118d58726f88ad0a8625b0d46242c Mon Sep 17 00:00:00 2001 From: Sumanau Sareen Date: Sun, 5 Apr 2020 00:32:02 +0530 Subject: [PATCH 177/860] Add DBT Plugin information to get_version_information --- dbt/adapters/bigquery/__version__.py | 1 + 1 file changed, 1 insertion(+) create mode 100644 dbt/adapters/bigquery/__version__.py diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py new file mode 100644 index 000000000..be4f93a26 --- /dev/null +++ b/dbt/adapters/bigquery/__version__.py @@ -0,0 +1 @@ +version = '0.17.0a1' From d78b60996ef38e57607af905df26c7cd9fc37598 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Mon, 6 Apr 2020 17:58:00 -0400 Subject: [PATCH 178/860] Fix for bq insert_overwrite incrementals, add additional tests --- .../bigquery/macros/materializations/incremental.sql | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 18a0c0bc3..6aabf8d75 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -34,7 +34,7 @@ ) {%- endset -%} - {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate]) }} + {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate], include_sql_header=true) }} {% else %} {# dynamic #} @@ -66,8 +66,12 @@ from {{ tmp_relation }} ); + {# + TODO: include_sql_header is a hack; consider a better approach that includes + the sql_header at the materialization-level instead + #} -- 3. run the merge statement - {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate]) }}; + {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate], include_sql_header=false) }}; -- 4. clean up the temp table drop table if exists {{ tmp_relation }} From 4fa3fb9a6fa62adfbcdf31dac10db552aad303a9 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 14 Apr 2020 14:32:35 -0600 Subject: [PATCH 179/860] Fix #2188 The BigQuery information schema previously used its quote policy as the basis for a new include policy, rather than its include policy. --- dbt/adapters/bigquery/relation.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py index 7cca7a1c2..08f2c8f06 100644 --- a/dbt/adapters/bigquery/relation.py +++ b/dbt/adapters/bigquery/relation.py @@ -65,7 +65,7 @@ def get_include_policy(cls, relation, information_schema_view): if information_schema_view == '__TABLES__': identifier = False - return relation.quote_policy.replace( + return relation.include_policy.replace( schema=schema, identifier=identifier, ) From 8f7febab16c5412cdec59265dfefe93dd301adcd Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 9 Apr 2020 09:29:38 -0600 Subject: [PATCH 180/860] Move configs into their own graph module Configs all get defaults, a base class Added SourceConfig Introduced the idea of MergeBehavior and ShowBehavior Implemented updates/merges based on MergeBehavior for configs Implemented typed adapter-specific configs and behavior --- dbt/adapters/bigquery/impl.py | 60 +++++------------------- dbt/include/bigquery/macros/adapters.sql | 2 +- 2 files changed, 13 insertions(+), 49 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 0ac599ffb..93fbea730 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -1,5 +1,5 @@ from dataclasses import dataclass -from typing import Dict, List, Optional, Any, Set +from typing import Dict, List, Optional, Any, Set, Union from hologram import JsonSchemaMixin, ValidationError import dbt.deprecations @@ -10,7 +10,7 @@ import dbt.links from dbt.adapters.base import ( - BaseAdapter, available, RelationType, SchemaSearchMap + BaseAdapter, available, RelationType, SchemaSearchMap, AdapterConfig ) from dbt.adapters.bigquery.relation import ( BigQueryRelation, BigQueryInformationSchema @@ -30,15 +30,6 @@ import time import agate -import re - - -BQ_INTEGER_RANGE_NOT_SUPPORTED = f""" -BigQuery integer range partitioning is only supported by the -`partition_by` config, which accepts a dictionary. - -See: {dbt.links.BigQueryNewPartitionBy} -""" @dataclass @@ -67,40 +58,6 @@ def _parse(cls, raw_partition_by) -> Optional['PartitionConfig']: dbt.exceptions.raise_compiler_error( f'Could not parse partition config: {msg}' ) - - elif isinstance(raw_partition_by, str): - raw_partition_by = raw_partition_by.strip() - if 'range_bucket' in raw_partition_by.lower(): - dbt.exceptions.raise_compiler_error( - BQ_INTEGER_RANGE_NOT_SUPPORTED - ) - - elif raw_partition_by.lower().startswith('date('): - matches = re.match(r'date\((.+)\)', raw_partition_by, - re.IGNORECASE) - if not matches: - dbt.exceptions.raise_compiler_error( - f"Specified partition_by '{raw_partition_by}' " - "is not parseable") - - partition_by = matches.group(1) - data_type = 'timestamp' - - else: - partition_by = raw_partition_by - data_type = 'date' - - inferred_partition_by = cls( - field=partition_by, - data_type=data_type - ) - - dbt.deprecations.warn( - 'bq-partition-by-string', - raw_partition_by=raw_partition_by, - inferred_partition_by=inferred_partition_by.to_dict() - ) - return inferred_partition_by else: return None @@ -126,6 +83,15 @@ def _stub_relation(*args, **kwargs): ) +class BigqueryConfig(AdapterConfig): + cluster_by: Optional[Union[List[str], str]] = None + partition_by: Optional[Dict[str, Any]] = None + kms_key_name: Optional[str] = None + labels: Optional[Dict[str, str]] = None + # TODO: should this accept `str` and `int`, too? + partitions: Optional[List[str]] = None + + class BigQueryAdapter(BaseAdapter): RELATION_TYPES = { @@ -138,9 +104,7 @@ class BigQueryAdapter(BaseAdapter): Column = BigQueryColumn ConnectionManager = BigQueryConnectionManager - AdapterSpecificConfigs = frozenset({ - "cluster_by", "partition_by", "kms_key_name", "labels", "partitions" - }) + AdapterSpecificConfigs = BigqueryConfig ### # Implementations of abstract methods diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index a7530d43b..a318257e7 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -61,7 +61,7 @@ {%- set raw_cluster_by = config.get('cluster_by', none) -%} {%- set raw_persist_docs = config.get('persist_docs', {}) -%} {%- set raw_kms_key_name = config.get('kms_key_name', none) -%} - {%- set raw_labels = config.get('labels', []) -%} + {%- set raw_labels = config.get('labels', {}) -%} {%- set sql_header = config.get('sql_header', none) -%} {%- set partition_config = adapter.parse_partition_by(raw_partition_by) -%} From c8719c37c267bda64b332f93bd3e4721de24f072 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 16 Apr 2020 13:05:12 -0600 Subject: [PATCH 181/860] Support for v1/v2 config blocks - Backwards compatiblity shims from v2 to v1 configs - config merging for v2 - compatibility shim for parsing/contexts - defer var lookups as late as possible - fixed ContextConfigType to be a proper Union of the two context config types - Fix adapter configs to be proper dataclasses - fix unused config paths error - make v2 config parsing not alter its source data --- dbt/adapters/bigquery/impl.py | 23 +++++++++-------------- 1 file changed, 9 insertions(+), 14 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 93fbea730..d3b862115 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -48,23 +48,17 @@ def render(self, alias: Optional[str] = None): else: return column - @classmethod - def _parse(cls, raw_partition_by) -> Optional['PartitionConfig']: - if isinstance(raw_partition_by, dict): - try: - return cls.from_dict(raw_partition_by) - except ValidationError as exc: - msg = dbt.exceptions.validator_error_message(exc) - dbt.exceptions.raise_compiler_error( - f'Could not parse partition config: {msg}' - ) - else: - return None - @classmethod def parse(cls, raw_partition_by) -> Optional['PartitionConfig']: + if raw_partition_by is None: + return None try: - return cls._parse(raw_partition_by) + return cls.from_dict(raw_partition_by) + except ValidationError as exc: + msg = dbt.exceptions.validator_error_message(exc) + dbt.exceptions.raise_compiler_error( + f'Could not parse partition config: {msg}' + ) except TypeError: dbt.exceptions.raise_compiler_error( f'Invalid partition_by config:\n' @@ -83,6 +77,7 @@ def _stub_relation(*args, **kwargs): ) +@dataclass class BigqueryConfig(AdapterConfig): cluster_by: Optional[Union[List[str], str]] = None partition_by: Optional[Dict[str, Any]] = None From d5947a2187128a7172586b1f13223b13dadde6e4 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 16 Apr 2020 13:23:27 -0600 Subject: [PATCH 182/860] deprecate v1 configs Bump included projects to v2 fix tests --- dbt/include/bigquery/dbt_project.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/include/bigquery/dbt_project.yml b/dbt/include/bigquery/dbt_project.yml index edae53869..b4e88b7b0 100644 --- a/dbt/include/bigquery/dbt_project.yml +++ b/dbt/include/bigquery/dbt_project.yml @@ -1,4 +1,4 @@ - +config-version: 2 name: dbt_bigquery version: 1.0 From 1afa38d2a2cafd0315e33adcf92397f0241c8975 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 21 Apr 2020 09:25:43 -0600 Subject: [PATCH 183/860] PR feedback Handle empty vars dict Check sources for unused configs - add a test Warn when dbt finds "vars" in "models", "seeds", etc blocks - add a test Clean up enabled/disabled code to share between sources and nodes - only log downstream tests at debug level when a source is disabled - include the offending project name in the v1 deprecation message - Fix tests that care about error messages, add new ones --- dbt/adapters/bigquery/impl.py | 1 - 1 file changed, 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index d3b862115..a11af60b7 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -83,7 +83,6 @@ class BigqueryConfig(AdapterConfig): partition_by: Optional[Dict[str, Any]] = None kms_key_name: Optional[str] = None labels: Optional[Dict[str, str]] = None - # TODO: should this accept `str` and `int`, too? partitions: Optional[List[str]] = None From eeb72eb43e565348905d98ceecb9db5e167aa1c4 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 27 Apr 2020 13:31:19 +0200 Subject: [PATCH 184/860] Add python_requires to setup.py This will let people know that they are running a too old version of Python Signed-off-by: Fokko Driesprong --- setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/setup.py b/setup.py index c5a9ad14c..41632b059 100644 --- a/setup.py +++ b/setup.py @@ -60,4 +60,5 @@ 'Programming Language :: Python :: 3.7', 'Programming Language :: Python :: 3.8', ], + python_requires=">=3.6.2", ) From 8bb93bea29e3c05fd876b38f04c24b5a82a17998 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 30 Apr 2020 12:19:11 -0600 Subject: [PATCH 185/860] remove location and information_schema.schematas query from catalog --- dbt/include/bigquery/macros/catalog.sql | 29 +++++-------------------- 1 file changed, 6 insertions(+), 23 deletions(-) diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql index d41b03604..ed64af881 100644 --- a/dbt/include/bigquery/macros/catalog.sql +++ b/dbt/include/bigquery/macros/catalog.sql @@ -7,22 +7,7 @@ {%- else -%} {%- set query -%} - with schemas as ( - - select - catalog_name as table_database, - schema_name as table_schema, - location - - from {{ information_schema.replace(information_schema_view='SCHEMATA') }} - where ( - {%- for schema in schemas -%} - upper(schema_name) = upper('{{ schema }}'){%- if not loop.last %} or {% endif -%} - {%- endfor -%} - ) - ), - - tables as ( + with tables as ( select project_id as table_database, dataset_id as table_schema, @@ -43,7 +28,11 @@ REGEXP_EXTRACT(table_id, '^.+([0-9]{8})$') as shard_name from {{ information_schema.replace(information_schema_view='__TABLES__') }} - + where ( + {%- for schema in schemas -%} + upper(dataset_id) = upper('{{ schema }}'){%- if not loop.last %} or {% endif -%} + {%- endfor -%} + ) ), extracted as ( @@ -171,11 +160,6 @@ coalesce(columns.column_type, '') as column_type, columns.column_comment, - 'Location' as `stats__location__label`, - location as `stats__location__value`, - 'The geographic location of this table' as `stats__location__description`, - location is not null as `stats__location__include`, - 'Shard count' as `stats__date_shards__label`, table_shards.shard_count as `stats__date_shards__value`, 'The number of date shards in this table' as `stats__date_shards__description`, @@ -215,7 +199,6 @@ -- sure that column metadata is picked up through the join. This will only -- return the column information for the "max" table in a date-sharded table set from unsharded_tables - left join schemas using(table_database, table_schema) left join columns using (relation_id) left join column_stats using (relation_id) {%- endset -%} From b6c658b067aa155f65fe42325c597a5867fc271d Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 29 Apr 2020 10:20:08 -0600 Subject: [PATCH 186/860] Column and table comments - add some table comment framework stuff - have redshift/postgres catalogs include table comments - have redshift/postgres add comments to columns/tables/views - push some bigquery-specific formatting into bigquery - add tests for table comments --- dbt/include/bigquery/macros/adapters.sql | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index a318257e7..78549c4d0 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -27,12 +27,21 @@ {%- endmacro -%} + +{%- macro bigquery_escape_comment(comment) -%} + {%- if comment is not string -%} + {%- do exceptions.raise_compiler_exception('cannot escape a non-string: ' ~ comment) -%} + {%- endif -%} + {%- do return((comment | tojson)[1:-1]) -%} +{%- endmacro -%} + + {% macro bigquery_table_options(persist_docs, temporary, kms_key_name, labels) %} {% set opts = {} -%} {%- set description = get_relation_comment(persist_docs, model) -%} {%- if description is not none -%} - {%- do opts.update({'description': "'" ~ description ~ "'"}) -%} + {%- do opts.update({'description': "'" ~ bigquery_escape_comment(description) ~ "'"}) -%} {%- endif -%} {%- if temporary -%} {% do opts.update({'expiration_timestamp': 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)'}) %} From 4ddb0c82d600cfc8da06deef3678a13052349724 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 4 May 2020 10:09:11 -0600 Subject: [PATCH 187/860] Suppress the bigquery job sql in exception messages --- dbt/adapters/bigquery/connections.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 98b0d8397..ac1606864 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -20,6 +20,9 @@ from hologram.helpers import StrEnum +BQ_QUERY_JOB_SPLIT = '-----Query Job SQL Follows-----' + + class Priority(StrEnum): Interactive = 'interactive' Batch = 'batch' @@ -95,7 +98,12 @@ def exception_handler(self, sql): # this sounds a lot like a signal handler and probably has # useful information, so raise it without modification. raise - raise RuntimeException(str(e)) + exc_message = str(e) + # the google bigquery library likes to add the query log, which we + # don't want to log. Hopefully they never change this! + if BQ_QUERY_JOB_SPLIT in exc_message: + exc_message = exc_message.split(BQ_QUERY_JOB_SPLIT)[0].strip() + raise RuntimeException(exc_message) def cancel_open(self) -> None: pass From 0ffaa0a66879b9d95a2909646dbd67c8092219ec Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 4 May 2020 14:59:39 -0600 Subject: [PATCH 188/860] =?UTF-8?q?Bump=20version:=200.17.0a1=20=E2=86=92?= =?UTF-8?q?=200.17.0b1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 41632b059..a8edd9cb5 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.0a1" +package_version = "0.17.0b1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 040698ea086dba38d42a7271d413a525338a701b Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 6 May 2020 09:41:10 -0600 Subject: [PATCH 189/860] update __version__.py files, add them to .bumpversion.cfg --- dbt/adapters/bigquery/__version__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index be4f93a26..dbd88ad93 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.0a1' +version = '0.17.0b1' From c75c5dd2475a26533848baff9ceb6d73096b84ca Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Wed, 6 May 2020 17:10:04 -0400 Subject: [PATCH 190/860] Fix error when trying to call list_schemas macro for BigQuery adapter --- dbt/include/bigquery/macros/adapters.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 78549c4d0..e96c2c10c 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -142,7 +142,7 @@ {% macro bigquery__list_schemas(database) -%} - {{ return(adapter.list_schemas()) }} + {{ return(adapter.list_schemas(database)) }} {% endmacro %} From 767c1c4568243b781d73818e99b1a28b7e570178 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Tue, 5 May 2020 17:37:55 -0400 Subject: [PATCH 191/860] Add column comments on BigQuery for views, tables, and incrementals --- dbt/adapters/bigquery/impl.py | 67 +++++++++++++++++-- dbt/include/bigquery/macros/adapters.sql | 55 +++++---------- .../macros/materializations/incremental.sql | 4 ++ .../macros/materializations/table.sql | 2 + .../bigquery/macros/materializations/view.sql | 8 ++- 5 files changed, 91 insertions(+), 45 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index a11af60b7..eb8083390 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -28,8 +28,20 @@ import google.cloud.exceptions import google.cloud.bigquery +from google.cloud.bigquery import SchemaField + import time import agate +import json + + +def sql_escape(string): + if not isinstance(string, str): + dbt.exceptions.raise_compiler_exception( + f'cannot escape a non-string: {string}' + ) + + return json.dumps(string)[1:-1] @dataclass @@ -320,16 +332,14 @@ def _get_dbt_columns_from_bq_table(self, table) -> List[BigQueryColumn]: def _agate_to_schema( self, agate_table: agate.Table, column_override: Dict[str, str] - ) -> List[google.cloud.bigquery.SchemaField]: + ) -> List[SchemaField]: """Convert agate.Table with column names to a list of bigquery schemas. """ bq_schema = [] for idx, col_name in enumerate(agate_table.column_names): inferred_type = self.convert_agate_type(agate_table, idx) type_ = column_override.get(col_name, inferred_type) - bq_schema.append( - google.cloud.bigquery.SchemaField(col_name, type_) - ) + bq_schema.append(SchemaField(col_name, type_)) return bq_schema def _materialize_as_view(self, model: Dict[str, Any]) -> str: @@ -545,6 +555,33 @@ def parse_partition_by( """ return PartitionConfig.parse(raw_partition_by) + def get_table_ref_from_relation(self, conn, relation): + return self.connections.table_ref(relation.database, + relation.schema, + relation.identifier, + conn) + + @available.parse_none + def update_column_descriptions(self, relation, columns): + if len(columns) == 0: + return + + conn = self.connections.get_thread_connection() + table_ref = self.get_table_ref_from_relation(conn, relation) + table = conn.handle.get_table(table_ref) + + new_schema = [] + for column in table.schema: + if column.name in columns: + column_config = columns[column.name] + column_dict = column.to_api_repr() + column_dict['description'] = column_config.get('description') + column = SchemaField.from_api_repr(column_dict) + new_schema.append(column) + + new_table = google.cloud.bigquery.Table(table_ref, schema=new_schema) + conn.handle.update_table(new_table, ['schema']) + @available.parse_none def alter_table_add_columns(self, relation, columns): @@ -614,3 +651,25 @@ def _get_cache_schemas( .format(database, candidate.schema) ) return result + + @available.parse(lambda *a, **k: {}) + def get_table_options( + self, config: Dict[str, Any], node: Dict[str, Any], temporary: bool + ) -> Dict[str, Any]: + opts = {} + if temporary: + expiration = 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)' + opts['expiration_timestamp'] = expiration + + if config.persist_relation_docs() and 'description' in node: + description = sql_escape(node['description']) + opts['description'] = '"""{}"""'.format(description) + + if config.get('kms_key_name') is not None: + opts['kms_key_name'] = "'{}'".format(config.get('kms_key_name')) + + if config.get('labels'): + labels = config.get('labels', {}) + opts['labels'] = list(labels.items()) + + return opts diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 78549c4d0..1502ece80 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -28,34 +28,8 @@ {%- endmacro -%} -{%- macro bigquery_escape_comment(comment) -%} - {%- if comment is not string -%} - {%- do exceptions.raise_compiler_exception('cannot escape a non-string: ' ~ comment) -%} - {%- endif -%} - {%- do return((comment | tojson)[1:-1]) -%} -{%- endmacro -%} - - -{% macro bigquery_table_options(persist_docs, temporary, kms_key_name, labels) %} - {% set opts = {} -%} - - {%- set description = get_relation_comment(persist_docs, model) -%} - {%- if description is not none -%} - {%- do opts.update({'description': "'" ~ bigquery_escape_comment(description) ~ "'"}) -%} - {%- endif -%} - {%- if temporary -%} - {% do opts.update({'expiration_timestamp': 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)'}) %} - {%- endif -%} - {%- if kms_key_name -%} - {%- do opts.update({'kms_key_name': "'" ~ kms_key_name ~ "'"}) -%} - {%- endif -%} - {%- if labels -%} - {%- set label_list = [] -%} - {%- for label, value in labels.items() -%} - {%- do label_list.append((label, value)) -%} - {%- endfor -%} - {%- do opts.update({'labels': label_list}) -%} - {%- endif -%} +{% macro bigquery_table_options(config, node, temporary) %} + {% set opts = adapter.get_table_options(config, node, temporary) %} {% set options -%} OPTIONS({% for opt_key, opt_val in opts.items() %} @@ -68,9 +42,6 @@ {% macro bigquery__create_table_as(temporary, relation, sql) -%} {%- set raw_partition_by = config.get('partition_by', none) -%} {%- set raw_cluster_by = config.get('cluster_by', none) -%} - {%- set raw_persist_docs = config.get('persist_docs', {}) -%} - {%- set raw_kms_key_name = config.get('kms_key_name', none) -%} - {%- set raw_labels = config.get('labels', {}) -%} {%- set sql_header = config.get('sql_header', none) -%} {%- set partition_config = adapter.parse_partition_by(raw_partition_by) -%} @@ -80,12 +51,7 @@ create or replace table {{ relation }} {{ partition_by(partition_config) }} {{ cluster_by(raw_cluster_by) }} - {{ bigquery_table_options( - persist_docs=raw_persist_docs, - temporary=temporary, - kms_key_name=raw_kms_key_name, - labels=raw_labels - ) }} + {{ bigquery_table_options(config, model, temporary) }} as ( {{ sql }} ); @@ -93,14 +59,12 @@ {%- endmacro -%} {% macro bigquery__create_view_as(relation, sql) -%} - {%- set raw_persist_docs = config.get('persist_docs', {}) -%} - {%- set raw_labels = config.get('labels', []) -%} {%- set sql_header = config.get('sql_header', none) -%} {{ sql_header if sql_header is not none }} create or replace view {{ relation }} - {{ bigquery_table_options(persist_docs=raw_persist_docs, temporary=false, labels=raw_labels) }} + {{ bigquery_table_options(config, model, temporary=false) }} as ( {{ sql }} ); @@ -149,3 +113,14 @@ {% macro bigquery__check_schema_exists(information_schema, schema) %} {{ return(adapter.check_schema_exists(information_schema.database, schema)) }} {% endmacro %} + +{#-- relation-level macro is not implemented. This is handled in the CTAs statement #} +{% macro bigquery__persist_docs(relation, model, for_relation, for_columns) -%} + {% if for_columns and config.persist_column_docs() %} + {% do alter_column_comment(relation, model.columns) %} + {% endif %} +{% endmacro %} + +{% macro bigquery__alter_column_comment(relation, column_dict) -%} + {% do adapter.update_column_descriptions(relation, column_dict) %} +{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 6aabf8d75..28ceedebc 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -155,6 +155,10 @@ {{ run_hooks(post_hooks) }} + {% set target_relation = this.incorporate(type='table') %} + + {% do persist_docs(target_relation, model) %} + {{ return({'relations': [target_relation]}) }} {%- endmaterialization %} diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql index ea9b7e4d3..cbb97ea4e 100644 --- a/dbt/include/bigquery/macros/materializations/table.sql +++ b/dbt/include/bigquery/macros/materializations/table.sql @@ -77,6 +77,8 @@ {{ run_hooks(post_hooks) }} + {% do persist_docs(target_relation, model) %} + {{ return({'relations': [target_relation]}) }} {% endmaterialization %} diff --git a/dbt/include/bigquery/macros/materializations/view.sql b/dbt/include/bigquery/macros/materializations/view.sql index 04cac1c1c..3e43abfcf 100644 --- a/dbt/include/bigquery/macros/materializations/view.sql +++ b/dbt/include/bigquery/macros/materializations/view.sql @@ -9,5 +9,11 @@ {% materialization view, adapter='bigquery' -%} - {{ return(create_or_replace_view(run_outside_transaction_hooks=False)) }} + {% set to_return = create_or_replace_view(run_outside_transaction_hooks=False) %} + + {% set target_relation = this.incorporate(type='view') %} + {% do persist_docs(target_relation, model) %} + + {% do return(to_return) %} + {%- endmaterialization %} From 2af936acb0ce17ba73f0eb131d22d27e4ab41bf0 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 6 May 2020 13:20:07 -0600 Subject: [PATCH 192/860] Change list_relations_without_caching macro to take a single argument The argument is a Relation object with no identifier field, configured with the appropriate quoting information Unique quoted/unquoted representations will be treated as distinct The logic for generating what schemas to search for relations is now distinct from the catalog search logic. Schema creation/dropping takes a similar relation argument Add tests --- dbt/adapters/bigquery/impl.py | 22 +++++++++++----------- dbt/include/bigquery/macros/adapters.sql | 8 ++++---- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index eb8083390..c1ea5aab7 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -12,9 +12,7 @@ from dbt.adapters.base import ( BaseAdapter, available, RelationType, SchemaSearchMap, AdapterConfig ) -from dbt.adapters.bigquery.relation import ( - BigQueryRelation, BigQueryInformationSchema -) +from dbt.adapters.bigquery.relation import BigQueryRelation from dbt.adapters.bigquery import BigQueryColumn from dbt.adapters.bigquery import BigQueryConnectionManager from dbt.contracts.connection import Connection @@ -215,13 +213,13 @@ def expand_target_column_types( pass def list_relations_without_caching( - self, information_schema: BigQueryInformationSchema, schema: str + self, schema_relation: BigQueryRelation ) -> List[BigQueryRelation]: connection = self.connections.get_thread_connection() client = connection.handle bigquery_dataset = self.connections.dataset( - information_schema.database, information_schema.schema, connection + schema_relation.database, schema_relation.schema, connection ) all_tables = client.list_tables( @@ -261,11 +259,15 @@ def get_relation( table = None return self._bq_table_to_relation(table) - def create_schema(self, database: str, schema: str) -> None: + def create_schema(self, relation: BigQueryRelation) -> None: + database = relation.database + schema = relation.schema logger.debug('Creating schema "{}.{}".', database, schema) self.connections.create_dataset(database, schema) - def drop_schema(self, database: str, schema: str) -> None: + def drop_schema(self, relation: BigQueryRelation) -> None: + database = relation.database + schema = relation.schema logger.debug('Dropping schema "{}.{}".', database, schema) self.connections.drop_dataset(database, schema) self.cache.drop_schema(database, schema) @@ -632,10 +634,8 @@ def _catalog_filter_table( }) return super()._catalog_filter_table(table, manifest) - def _get_cache_schemas( - self, manifest: Manifest, exec_only: bool = False - ) -> SchemaSearchMap: - candidates = super()._get_cache_schemas(manifest, exec_only) + def _get_catalog_schemas(self, manifest: Manifest) -> SchemaSearchMap: + candidates = super()._get_catalog_schemas(manifest) db_schemas: Dict[str, Set[str]] = {} result = SchemaSearchMap() diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index dd1d9c469..c52172f81 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -74,8 +74,8 @@ {{ adapter.create_schema(database_name, schema_name) }} {% endmacro %} -{% macro bigquery__drop_schema(database_name, schema_name) -%} - {{ adapter.drop_schema(database_name, schema_name) }} +{% macro bigquery__drop_schema(relation) -%} + {{ adapter.drop_schema(relation) }} {% endmacro %} {% macro bigquery__drop_relation(relation) -%} @@ -89,8 +89,8 @@ {% endmacro %} -{% macro bigquery__list_relations_without_caching(information_schema, schema) -%} - {{ return(adapter.list_relations_without_caching(information_schema, schema)) }} +{% macro bigquery__list_relations_without_caching(schema_relation) -%} + {{ return(adapter.list_relations_without_caching(schema_relation)) }} {%- endmacro %} From 7207ca7c05948ceda3548b9cfc3d520988fd1a8f Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Fri, 8 May 2020 15:36:29 -0400 Subject: [PATCH 193/860] Fix for extra spacing and parentheses for BigQuery views --- dbt/include/bigquery/macros/adapters.sql | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index c52172f81..d4e635822 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -65,9 +65,8 @@ create or replace view {{ relation }} {{ bigquery_table_options(config, model, temporary=false) }} - as ( - {{ sql }} - ); + as {{ sql }} + {% endmacro %} {% macro bigquery__create_schema(database_name, schema_name) -%} From 9c8805089f19e214c544ff3ef28207ce427cb76f Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Fri, 8 May 2020 15:38:45 -0400 Subject: [PATCH 194/860] Re-add dropped semi-colon --- dbt/include/bigquery/macros/adapters.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index d4e635822..ef4b39a01 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -65,7 +65,7 @@ create or replace view {{ relation }} {{ bigquery_table_options(config, model, temporary=false) }} - as {{ sql }} + as {{ sql }}; {% endmacro %} From 2f31950ed5e46150399ee2b7a6756fceb5dac50a Mon Sep 17 00:00:00 2001 From: Hauke Duden Date: Mon, 11 May 2020 20:06:25 +0200 Subject: [PATCH 195/860] - added maximum_bytes_billed option to bigquery profiles - added basic unit test for maximum_bytes_billed --- dbt/adapters/bigquery/connections.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index ac1606864..1d6656cd3 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -43,6 +43,7 @@ class BigQueryCredentials(Credentials): location: Optional[str] = None priority: Optional[Priority] = None retries: Optional[int] = 1 + maximum_bytes_billed: Optional[int] = None _ALIASES = { 'project': 'database', 'dataset': 'schema', @@ -54,7 +55,7 @@ def type(self): def _connection_keys(self): return ('method', 'database', 'schema', 'location', 'priority', - 'timeout_seconds') + 'timeout_seconds', 'maximum_bytes_billed') class BigQueryConnectionManager(BaseConnectionManager): @@ -216,6 +217,10 @@ def raw_execute(self, sql, fetch=False): job_params[ 'priority'] = google.cloud.bigquery.QueryPriority.INTERACTIVE + maximum_bytes_billed = conn.credentials.maximum_bytes_billed + if maximum_bytes_billed is not None and maximum_bytes_billed!=0: + job_params['maximum_bytes_billed'] = maximum_bytes_billed + def fn(): return self._query_and_results(client, sql, conn, job_params) From 31e0dd5b3b1751609ac20e9259d2d386b28722e8 Mon Sep 17 00:00:00 2001 From: Hauke Duden Date: Tue, 12 May 2020 15:37:19 +0200 Subject: [PATCH 196/860] added whitespace around operator to pass flake8 tests --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 1d6656cd3..ec07565ed 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -218,7 +218,7 @@ def raw_execute(self, sql, fetch=False): 'priority'] = google.cloud.bigquery.QueryPriority.INTERACTIVE maximum_bytes_billed = conn.credentials.maximum_bytes_billed - if maximum_bytes_billed is not None and maximum_bytes_billed!=0: + if maximum_bytes_billed is not None and maximum_bytes_billed != 0: job_params['maximum_bytes_billed'] = maximum_bytes_billed def fn(): From e203969fbebcc2491c984df2c61b7137d8b507da Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 12 May 2020 10:45:59 -0600 Subject: [PATCH 197/860] =?UTF-8?q?Bump=20version:=200.17.0b1=20=E2=86=92?= =?UTF-8?q?=200.17.0rc1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index dbd88ad93..236254191 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.0b1' +version = '0.17.0rc1' diff --git a/setup.py b/setup.py index a8edd9cb5..e690a118c 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.0b1" +package_version = "0.17.0rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From d1f40c23180e81ca43fe4f90a012dba0295110c4 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 12 May 2020 11:04:15 -0600 Subject: [PATCH 198/860] =?UTF-8?q?Bump=20version:=200.17.0rc1=20=E2=86=92?= =?UTF-8?q?=200.18.0a1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 236254191..13a1c8c16 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.0rc1' +version = '0.18.0a1' diff --git a/setup.py b/setup.py index e690a118c..b9d9e75d4 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.0rc1" +package_version = "0.18.0a1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 04105a2617c7bc4fda764cf1b982f55170b1fbd9 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 12 May 2020 07:09:00 -0600 Subject: [PATCH 199/860] make full refresh a config item Added integration tests, fixed existing tests --- .../macros/materializations/incremental.sql | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 28ceedebc..fb6cccb25 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -17,11 +17,11 @@ {% macro bq_insert_overwrite(tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns) %} {%- set partition_type = - 'date' if partition_by.data_type in ('timestamp, datetime') + 'date' if partition_by.data_type in ('timestamp, datetime') else partition_by.data_type -%} {% if partitions is not none and partitions != [] %} {# static #} - + {% set predicate -%} {{ partition_by.render(alias='DBT_INTERNAL_DEST') }} in ( {{ partitions | join (', ') }} @@ -33,11 +33,11 @@ {{sql}} ) {%- endset -%} - + {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate], include_sql_header=true) }} - + {% else %} {# dynamic #} - + {% set predicate -%} {{ partition_by.render(alias='DBT_INTERNAL_DEST') }} in unnest(dbt_partitions_for_replacement) {%- endset %} @@ -65,7 +65,7 @@ array_agg(distinct {{ partition_by.render() }}) from {{ tmp_relation }} ); - + {# TODO: include_sql_header is a hack; consider a better approach that includes the sql_header at the materialization-level instead @@ -75,7 +75,7 @@ -- 4. clean up the temp table drop table if exists {{ tmp_relation }} - + {% endif %} {% endmacro %} @@ -84,12 +84,12 @@ {% materialization incremental, adapter='bigquery' -%} {%- set unique_key = config.get('unique_key') -%} - {%- set full_refresh_mode = (flags.FULL_REFRESH == True) -%} + {%- set full_refresh_mode = (should_full_refresh()) -%} {%- set target_relation = this %} {%- set existing_relation = load_relation(this) %} {%- set tmp_relation = make_temp_relation(this) %} - + {#-- Validate early so we don't run SQL if the strategy is invalid --#} {% set strategy = dbt_bigquery_validate_get_incremental_strategy(config) -%} @@ -118,14 +118,14 @@ {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#} {% if strategy == 'insert_overwrite' %} - + {% set missing_partition_msg -%} The 'insert_overwrite' strategy requires the `partition_by` config. {%- endset %} {% if partition_by is none %} {% do exceptions.raise_compiler_error(missing_partition_msg) %} {% endif %} - + {% set build_sql = bq_insert_overwrite( tmp_relation, target_relation, From 95fdc9dab8427c2b444d9e7172830c387f3cc60c Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 12 May 2020 14:24:46 -0600 Subject: [PATCH 200/860] Only run alter_column_comment if there are columns defined Also remove unused (and confusing/misleading) test data --- dbt/include/bigquery/macros/adapters.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index ef4b39a01..5bf24ef2d 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -115,7 +115,7 @@ {#-- relation-level macro is not implemented. This is handled in the CTAs statement #} {% macro bigquery__persist_docs(relation, model, for_relation, for_columns) -%} - {% if for_columns and config.persist_column_docs() %} + {% if for_columns and config.persist_column_docs() and model.columns %} {% do alter_column_comment(relation, model.columns) %} {% endif %} {% endmacro %} From ef3a88e79e25c7eb7a7392bc01841360ec715b46 Mon Sep 17 00:00:00 2001 From: Claire Carroll Date: Wed, 13 May 2020 13:56:27 -0400 Subject: [PATCH 201/860] Replace slack references --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index d586baeea..ec3527497 100644 --- a/README.md +++ b/README.md @@ -19,12 +19,12 @@ more information on using dbt with BigQuery, consult [the docs](https://docs.get ## Join thousands of analysts in the dbt community -- Join the [chat](http://slack.getdbt.com/) on Slack. +- Join the [chat](http://community.getdbt.com/) on Slack. - Find community posts on [dbt Discourse](https://discourse.getdbt.com). ## Reporting bugs and contributing code -- Want to report a bug or request a feature? Let us know on [Slack](http://slack.getdbt.com/), or open [an issue](https://github.com/fishtown-analytics/dbt/issues/new). +- Want to report a bug or request a feature? Let us know on [Slack](http://community.getdbt.com/), or open [an issue](https://github.com/fishtown-analytics/dbt/issues/new). - Want to help us build dbt? Check out the [Contributing Getting Started Guide](/CONTRIBUTING.md) ## Code of Conduct From 327d05d3e95da2abffdeaf1f8f04fa13b6274891 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 13 May 2020 10:16:53 -0600 Subject: [PATCH 202/860] Fix create_schema macro on bigquery --- dbt/adapters/bigquery/impl.py | 1 + dbt/include/bigquery/macros/adapters.sql | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index c1ea5aab7..caafcb0fb 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -212,6 +212,7 @@ def expand_target_column_types( # This is a no-op on BigQuery pass + @available.parse_list def list_relations_without_caching( self, schema_relation: BigQueryRelation ) -> List[BigQueryRelation]: diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 5bf24ef2d..d6365a276 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -69,8 +69,8 @@ {% endmacro %} -{% macro bigquery__create_schema(database_name, schema_name) -%} - {{ adapter.create_schema(database_name, schema_name) }} +{% macro bigquery__create_schema(relation) -%} + {{ adapter.create_schema(relation) }} {% endmacro %} {% macro bigquery__drop_schema(relation) -%} From 43ac5489651ed771b146a3821079ecac322c7a42 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 22 May 2020 14:18:45 -0600 Subject: [PATCH 203/860] =?UTF-8?q?Bump=20version:=200.17.0rc1=20=E2=86=92?= =?UTF-8?q?=200.17.0rc2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 236254191..113b6c6bc 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.0rc1' +version = '0.17.0rc2' diff --git a/setup.py b/setup.py index e690a118c..6b58f236e 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.0rc1" +package_version = "0.17.0rc2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From e86921e260656fb86d990623fdf7ff634cdb1ea5 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 26 May 2020 14:51:11 -0600 Subject: [PATCH 204/860] =?UTF-8?q?Bump=20version:=200.17.0rc2=20=E2=86=92?= =?UTF-8?q?=200.17.0rc3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 113b6c6bc..e5d1a220d 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.0rc2' +version = '0.17.0rc3' diff --git a/setup.py b/setup.py index 6b58f236e..3db892d58 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.0rc2" +package_version = "0.17.0rc3" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 4d14f5796fe291643a1d54804afbc26235dabbe0 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 27 May 2020 09:50:42 -0600 Subject: [PATCH 205/860] Pin protobuf --- setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/setup.py b/setup.py index 3db892d58..a4545b80c 100644 --- a/setup.py +++ b/setup.py @@ -40,6 +40,7 @@ }, install_requires=[ 'dbt-core=={}'.format(package_version), + 'protobuf>=3.6.0,<3.12', 'google-cloud-core>=1.3.0,<1.4', 'google-cloud-bigquery>=1.24.0,<1.25.0', 'google-api-core>=1.16.0,<1.17.0', From 26df71b777f50f918556bd58b9a7136af6bf0d58 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 2 Jun 2020 07:48:09 -0600 Subject: [PATCH 206/860] =?UTF-8?q?Bump=20version:=200.17.0rc3=20=E2=86=92?= =?UTF-8?q?=200.17.0rc4?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index e5d1a220d..294df9956 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.0rc3' +version = '0.17.0rc4' diff --git a/setup.py b/setup.py index a4545b80c..05623fa6b 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.0rc3" +package_version = "0.17.0rc4" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 4fadb473be62b7c32da443088ff2a93a29986640 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Mon, 8 Jun 2020 17:12:47 +0000 Subject: [PATCH 207/860] Release dbt v0.17.0 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 294df9956..063151983 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.0rc4' +version = '0.17.0' diff --git a/setup.py b/setup.py index 05623fa6b..e4d76a5d2 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.0rc4" +package_version = "0.17.0" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 32ae41f5ace2a9c85a74d7d1efe3179d6c3ef207 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Mon, 8 Jun 2020 21:27:13 +0000 Subject: [PATCH 208/860] Release dbt v0.18.0b1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 13a1c8c16..7985b917d 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.18.0a1' +version = '0.18.0b1' diff --git a/setup.py b/setup.py index b9d9e75d4..b88d6cf25 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.18.0a1" +package_version = "0.18.0b1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From f7758b9081a98b04906f9ff6fa2b0a53697b8ae2 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Mon, 8 Jun 2020 17:30:26 -0400 Subject: [PATCH 209/860] #1718 - Added support for BigQuery authorized views --- dbt/adapters/bigquery/connections.py | 4 +++ dbt/adapters/bigquery/impl.py | 32 ++++++++++++++++++- dbt/include/bigquery/macros/adapters.sql | 4 +++ .../bigquery/macros/materializations/view.sql | 6 ++++ 4 files changed, 45 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index ec07565ed..74e5d7d36 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -313,6 +313,10 @@ def dataset(database, schema, conn): dataset_ref = conn.handle.dataset(schema, database) return google.cloud.bigquery.Dataset(dataset_ref) + @staticmethod + def dataset_from_id(dataset_id): + return google.cloud.bigquery.Dataset.from_string(dataset_id) + def table_ref(self, database, schema, table_name, conn): dataset = self.dataset(database, schema, conn) return dataset.table(table_name) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index caafcb0fb..8c85bb54d 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -26,7 +26,7 @@ import google.cloud.exceptions import google.cloud.bigquery -from google.cloud.bigquery import SchemaField +from google.cloud.bigquery import AccessEntry, SchemaField import time import agate @@ -94,6 +94,7 @@ class BigqueryConfig(AdapterConfig): kms_key_name: Optional[str] = None labels: Optional[Dict[str, str]] = None partitions: Optional[List[str]] = None + grant_access_to: Optional[List[str]] = None class BigQueryAdapter(BaseAdapter): @@ -674,3 +675,32 @@ def get_table_options( opts['labels'] = list(labels.items()) return opts + + @available.parse_none + def grant_access_to(self, entity, entity_type, role, dataset_id): + """ + Given an entity, grants it access to a permissioned dataset. + """ + conn = self.connections.get_thread_connection() + client = conn.handle + + dataset = client.get_dataset( + self.connections.dataset_from_id(dataset_id)) + + if entity_type == 'view': + entity = self.connections.table_ref( + entity.database, + entity.schema, + entity.identifier, + conn).to_api_repr() + + access_entry = AccessEntry(role, entity_type, entity) + access_entries = dataset.access_entries + + if access_entry in access_entries: + logger.debug(f"Access entry {access_entry} already in dataset") + return + + access_entries.append(AccessEntry(role, entity_type, entity)) + dataset.access_entries = access_entries + client.update_dataset(dataset, ['access_entries']) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index d6365a276..98739aea0 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -123,3 +123,7 @@ {% macro bigquery__alter_column_comment(relation, column_dict) -%} {% do adapter.update_column_descriptions(relation, column_dict) %} {% endmacro %} + +{% macro bigquery__grant_access_to(entity, entity_type, role, dataset_id) -%} + {% do adapter.grant_access_to(entity, entity_type, role, dataset_id) %} +{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/view.sql b/dbt/include/bigquery/macros/materializations/view.sql index 3e43abfcf..b5de687e0 100644 --- a/dbt/include/bigquery/macros/materializations/view.sql +++ b/dbt/include/bigquery/macros/materializations/view.sql @@ -14,6 +14,12 @@ {% set target_relation = this.incorporate(type='view') %} {% do persist_docs(target_relation, model) %} + {% if config.get('grant_access_to') %} + {% for dataset_id in config.get('grant_access_to') %} + {% do adapter.grant_access_to(this, 'view', None, dataset_id) %} + {% endfor %} + {% endif %} + {% do return(to_return) %} {%- endmaterialization %} From 213cf31c0481f81009e508f822bc3d7c9cff1931 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Tue, 9 Jun 2020 11:17:17 -0400 Subject: [PATCH 210/860] Added support for BigQuery relation renaming --- dbt/adapters/bigquery/impl.py | 24 +++++++++++++++++++++--- dbt/include/bigquery/macros/adapters.sql | 4 ++++ 2 files changed, 25 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index caafcb0fb..111189234 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -143,9 +143,27 @@ def truncate_relation(self, relation: BigQueryRelation) -> None: def rename_relation( self, from_relation: BigQueryRelation, to_relation: BigQueryRelation ) -> None: - raise dbt.exceptions.NotImplementedException( - '`rename_relation` is not implemented for this adapter!' - ) + self.cache_renamed(from_relation, to_relation) + + conn = self.connections.get_thread_connection() + client = conn.handle + + from_table_ref = self.connections.table_ref(from_relation.database, + from_relation.schema, + from_relation.identifier, + conn) + from_table = client.get_table(from_table_ref) + if from_table.table_type == "VIEW": + raise dbt.exceptions.RuntimeException( + 'Renaming of views is not currently supported in BigQuery' + ) + + to_table_ref = self.connections.table_ref(to_relation.database, + to_relation.schema, + to_relation.identifier, + conn) + client.copy_table(from_table_ref, to_table_ref) + client.delete_table(from_table_ref) @available def list_schemas(self, database: str) -> List[str]: diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index d6365a276..1a06a5457 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -123,3 +123,7 @@ {% macro bigquery__alter_column_comment(relation, column_dict) -%} {% do adapter.update_column_descriptions(relation, column_dict) %} {% endmacro %} + +{% macro bigquery__rename_relation(from_relation, to_relation) -%} + {% do adapter.rename_relation(from_relation, to_relation) %} +{% endmacro %} From f108e8d2704d71bcba5621b7319f08459f723ff2 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Tue, 9 Jun 2020 11:58:24 -0400 Subject: [PATCH 211/860] Changed dataset_id to relation --- dbt/adapters/bigquery/impl.py | 9 ++++++--- dbt/include/bigquery/macros/adapters.sql | 4 ++-- dbt/include/bigquery/macros/materializations/view.sql | 4 ++-- 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 8c85bb54d..cf0ecc900 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -94,7 +94,7 @@ class BigqueryConfig(AdapterConfig): kms_key_name: Optional[str] = None labels: Optional[Dict[str, str]] = None partitions: Optional[List[str]] = None - grant_access_to: Optional[List[str]] = None + grant_access_to: Optional[List[BigQueryRelation]] = None class BigQueryAdapter(BaseAdapter): @@ -677,7 +677,7 @@ def get_table_options( return opts @available.parse_none - def grant_access_to(self, entity, entity_type, role, dataset_id): + def grant_access_to(self, entity, entity_type, role, relation): """ Given an entity, grants it access to a permissioned dataset. """ @@ -685,7 +685,10 @@ def grant_access_to(self, entity, entity_type, role, dataset_id): client = conn.handle dataset = client.get_dataset( - self.connections.dataset_from_id(dataset_id)) + self.connections.dataset_from_id( + f'{relation.database}.{relation.schema}' + ) + ) if entity_type == 'view': entity = self.connections.table_ref( diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 98739aea0..33be32a0d 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -124,6 +124,6 @@ {% do adapter.update_column_descriptions(relation, column_dict) %} {% endmacro %} -{% macro bigquery__grant_access_to(entity, entity_type, role, dataset_id) -%} - {% do adapter.grant_access_to(entity, entity_type, role, dataset_id) %} +{% macro bigquery__grant_access_to(entity, entity_type, role, relation) -%} + {% do adapter.grant_access_to(entity, entity_type, role, relation) %} {% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/view.sql b/dbt/include/bigquery/macros/materializations/view.sql index b5de687e0..8e0fd7251 100644 --- a/dbt/include/bigquery/macros/materializations/view.sql +++ b/dbt/include/bigquery/macros/materializations/view.sql @@ -15,8 +15,8 @@ {% do persist_docs(target_relation, model) %} {% if config.get('grant_access_to') %} - {% for dataset_id in config.get('grant_access_to') %} - {% do adapter.grant_access_to(this, 'view', None, dataset_id) %} + {% for relation in config.get('grant_access_to') %} + {% do adapter.grant_access_to(this, 'view', None, relation) %} {% endfor %} {% endif %} From 84ce5d1d7a022741fa8433f0af673c7e98e006a4 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Tue, 9 Jun 2020 14:43:59 -0400 Subject: [PATCH 212/860] Addressed PR comments --- dbt/adapters/bigquery/impl.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 111189234..762e579d6 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -143,7 +143,6 @@ def truncate_relation(self, relation: BigQueryRelation) -> None: def rename_relation( self, from_relation: BigQueryRelation, to_relation: BigQueryRelation ) -> None: - self.cache_renamed(from_relation, to_relation) conn = self.connections.get_thread_connection() client = conn.handle @@ -153,7 +152,9 @@ def rename_relation( from_relation.identifier, conn) from_table = client.get_table(from_table_ref) - if from_table.table_type == "VIEW": + if from_table.table_type == "VIEW" or \ + from_relation.type == RelationType.View or \ + to_relation.type == RelationType.View: raise dbt.exceptions.RuntimeException( 'Renaming of views is not currently supported in BigQuery' ) @@ -162,6 +163,8 @@ def rename_relation( to_relation.schema, to_relation.identifier, conn) + + self.cache_renamed(from_relation, to_relation) client.copy_table(from_table_ref, to_table_ref) client.delete_table(from_table_ref) From 10f2f35082946aee56f4877287340f821ed6f6c9 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Wed, 10 Jun 2020 11:35:18 -0400 Subject: [PATCH 213/860] grant_access no longer common and takes in grant_target_dict param --- dbt/adapters/bigquery/impl.py | 18 +++++++++++++----- dbt/include/bigquery/macros/adapters.sql | 4 ---- dbt/include/bigquery/macros/etc.sql | 5 ++++- .../bigquery/macros/materializations/view.sql | 4 ++-- 4 files changed, 19 insertions(+), 12 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index cf0ecc900..e471474e8 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -77,6 +77,15 @@ def parse(cls, raw_partition_by) -> Optional['PartitionConfig']: ) +@dataclass +class GrantTarget(JsonSchemaMixin): + dataset: str + project: str + + def render(self): + return f'{self.project}.{self.dataset}' + + def _stub_relation(*args, **kwargs): return BigQueryRelation.create( database='', @@ -94,7 +103,7 @@ class BigqueryConfig(AdapterConfig): kms_key_name: Optional[str] = None labels: Optional[Dict[str, str]] = None partitions: Optional[List[str]] = None - grant_access_to: Optional[List[BigQueryRelation]] = None + grant_access_to: Optional[List[Dict[str, str]]] = None class BigQueryAdapter(BaseAdapter): @@ -677,17 +686,16 @@ def get_table_options( return opts @available.parse_none - def grant_access_to(self, entity, entity_type, role, relation): + def grant_access_to(self, entity, entity_type, role, grant_target_dict): """ Given an entity, grants it access to a permissioned dataset. """ conn = self.connections.get_thread_connection() client = conn.handle + grant_target = GrantTarget.from_dict(grant_target_dict) dataset = client.get_dataset( - self.connections.dataset_from_id( - f'{relation.database}.{relation.schema}' - ) + self.connections.dataset_from_id(grant_target.render()) ) if entity_type == 'view': diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 33be32a0d..d6365a276 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -123,7 +123,3 @@ {% macro bigquery__alter_column_comment(relation, column_dict) -%} {% do adapter.update_column_descriptions(relation, column_dict) %} {% endmacro %} - -{% macro bigquery__grant_access_to(entity, entity_type, role, relation) -%} - {% do adapter.grant_access_to(entity, entity_type, role, relation) %} -{% endmacro %} diff --git a/dbt/include/bigquery/macros/etc.sql b/dbt/include/bigquery/macros/etc.sql index 7bd0cba90..a10ad1a5b 100644 --- a/dbt/include/bigquery/macros/etc.sql +++ b/dbt/include/bigquery/macros/etc.sql @@ -1,4 +1,7 @@ - {% macro date_sharded_table(base_name) %} {{ return(base_name ~ "[DBT__PARTITION_DATE]") }} {% endmacro %} + +{% macro grant_access_to(entity, entity_type, role, grant_target_dict) -%} + {% do adapter.grant_access_to(entity, entity_type, role, grant_target_dict) %} +{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/view.sql b/dbt/include/bigquery/macros/materializations/view.sql index 8e0fd7251..887905129 100644 --- a/dbt/include/bigquery/macros/materializations/view.sql +++ b/dbt/include/bigquery/macros/materializations/view.sql @@ -15,8 +15,8 @@ {% do persist_docs(target_relation, model) %} {% if config.get('grant_access_to') %} - {% for relation in config.get('grant_access_to') %} - {% do adapter.grant_access_to(this, 'view', None, relation) %} + {% for grant_target_dict in config.get('grant_access_to') %} + {% do adapter.grant_access_to(this, 'view', None, grant_target_dict) %} {% endfor %} {% endif %} From 55f7c58b617a0822b86d96cb5463148b5573c3d4 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Wed, 10 Jun 2020 14:33:29 -0400 Subject: [PATCH 214/860] Debug message change --- dbt/adapters/bigquery/impl.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 1f7a144af..325190aed 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -730,7 +730,8 @@ def grant_access_to(self, entity, entity_type, role, grant_target_dict): access_entries = dataset.access_entries if access_entry in access_entries: - logger.debug(f"Access entry {access_entry} already in dataset") + logger.debug(f"Access entry {access_entry} " + f"already exists in dataset") return access_entries.append(AccessEntry(role, entity_type, entity)) From 1fbfc9dd5d33c015f16b1b31b0dce319e66bc20a Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 10 Jun 2020 15:11:31 -0600 Subject: [PATCH 215/860] =?UTF-8?q?Bump=20version:=200.17.0=20=E2=86=92=20?= =?UTF-8?q?0.17.1a1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 063151983..45490b1a2 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.0' +version = '0.17.1a1' diff --git a/setup.py b/setup.py index e4d76a5d2..409e2902f 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.0" +package_version = "0.17.1a1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From aa1d8da2b2327cb408d74acd9183eb1f164edecb Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Thu, 11 Jun 2020 13:10:10 -0400 Subject: [PATCH 216/860] Added integration test --- dbt/adapters/bigquery/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/__init__.py b/dbt/adapters/bigquery/__init__.py index daff48a32..dc45b50c3 100644 --- a/dbt/adapters/bigquery/__init__.py +++ b/dbt/adapters/bigquery/__init__.py @@ -2,7 +2,7 @@ from dbt.adapters.bigquery.connections import BigQueryCredentials from dbt.adapters.bigquery.relation import BigQueryRelation # noqa from dbt.adapters.bigquery.column import BigQueryColumn # noqa -from dbt.adapters.bigquery.impl import BigQueryAdapter +from dbt.adapters.bigquery.impl import BigQueryAdapter, GrantTarget from dbt.adapters.base import AdapterPlugin from dbt.include import bigquery From 3a2f63d39f1f417828a4c307b02f6e5ecd602ee6 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Thu, 11 Jun 2020 13:19:05 -0400 Subject: [PATCH 217/860] Added noqa flag to GrantTarget import --- dbt/adapters/bigquery/__init__.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/__init__.py b/dbt/adapters/bigquery/__init__.py index dc45b50c3..c85e79b38 100644 --- a/dbt/adapters/bigquery/__init__.py +++ b/dbt/adapters/bigquery/__init__.py @@ -2,7 +2,7 @@ from dbt.adapters.bigquery.connections import BigQueryCredentials from dbt.adapters.bigquery.relation import BigQueryRelation # noqa from dbt.adapters.bigquery.column import BigQueryColumn # noqa -from dbt.adapters.bigquery.impl import BigQueryAdapter, GrantTarget +from dbt.adapters.bigquery.impl import BigQueryAdapter, GrantTarget # noqa from dbt.adapters.base import AdapterPlugin from dbt.include import bigquery From 0a49b10c8bcb90c86667568aac3ac9a6c6ea2611 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Mon, 15 Jun 2020 12:54:27 -0400 Subject: [PATCH 218/860] Add support for altering BigQuery column types --- dbt/include/bigquery/macros/adapters.sql | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 1a06a5457..c092a174b 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -127,3 +127,25 @@ {% macro bigquery__rename_relation(from_relation, to_relation) -%} {% do adapter.rename_relation(from_relation, to_relation) %} {% endmacro %} + +{% macro bigquery__alter_column_type(relation, column_name, new_column_type) -%} + {% set relation_columns = get_columns_in_relation(relation) %} + + {% set sql %} + select + {%- for col in relation_columns -%} + {% if col.column == column_name %} + CAST({{ col.quoted }} AS {{ new_column_type }}) AS {{ col.quoted }} + {%- else %} + {{ col.quoted }} + {%- endif %} + {%- if not loop.last %},{% endif -%} + {%- endfor %} + from {{ relation }} + {% endset %} + + {% call statement('alter_column_type') %} + {{ create_table_as(False, relation, sql)}} + {%- endcall %} + +{% endmacro %} From b8982578878610220cda1a0d581e7bdcd9e30429 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Tue, 16 Jun 2020 18:11:08 -0400 Subject: [PATCH 219/860] Added warning comment to alter macro --- dbt/include/bigquery/macros/adapters.sql | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index c092a174b..1fda4609c 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -129,6 +129,12 @@ {% endmacro %} {% macro bigquery__alter_column_type(relation, column_name, new_column_type) -%} + {# + Changing a column's data type using a query requires you to scan the entire table. + The query charges can be significant if the table is very large. + + https://cloud.google.com/bigquery/docs/manually-changing-schemas#changing_a_columns_data_type + #} {% set relation_columns = get_columns_in_relation(relation) %} {% set sql %} From 3ad5b970674a11fc1c618a0393a53e76ae7e00ef Mon Sep 17 00:00:00 2001 From: Bob De Schutter Date: Fri, 19 Jun 2020 16:24:01 +0200 Subject: [PATCH 220/860] Fix persist docs functionality for bigquery to include descriptions for nested columns --- dbt/adapters/bigquery/impl.py | 44 +++++++++++++++++++++---- dbt/include/bigquery/macros/catalog.sql | 7 ++-- 2 files changed, 41 insertions(+), 10 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index caafcb0fb..0b07eb728 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -564,6 +564,36 @@ def get_table_ref_from_relation(self, conn, relation): relation.identifier, conn) + def _update_column_dict(self, bq_column_dict, dbt_columns, parent=''): + """ + Helper function to recursively traverse the schema of a table in the + update_column_descriptions function below. + + bq_column_dict should be a dict as obtained by the to_api_repr() + function of a SchemaField object. + """ + if parent: + dotted_column_name = '{}.{}'.format(parent, bq_column_dict['name']) + else: + dotted_column_name = bq_column_dict['name'] + + if dotted_column_name in dbt_columns: + column_config = dbt_columns[dotted_column_name] + bq_column_dict['description'] = column_config.get('description') + + new_fields = [] + for child_col_dict in bq_column_dict.get('fields', list()): + new_child_column_dict = self._update_column_dict( + child_col_dict, + dbt_columns, + parent=dotted_column_name + ) + new_fields.append(new_child_column_dict) + + bq_column_dict['fields'] = new_fields + + return bq_column_dict + @available.parse_none def update_column_descriptions(self, relation, columns): if len(columns) == 0: @@ -574,13 +604,13 @@ def update_column_descriptions(self, relation, columns): table = conn.handle.get_table(table_ref) new_schema = [] - for column in table.schema: - if column.name in columns: - column_config = columns[column.name] - column_dict = column.to_api_repr() - column_dict['description'] = column_config.get('description') - column = SchemaField.from_api_repr(column_dict) - new_schema.append(column) + for bq_column in table.schema: + bq_column_dict = bq_column.to_api_repr() + new_bq_column_dict = self._update_column_dict( + bq_column_dict, + columns + ) + new_schema.append(SchemaField.from_api_repr(new_bq_column_dict)) new_table = google.cloud.bigquery.Table(table_ref, schema=new_schema) conn.handle.update_table(new_table, ['schema']) diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql index ed64af881..343a1ee04 100644 --- a/dbt/include/bigquery/macros/catalog.sql +++ b/dbt/include/bigquery/macros/catalog.sql @@ -83,7 +83,7 @@ -- use the "real" column name from the paths query below column_name as base_column_name, ordinal_position as column_index, - cast(null as string) as column_comment, + -- cast(null as string) as column_comment, is_partitioning_column, clustering_ordinal_position @@ -99,10 +99,11 @@ concat(table_catalog, '.', table_schema, '.', table_name) as relation_id, field_path as column_name, data_type as column_type, - column_name as base_column_name + column_name as base_column_name, + description as column_comment from {{ information_schema.replace(information_schema_view='COLUMN_FIELD_PATHS') }} - where data_type not like 'STRUCT%' + -- where data_type not like 'STRUCT%' ), From 17a1783bb17d08723f83c4cea51a9b644fbad50b Mon Sep 17 00:00:00 2001 From: Bob De Schutter Date: Fri, 19 Jun 2020 16:56:17 +0200 Subject: [PATCH 221/860] Remove commented out code Co-authored-by: Jacob Beck --- dbt/include/bigquery/macros/catalog.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql index 343a1ee04..6822d88a6 100644 --- a/dbt/include/bigquery/macros/catalog.sql +++ b/dbt/include/bigquery/macros/catalog.sql @@ -83,7 +83,6 @@ -- use the "real" column name from the paths query below column_name as base_column_name, ordinal_position as column_index, - -- cast(null as string) as column_comment, is_partitioning_column, clustering_ordinal_position @@ -103,7 +102,6 @@ description as column_comment from {{ information_schema.replace(information_schema_view='COLUMN_FIELD_PATHS') }} - -- where data_type not like 'STRUCT%' ), From 0c40cd674dfadeef75c0cc361ab6f4c4ed48a8dc Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Fri, 19 Jun 2020 21:13:16 +0000 Subject: [PATCH 222/860] Release dbt v0.17.1rc1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 45490b1a2..a2e726b39 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.1a1' +version = '0.17.1rc1' diff --git a/setup.py b/setup.py index 409e2902f..35eff781d 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.1a1" +package_version = "0.17.1rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 4278a3d8920d19d21b7dd1b5b70bc93e28e9f818 Mon Sep 17 00:00:00 2001 From: Claire Carroll Date: Mon, 22 Jun 2020 08:48:59 -0400 Subject: [PATCH 223/860] Update other logos --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index ec3527497..b7e967abe 100644 --- a/README.md +++ b/README.md @@ -1,5 +1,5 @@

- dbt logo + dbt logo

**[dbt](https://www.getdbt.com/)** (data build tool) enables data analysts and engineers to transform their data using the same practices that software engineers use to build applications. From 23d16c18e950762c9aff43f6a9e2771e1654a907 Mon Sep 17 00:00:00 2001 From: Alessandro Puccetti Date: Wed, 10 Jun 2020 18:45:19 +0100 Subject: [PATCH 224/860] plugins/bigquery: log processed bytes value for all statement types Fixes https://github.com/fishtown-analytics/dbt/issues/2526 --- dbt/adapters/bigquery/connections.py | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 74e5d7d36..b62dba532 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -8,7 +8,7 @@ from google.api_core import retry, client_info from google.oauth2 import service_account -from dbt.utils import format_bytes +from dbt.utils import format_bytes, format_rows_number from dbt.clients import agate_helper, gcloud from dbt.exceptions import ( FailedToConnectException, RuntimeException, DatabaseException @@ -245,16 +245,21 @@ def execute(self, sql, auto_begin=False, fetch=None): conn = self.get_thread_connection() client = conn.handle table = client.get_table(query_job.destination) - status = 'CREATE TABLE ({})'.format(table.num_rows) + processed = format_bytes(query_job.total_bytes_processed) + status = 'CREATE TABLE ({} rows, {} processed)'.format( + format_rows_number(table.num_rows), + format_bytes(query_job.total_bytes_processed), + ) elif query_job.statement_type == 'SCRIPT': processed = format_bytes(query_job.total_bytes_processed) status = f'SCRIPT ({processed} processed)' elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: - status = '{} ({})'.format( + status = '{} ({} rows, {} processed)'.format( query_job.statement_type, - query_job.num_dml_affected_rows + format_rows_number(query_job.num_dml_affected_rows), + format_bytes(query_job.total_bytes_processed), ) else: From f10be44957b980d5d794303bfe69f08f91945788 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Tue, 23 Jun 2020 10:14:24 -0600 Subject: [PATCH 225/860] Try to make imports a little more sane, ordering-wise consolidate dbt.ui, move non-rpc node_runners into their tasks move parse_cli_vars into config.utils get rid of logger/exceptions requirements in dbt.utils --- dbt/adapters/bigquery/impl.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 325190aed..1ae977695 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -7,8 +7,8 @@ import dbt.flags as flags import dbt.clients.gcloud import dbt.clients.agate_helper -import dbt.links +from dbt import ui from dbt.adapters.base import ( BaseAdapter, available, RelationType, SchemaSearchMap, AdapterConfig ) @@ -17,7 +17,7 @@ from dbt.adapters.bigquery import BigQueryConnectionManager from dbt.contracts.connection import Connection from dbt.contracts.graph.manifest import Manifest -from dbt.logger import GLOBAL_LOGGER as logger +from dbt.logger import GLOBAL_LOGGER as logger, print_timestamped_line from dbt.utils import filter_null_values import google.auth @@ -454,8 +454,9 @@ def _bq_table_to_relation(self, bq_table): @classmethod def warning_on_hooks(hook_type): msg = "{} is not supported in bigquery and will be ignored" - dbt.ui.printer.print_timestamped_line(msg.format(hook_type), - dbt.ui.printer.COLOR_FG_YELLOW) + print_timestamped_line( + msg.format(hook_type), ui.COLOR_FG_YELLOW + ) @available def add_query(self, sql, auto_begin=True, bindings=None, From 0024cc9099be94fc6ad8dc8b4a1556bf0a86882d Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Thu, 25 Jun 2020 21:05:41 +0000 Subject: [PATCH 226/860] Release dbt v0.17.1rc2 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index a2e726b39..8085c242a 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.1rc1' +version = '0.17.1rc2' diff --git a/setup.py b/setup.py index 35eff781d..a460e58d0 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.1rc1" +package_version = "0.17.1rc2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 89398fe4d9e523f782a75152604aac3335723fc1 Mon Sep 17 00:00:00 2001 From: brunomurino Date: Thu, 25 Jun 2020 23:27:08 +0100 Subject: [PATCH 227/860] added option --adapter to dbt init, to create sample profiles.yml based on chosen adapter --- dbt/include/bigquery/sample_profiles.yml | 13 +++++++++++++ 1 file changed, 13 insertions(+) create mode 100644 dbt/include/bigquery/sample_profiles.yml diff --git a/dbt/include/bigquery/sample_profiles.yml b/dbt/include/bigquery/sample_profiles.yml new file mode 100644 index 000000000..7e5193e64 --- /dev/null +++ b/dbt/include/bigquery/sample_profiles.yml @@ -0,0 +1,13 @@ +my-bigquery-db: + target: dev + outputs: + dev: + type: bigquery + method: service-account + project: [GCP project id] + dataset: [the name of your dbt dataset] + threads: [1 or more] + keyfile: [/path/to/bigquery/keyfile.json] + timeout_seconds: 300 + priority: interactive + retries: 1 \ No newline at end of file From d356a3f21742f5520d71f235cd5806c6a1c16051 Mon Sep 17 00:00:00 2001 From: brunomurino Date: Fri, 26 Jun 2020 18:12:18 +0100 Subject: [PATCH 228/860] updated setup.py of adapters to include sample_profiles.yml --- setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/setup.py b/setup.py index b88d6cf25..7e91b6e44 100644 --- a/setup.py +++ b/setup.py @@ -34,6 +34,7 @@ package_data={ 'dbt': [ 'include/bigquery/dbt_project.yml', + 'include/bigquery/sample_profiles.yml', 'include/bigquery/macros/*.sql', 'include/bigquery/macros/**/*.sql', ] From b9cb5b7272ef001a1898f4b11afabcf4f8fb107c Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 29 Jun 2020 14:57:05 -0600 Subject: [PATCH 229/860] On bigquery, persist docs for seeds as well as tables/views --- dbt/adapters/bigquery/impl.py | 15 +++++++++++++++ .../bigquery/macros/materializations/seed.sql | 3 +++ 2 files changed, 18 insertions(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 0b07eb728..5800b2d6b 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -615,6 +615,21 @@ def update_column_descriptions(self, relation, columns): new_table = google.cloud.bigquery.Table(table_ref, schema=new_schema) conn.handle.update_table(new_table, ['schema']) + @available.parse_none + def update_table_description(self, database, schema, identifier, description): + conn = self.connections.get_thread_connection() + client = conn.handle + + table_ref = self.connections.table_ref( + database, + schema, + identifier, + conn + ) + table = client.get_table(table_ref) + table.description = description + client.update_table(table, ['description']) + @available.parse_none def alter_table_add_columns(self, relation, columns): diff --git a/dbt/include/bigquery/macros/materializations/seed.sql b/dbt/include/bigquery/macros/materializations/seed.sql index d62867374..d95cc4e1b 100644 --- a/dbt/include/bigquery/macros/materializations/seed.sql +++ b/dbt/include/bigquery/macros/materializations/seed.sql @@ -12,5 +12,8 @@ {%- set column_override = model['config'].get('column_types', {}) -%} {{ adapter.load_dataframe(model['database'], model['schema'], model['alias'], agate_table, column_override) }} + {% if config.persist_relation_docs() and 'description' in model %} + {{ adapter.update_table_description(model['database'], model['schema'], model['alias'], model['description']) }} + {% endif %} {% endmacro %} From d51c6983199875cbe0df285079fc57b7fbcf374a Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 29 Jun 2020 16:02:49 -0600 Subject: [PATCH 230/860] flake8 --- dbt/adapters/bigquery/impl.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 5800b2d6b..fdb6a794f 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -616,7 +616,9 @@ def update_column_descriptions(self, relation, columns): conn.handle.update_table(new_table, ['schema']) @available.parse_none - def update_table_description(self, database, schema, identifier, description): + def update_table_description( + self, database: str, schema: str, identifier: str, description: str + ): conn = self.connections.get_thread_connection() client = conn.handle From 0f09d787c9cf41ec5d306df9f7eb67cac20e3e88 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Wed, 1 Jul 2020 13:19:07 +0000 Subject: [PATCH 231/860] Release dbt v0.17.1rc3 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 8085c242a..c8cc53400 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.1rc2' +version = '0.17.1rc3' diff --git a/setup.py b/setup.py index a460e58d0..b154a1dea 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.1rc2" +package_version = "0.17.1rc3" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 91416da3346365dbbda3f3254691b412dcd502ae Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 2 Jul 2020 07:58:39 -0600 Subject: [PATCH 232/860] move the sql for getting different rows into dbt proper, from the test suite. Bump pytest dependency. --- dbt/adapters/bigquery/impl.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 1ae977695..7b80f096f 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -738,3 +738,17 @@ def grant_access_to(self, entity, entity_type, role, grant_target_dict): access_entries.append(AccessEntry(role, entity_type, entity)) dataset.access_entries = access_entries client.update_dataset(dataset, ['access_entries']) + + def get_rows_different_sql( + self, + relation_a: BigQueryRelation, + relation_b: BigQueryRelation, + column_names: Optional[List[str]] = None, + except_operator='EXCEPT DISTINCT' + ) -> str: + return super().get_rows_different_sql( + relation_a=relation_a, + relation_b=relation_b, + column_names=column_names, + except_operator=except_operator, + ) From 87095b142d7f122c6bfc3de2a7ce2f38ba61b97f Mon Sep 17 00:00:00 2001 From: brunomurino Date: Thu, 2 Jul 2020 21:14:52 +0100 Subject: [PATCH 233/860] updated bigquery sample_profiles.yml --- dbt/include/bigquery/sample_profiles.yml | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/dbt/include/bigquery/sample_profiles.yml b/dbt/include/bigquery/sample_profiles.yml index 7e5193e64..36f47fdbc 100644 --- a/dbt/include/bigquery/sample_profiles.yml +++ b/dbt/include/bigquery/sample_profiles.yml @@ -1,7 +1,18 @@ -my-bigquery-db: - target: dev +default: outputs: + dev: + type: bigquery + method: oauth + project: [GCP project id] + dataset: [the name of your dbt dataset] # You can also use "schema" here + threads: [1 or more] + timeout_seconds: 300 + location: US # Optional, one of US or EU + priority: interactive + retries: 1 + + prod: type: bigquery method: service-account project: [GCP project id] @@ -10,4 +21,6 @@ my-bigquery-db: keyfile: [/path/to/bigquery/keyfile.json] timeout_seconds: 300 priority: interactive - retries: 1 \ No newline at end of file + retries: 1 + + target: dev \ No newline at end of file From c0ee8571d6ed8368916dfcc0224079bef19a4586 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Wed, 8 Jul 2020 16:12:55 +0000 Subject: [PATCH 234/860] Release dbt v0.17.1rc4 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index c8cc53400..c8e29143d 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.1rc3' +version = '0.17.1rc4' diff --git a/setup.py b/setup.py index b154a1dea..09977e44f 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.1rc3" +package_version = "0.17.1rc4" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 54676f84c506b9f9abf990b7885004570be4a7f0 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Mon, 20 Jul 2020 12:48:01 +0000 Subject: [PATCH 235/860] Release dbt v0.17.1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index c8e29143d..b691477f5 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.1rc4' +version = '0.17.1' diff --git a/setup.py b/setup.py index 09977e44f..014c6b38d 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.1rc4" +package_version = "0.17.1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 98d0c3bf1f0c32a0e5a859d746fa628e7a7e587b Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 20 Jul 2020 09:39:04 -0600 Subject: [PATCH 236/860] =?UTF-8?q?Bump=20version:=200.17.1=20=E2=86=92=20?= =?UTF-8?q?0.17.2a1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index b691477f5..07399aabe 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.1' +version = '0.17.2a1' diff --git a/setup.py b/setup.py index 014c6b38d..8259dece8 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.1" +package_version = "0.17.2a1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From ba3a98b5fd69b179edf6de4965a9ba8b2073808e Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Tue, 21 Jul 2020 13:28:43 +0000 Subject: [PATCH 237/860] Release dbt v0.17.2b1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 07399aabe..7d15c5aa5 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.2a1' +version = '0.17.2b1' diff --git a/setup.py b/setup.py index 8259dece8..a58c7bc1e 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.2a1" +package_version = "0.17.2b1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 1dc2a7da9e121df39855b4c33d00b7e7aeeaca8a Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Mon, 27 Jul 2020 10:36:32 -0600 Subject: [PATCH 238/860] Handle the fallout of closing connections in release - close() implies rollback, so do not call it - make sure to not open new connections for executors in single-threaded mode - logging cleanups - fix a test case that never acquired connections - to cancel other connections, one must first acquire a connection for the master thread - change a number of release() calls to rollback release vs rollback --- dbt/adapters/bigquery/connections.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index ec07565ed..7254d9058 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -10,6 +10,7 @@ from dbt.utils import format_bytes from dbt.clients import agate_helper, gcloud +from dbt.contracts.connection import ConnectionState from dbt.exceptions import ( FailedToConnectException, RuntimeException, DatabaseException ) @@ -111,7 +112,7 @@ def cancel_open(self) -> None: @classmethod def close(cls, connection): - connection.state = 'closed' + connection.state = ConnectionState.CLOSED return connection From 68994b9f149a28f631ee5e7e30a55aaf1b9ddafb Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Tue, 28 Jul 2020 20:20:16 +0000 Subject: [PATCH 239/860] Release dbt v0.17.2rc1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 7d15c5aa5..7ffcabccf 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.2b1' +version = '0.17.2rc1' diff --git a/setup.py b/setup.py index a58c7bc1e..9ed4fee9f 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.2b1" +package_version = "0.17.2rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From a90418ad01bf3b05c79553c27f4cc74cfe1ae697 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Wed, 29 Jul 2020 19:09:25 +0000 Subject: [PATCH 240/860] Release dbt v0.17.2 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 7ffcabccf..c89f21ad3 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.17.2rc1' +version = '0.17.2' diff --git a/setup.py b/setup.py index 9ed4fee9f..909765b92 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.17.2rc1" +package_version = "0.17.2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 8ab47c86a14fe322e30d80fd58a4a60b09717aa5 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Thu, 25 Jun 2020 10:17:24 -0400 Subject: [PATCH 241/860] Added support for setting policy tags for columns in BigQuery --- dbt/adapters/bigquery/impl.py | 16 +++++++++++++++- dbt/include/bigquery/macros/adapters.sql | 2 +- setup.py | 2 +- 3 files changed, 17 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 250407e7c..f9f413f0b 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -612,6 +612,10 @@ def _update_column_dict(self, bq_column_dict, dbt_columns, parent=''): if dotted_column_name in dbt_columns: column_config = dbt_columns[dotted_column_name] bq_column_dict['description'] = column_config.get('description') + if column_config.get('tags'): + bq_column_dict['policyTags'] = {} + bq_column_dict['policyTags']['names'] = \ + column_config.get('tags') new_fields = [] for child_col_dict in bq_column_dict.get('fields', list()): @@ -627,7 +631,7 @@ def _update_column_dict(self, bq_column_dict, dbt_columns, parent=''): return bq_column_dict @available.parse_none - def update_column_descriptions(self, relation, columns): + def update_column(self, relation, columns): if len(columns) == 0: return @@ -643,6 +647,16 @@ def update_column_descriptions(self, relation, columns): columns ) new_schema.append(SchemaField.from_api_repr(new_bq_column_dict)) + # for column in table.schema: + # if column.name in columns: + # column_config = columns[column.name] + # column_dict = column.to_api_repr() + # column_dict['description'] = column_config.get('description') + # if column_config.get('tags'): + # column_dict['policyTags'] = {} + # column_dict['policyTags']['names'] = column_config.get('tags') + # column = SchemaField.from_api_repr(column_dict) + # new_schema.append(column) new_table = google.cloud.bigquery.Table(table_ref, schema=new_schema) conn.handle.update_table(new_table, ['schema']) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 1fda4609c..37cb04090 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -121,7 +121,7 @@ {% endmacro %} {% macro bigquery__alter_column_comment(relation, column_dict) -%} - {% do adapter.update_column_descriptions(relation, column_dict) %} + {% do adapter.update_column(relation, column_dict) %} {% endmacro %} {% macro bigquery__rename_relation(from_relation, to_relation) -%} diff --git a/setup.py b/setup.py index 7b8274665..c9cebb7fd 100644 --- a/setup.py +++ b/setup.py @@ -42,7 +42,7 @@ 'dbt-core=={}'.format(package_version), 'protobuf>=3.6.0,<3.12', 'google-cloud-core>=1.3.0,<1.4', - 'google-cloud-bigquery>=1.24.0,<1.25.0', + 'google-cloud-bigquery>=1.25.0,<1.26.0', 'google-api-core>=1.16.0,<1.17.0', 'googleapis-common-protos>=1.6.0,<1.7.0', 'six>=1.14.0', From 8e85fa5fc1753bdff67a11bd3814164b0f06a8c4 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Thu, 25 Jun 2020 10:22:21 -0400 Subject: [PATCH 242/860] Changed update_column to update_columns --- dbt/adapters/bigquery/impl.py | 2 +- dbt/include/bigquery/macros/adapters.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index f9f413f0b..062fd6099 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -631,7 +631,7 @@ def _update_column_dict(self, bq_column_dict, dbt_columns, parent=''): return bq_column_dict @available.parse_none - def update_column(self, relation, columns): + def update_columns(self, relation, columns): if len(columns) == 0: return diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 37cb04090..3bb7cd606 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -121,7 +121,7 @@ {% endmacro %} {% macro bigquery__alter_column_comment(relation, column_dict) -%} - {% do adapter.update_column(relation, column_dict) %} + {% do adapter.update_columns(relation, column_dict) %} {% endmacro %} {% macro bigquery__rename_relation(from_relation, to_relation) -%} From 7ca469ea5206483e88e733904b65e35dbe8aa9e6 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Thu, 25 Jun 2020 10:36:35 -0400 Subject: [PATCH 243/860] Linting fix --- dbt/adapters/bigquery/impl.py | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 062fd6099..67357ad5e 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -647,16 +647,6 @@ def update_columns(self, relation, columns): columns ) new_schema.append(SchemaField.from_api_repr(new_bq_column_dict)) - # for column in table.schema: - # if column.name in columns: - # column_config = columns[column.name] - # column_dict = column.to_api_repr() - # column_dict['description'] = column_config.get('description') - # if column_config.get('tags'): - # column_dict['policyTags'] = {} - # column_dict['policyTags']['names'] = column_config.get('tags') - # column = SchemaField.from_api_repr(column_dict) - # new_schema.append(column) new_table = google.cloud.bigquery.Table(table_ref, schema=new_schema) conn.handle.update_table(new_table, ['schema']) From f5087ddfedf3dd3f52fa59ec30b16a3986b492d4 Mon Sep 17 00:00:00 2001 From: Azhar Dewji Date: Wed, 29 Jul 2020 17:19:54 -0400 Subject: [PATCH 244/860] Addressed PR comments --- dbt/adapters/bigquery/impl.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 67357ad5e..36049a46d 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -612,10 +612,10 @@ def _update_column_dict(self, bq_column_dict, dbt_columns, parent=''): if dotted_column_name in dbt_columns: column_config = dbt_columns[dotted_column_name] bq_column_dict['description'] = column_config.get('description') - if column_config.get('tags'): - bq_column_dict['policyTags'] = {} - bq_column_dict['policyTags']['names'] = \ - column_config.get('tags') + if column_config.get('policy_tags'): + bq_column_dict['policyTags'] = { + 'names': column_config.get('policy_tags') + } new_fields = [] for child_col_dict in bq_column_dict.get('fields', list()): From c60e6e0def172c55efc88bc66c9a53075dfdfbea Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Thu, 30 Jul 2020 17:11:32 +0000 Subject: [PATCH 245/860] Release dbt v0.18.0b2 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 7985b917d..33d6ab18e 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.18.0b1' +version = '0.18.0b2' diff --git a/setup.py b/setup.py index c9cebb7fd..b51a03724 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.18.0b1" +package_version = "0.18.0b2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From e48e7068a31f85d627d79fe4faaf3f9123f0a164 Mon Sep 17 00:00:00 2001 From: Preston Marshall Date: Sat, 1 Aug 2020 15:12:00 -0400 Subject: [PATCH 246/860] first try --- dbt/adapters/bigquery/connections.py | 27 ++++++++++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index d18c4ec92..9ed789e62 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -3,9 +3,11 @@ from typing import Optional, Any, Dict import google.auth +import google.auth.exceptions import google.cloud.bigquery import google.cloud.exceptions from google.api_core import retry, client_info +from google.auth import impersonated_credentials from google.oauth2 import service_account from dbt.utils import format_bytes, format_rows_number @@ -45,6 +47,7 @@ class BigQueryCredentials(Credentials): priority: Optional[Priority] = None retries: Optional[int] = 1 maximum_bytes_billed: Optional[int] = None + impersonate_service_account: Optional[str] = None _ALIASES = { 'project': 'database', 'dataset': 'schema', @@ -92,6 +95,14 @@ def exception_handler(self, sql): message = "Access denied while running query" self.handle_error(e, message) + except google.auth.exceptions.RefreshError: + message = "Unable to generate access token, if you're using " \ + "impersonate_service_account, make sure your " \ + 'initial account has the "roles/' \ + 'iam.serviceAccountTokenCreator" role on the ' \ + 'account you are trying to impersonate.' + raise RuntimeException(message) + except Exception as e: logger.debug("Unhandled error while running:\n{}".format(sql)) logger.debug(e) @@ -142,10 +153,24 @@ def get_bigquery_credentials(cls, profile_credentials): error = ('Invalid `method` in profile: "{}"'.format(method)) raise FailedToConnectException(error) + @classmethod + def get_impersonated_bigquery_credentials(cls, profile_credentials): + source_credentials = cls.get_bigquery_credentials(profile_credentials) + return impersonated_credentials.Credentials( + source_credentials=source_credentials, + target_principal=profile_credentials.impersonate_service_account, + target_scopes=list(cls.SCOPE), + lifetime=profile_credentials.timeout_seconds, + ) + @classmethod def get_bigquery_client(cls, profile_credentials): + if profile_credentials.impersonate_service_account: + creds =\ + cls.get_impersonated_bigquery_credentials(profile_credentials) + else: + creds = cls.get_bigquery_credentials(profile_credentials) database = profile_credentials.database - creds = cls.get_bigquery_credentials(profile_credentials) location = getattr(profile_credentials, 'location', None) info = client_info.ClientInfo(user_agent=f'dbt-{dbt_version}') From 9c1f5ff2530d715b957cd417e8bd4c8b4c2a4415 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Tue, 11 Aug 2020 12:29:52 -0600 Subject: [PATCH 247/860] Add test from dev --- dbt/adapters/bigquery/connections.py | 31 +++++++++++++++++++++++++--- 1 file changed, 28 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 9ed789e62..7f92a2db8 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -1,5 +1,6 @@ from contextlib import contextmanager from dataclasses import dataclass +from requests.exceptions import ConnectionError from typing import Optional, Any, Dict import google.auth @@ -25,6 +26,18 @@ BQ_QUERY_JOB_SPLIT = '-----Query Job SQL Follows-----' +REOPENABLE_ERRORS = { + ConnectionResetError, + ConnectionError, +} + +RETRYABLE_ERRORS = { + google.cloud.exceptions.ServerError, + google.cloud.exceptions.BadRequest, + ConnectionResetError, + ConnectionError, +} + class Priority(StrEnum): Interactive = 'interactive' @@ -390,12 +403,21 @@ def _query_and_results(self, client, sql, conn, job_params, timeout=None): def _retry_and_handle(self, msg, conn, fn): """retry a function call within the context of exception_handler.""" + def reopen_conn_on_error(error): + for type in REOPENABLE_ERRORS: + if isinstance(error, type): + logger.warning('Reopening connection after {!r}', error) + self.close(conn) + self.open(conn) + return + with self.exception_handler(msg): return retry.retry_target( target=fn, predicate=_ErrorCounter(self.get_retries(conn)).count_error, sleep_generator=self._retry_generator(), - deadline=None) + deadline=None, + on_error=reopen_conn_on_error) def _retry_generator(self): """Generates retry intervals that exponentially back off.""" @@ -425,5 +447,8 @@ def count_error(self, error): def _is_retryable(error): - """Return true for 500 level (retryable) errors.""" - return isinstance(error, google.cloud.exceptions.ServerError) + """Return true for errors that are unlikely to occur again if retried.""" + for error_type in RETRYABLE_ERRORS: + if isinstance(error, error_type): + return True + return False From 71c38eabeedfffae5daac7e515cbd70e55e9ab36 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Tue, 11 Aug 2020 15:15:04 -0600 Subject: [PATCH 248/860] use isinstance with tuple --- dbt/adapters/bigquery/connections.py | 24 +++++++++++------------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 7f92a2db8..97b386f30 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -26,17 +26,17 @@ BQ_QUERY_JOB_SPLIT = '-----Query Job SQL Follows-----' -REOPENABLE_ERRORS = { +REOPENABLE_ERRORS = ( ConnectionResetError, ConnectionError, -} +) -RETRYABLE_ERRORS = { +RETRYABLE_ERRORS = ( google.cloud.exceptions.ServerError, google.cloud.exceptions.BadRequest, ConnectionResetError, ConnectionError, -} +) class Priority(StrEnum): @@ -404,12 +404,11 @@ def _query_and_results(self, client, sql, conn, job_params, timeout=None): def _retry_and_handle(self, msg, conn, fn): """retry a function call within the context of exception_handler.""" def reopen_conn_on_error(error): - for type in REOPENABLE_ERRORS: - if isinstance(error, type): - logger.warning('Reopening connection after {!r}', error) - self.close(conn) - self.open(conn) - return + if isinstance(error, type): + logger.warning('Reopening connection after {!r}', error) + self.close(conn) + self.open(conn) + return with self.exception_handler(msg): return retry.retry_target( @@ -448,7 +447,6 @@ def count_error(self, error): def _is_retryable(error): """Return true for errors that are unlikely to occur again if retried.""" - for error_type in RETRYABLE_ERRORS: - if isinstance(error, error_type): - return True + if isinstance(error, RETRYABLE_ERRORS): + return True return False From d2082044380fbb7f8794948fe74a16f523bb7897 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Tue, 11 Aug 2020 15:16:15 -0600 Subject: [PATCH 249/860] Fix variable name --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 97b386f30..a64435aec 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -404,7 +404,7 @@ def _query_and_results(self, client, sql, conn, job_params, timeout=None): def _retry_and_handle(self, msg, conn, fn): """retry a function call within the context of exception_handler.""" def reopen_conn_on_error(error): - if isinstance(error, type): + if isinstance(error, REOPENABLE_ERRORS): logger.warning('Reopening connection after {!r}', error) self.close(conn) self.open(conn) From 6145d6fcf1c38b98753d714243cd38860100e5c0 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 17 Aug 2020 09:01:17 -0600 Subject: [PATCH 250/860] Add copy_job macro --- dbt/adapters/bigquery/connections.py | 35 ++++++++++++++++++- dbt/adapters/bigquery/impl.py | 5 +++ dbt/include/bigquery/macros/copy_table.sql | 31 ++++++++++++++++ .../bigquery/macros/materializations/copy.sql | 15 ++++++++ 4 files changed, 85 insertions(+), 1 deletion(-) create mode 100644 dbt/include/bigquery/macros/copy_table.sql create mode 100644 dbt/include/bigquery/macros/materializations/copy.sql diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index a64435aec..7d66c7bcb 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -26,6 +26,10 @@ BQ_QUERY_JOB_SPLIT = '-----Query Job SQL Follows-----' +# Write dispositions for bigquery. +WRITE_APPEND = google.cloud.bigquery.job.WriteDisposition.WRITE_APPEND +WRITE_TRUNCATE = google.cloud.bigquery.job.WriteDisposition.WRITE_TRUNCATE + REOPENABLE_ERRORS = ( ConnectionResetError, ConnectionError, @@ -336,7 +340,7 @@ def create_table(self, database, schema, table_name, sql): table_ref = self.table_ref(database, schema, table_name, conn) job_params = {'destination': table_ref, - 'write_disposition': 'WRITE_TRUNCATE'} + 'write_disposition': WRITE_TRUNCATE} timeout = self.get_timeout(conn) @@ -352,6 +356,35 @@ def callback(table): self.create_bigquery_table(database, schema, table_name, callback, 'CREATE DAY PARTITIONED TABLE') + def copy_bq_table(self, source, destination, materialization): + conn = self.get_thread_connection() + client = conn.handle + + if materialization is 'incremental': + write_disposition = WRITE_APPEND + else: + write_disposition = WRITE_TRUNCATE + + source = self.table_ref( + source.database, source.schema, source.table, conn) + destination = self.table_ref( + destination.database, destination.schema, destination.table, conn) + logger.debug( + 'Copying table "{}" to "{}" with disposition: "{}"', source.path, + destination.path, write_disposition) + + def copy_and_results(): + job_config = google.cloud.bigquery.CopyJobConfig( + write_disposition=write_disposition) + copy_job = client.copy_table( + source, destination, job_config=job_config) + iterator = copy_job.result(timeout=self.get_timeout(conn)) + return copy_job, iterator + self._retry_and_handle( + msg='copy table "{}" to "{}"'.format( + source.path, destination.path), conn=conn, fn=copy_and_results) + + @staticmethod def dataset(database, schema, conn): dataset_ref = conn.handle.dataset(schema, database) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 36049a46d..38b849f17 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -416,6 +416,11 @@ def _materialize_as_table( return "CREATE TABLE" + @available.parse(lambda *a, **k: '') + def copy_table(self, source, destination, materialization): + self.connections.copy_bq_table( + source, destination, materialization) + @classmethod def poll_until_job_completes(cls, job, timeout): retry_count = timeout diff --git a/dbt/include/bigquery/macros/copy_table.sql b/dbt/include/bigquery/macros/copy_table.sql new file mode 100644 index 000000000..66f95e7c6 --- /dev/null +++ b/dbt/include/bigquery/macros/copy_table.sql @@ -0,0 +1,31 @@ +{# copy_table macro allows for a table copy job to be submitted to BigQuery, which should run + # faster than `select * from`. The macro takes in a BigQueryRelation (returned by a ref() or + # source()) defining the source table for the copy, which is forwarded to the adapter. If the + # materialization is set to table, creates a table or overwrites and existing one. If incremental, + # appends to the existing table. Other materializations are not supported, and throw an error. + #} +{% macro copy_table(source) %} + + {%- if not execute: -%} + {%- set materialized_method = config.source_config['config'].get('materialized', '') -%} + {{ config(copy_materialization=materialized_method) }} + {%- if materialized_method not in ('table', 'incremental') -%} + {{ + exceptions.raise_not_implemented( + 'Copy must materialize as table or incremental, not %s' % + materialized_method) + }} + {%- endif -%} + {%- endif -%} + + {{ config(materialized='copy') }} + + {%- set destination = api.Relation.create( + database=database, schema=schema, identifier=model['alias'], type='table') -%} + + {{ + adapter.copy_table( + source, + destination, + config.get('copy_materialization')) + }} \ No newline at end of file diff --git a/dbt/include/bigquery/macros/materializations/copy.sql b/dbt/include/bigquery/macros/materializations/copy.sql new file mode 100644 index 000000000..8551ddfbb --- /dev/null +++ b/dbt/include/bigquery/macros/materializations/copy.sql @@ -0,0 +1,15 @@ +{% materialization copy, adapter='bigquery' -%} + + {# Setup #} + {{ run_hooks(pre_hooks) }} + + {# execute the macro sql #} + {{ write(sql) }} + {{ store_result(name='main', status='COPY TABLE') }} + + {# Clean up #} + {{ run_hooks(post_hooks) }} + {{ adapter.commit() }} + + { return({'relations': [context['destination']]}) } +{%- endmaterialization %} \ No newline at end of file From 56a712be55dfaa4645e7209f3491f28dd499b130 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 17 Aug 2020 11:44:51 -0600 Subject: [PATCH 251/860] move bq context to impl.py --- dbt/adapters/bigquery/connections.py | 22 +++++++------------ dbt/adapters/bigquery/impl.py | 11 +++++++++- dbt/include/bigquery/macros/copy_table.sql | 2 +- .../bigquery/macros/materializations/copy.sql | 2 +- 4 files changed, 20 insertions(+), 17 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 7d66c7bcb..9761f368e 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -26,8 +26,6 @@ BQ_QUERY_JOB_SPLIT = '-----Query Job SQL Follows-----' -# Write dispositions for bigquery. -WRITE_APPEND = google.cloud.bigquery.job.WriteDisposition.WRITE_APPEND WRITE_TRUNCATE = google.cloud.bigquery.job.WriteDisposition.WRITE_TRUNCATE REOPENABLE_ERRORS = ( @@ -356,33 +354,29 @@ def callback(table): self.create_bigquery_table(database, schema, table_name, callback, 'CREATE DAY PARTITIONED TABLE') - def copy_bq_table(self, source, destination, materialization): + def copy_bq_table(self, source, destination, write_disposition): conn = self.get_thread_connection() client = conn.handle - if materialization is 'incremental': - write_disposition = WRITE_APPEND - else: - write_disposition = WRITE_TRUNCATE - - source = self.table_ref( + source_ref = self.table_ref( source.database, source.schema, source.table, conn) - destination = self.table_ref( + destination_ref = self.table_ref( destination.database, destination.schema, destination.table, conn) + logger.debug( - 'Copying table "{}" to "{}" with disposition: "{}"', source.path, - destination.path, write_disposition) + 'Copying table "{}" to "{}" with disposition: "{}"', + source_ref.path, destination_ref.path, write_disposition) def copy_and_results(): job_config = google.cloud.bigquery.CopyJobConfig( write_disposition=write_disposition) copy_job = client.copy_table( - source, destination, job_config=job_config) + source_ref, destination_ref, job_config=job_config) iterator = copy_job.result(timeout=self.get_timeout(conn)) return copy_job, iterator self._retry_and_handle( msg='copy table "{}" to "{}"'.format( - source.path, destination.path), conn=conn, fn=copy_and_results) + source_ref.path, destination_ref.path), conn=conn, fn=copy_and_results) @staticmethod diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 38b849f17..34f246a29 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -32,6 +32,10 @@ import agate import json +# Write dispositions for bigquery. +WRITE_APPEND = google.cloud.bigquery.job.WriteDisposition.WRITE_APPEND +WRITE_TRUNCATE = google.cloud.bigquery.job.WriteDisposition.WRITE_TRUNCATE + def sql_escape(string): if not isinstance(string, str): @@ -418,8 +422,13 @@ def _materialize_as_table( @available.parse(lambda *a, **k: '') def copy_table(self, source, destination, materialization): + if materialization is 'incremental': + write_disposition = WRITE_APPEND + else: + write_disposition = WRITE_TRUNCATE + self.connections.copy_bq_table( - source, destination, materialization) + source, destination, write_disposition) @classmethod def poll_until_job_completes(cls, job, timeout): diff --git a/dbt/include/bigquery/macros/copy_table.sql b/dbt/include/bigquery/macros/copy_table.sql index 66f95e7c6..f840781e1 100644 --- a/dbt/include/bigquery/macros/copy_table.sql +++ b/dbt/include/bigquery/macros/copy_table.sql @@ -28,4 +28,4 @@ source, destination, config.get('copy_materialization')) - }} \ No newline at end of file + }} diff --git a/dbt/include/bigquery/macros/materializations/copy.sql b/dbt/include/bigquery/macros/materializations/copy.sql index 8551ddfbb..5d2207f62 100644 --- a/dbt/include/bigquery/macros/materializations/copy.sql +++ b/dbt/include/bigquery/macros/materializations/copy.sql @@ -12,4 +12,4 @@ {{ adapter.commit() }} { return({'relations': [context['destination']]}) } -{%- endmaterialization %} \ No newline at end of file +{%- endmaterialization %} From 68d8ec7f78c205396d071ec1a073f779006c70cc Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 17 Aug 2020 11:56:14 -0600 Subject: [PATCH 252/860] Reference consts through the right module --- dbt/include/bigquery/macros/copy_table.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/copy_table.sql b/dbt/include/bigquery/macros/copy_table.sql index f840781e1..f87bcf589 100644 --- a/dbt/include/bigquery/macros/copy_table.sql +++ b/dbt/include/bigquery/macros/copy_table.sql @@ -7,7 +7,7 @@ {% macro copy_table(source) %} {%- if not execute: -%} - {%- set materialized_method = config.source_config['config'].get('materialized', '') -%} + {%- set materialized_method = model.get('config', '{}').get('materialized', '') -%} {{ config(copy_materialization=materialized_method) }} {%- if materialized_method not in ('table', 'incremental') -%} {{ From 9720cbe16fd79bd581cfee63f4765d6475f1a676 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 17 Aug 2020 12:16:20 -0600 Subject: [PATCH 253/860] Attempt at integration tests --- dbt/adapters/bigquery/connections.py | 4 ++-- dbt/adapters/bigquery/impl.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 9761f368e..2ae350ade 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -376,8 +376,8 @@ def copy_and_results(): return copy_job, iterator self._retry_and_handle( msg='copy table "{}" to "{}"'.format( - source_ref.path, destination_ref.path), conn=conn, fn=copy_and_results) - + source_ref.path, destination_ref.path), conn=conn, + fn=copy_and_results) @staticmethod def dataset(database, schema, conn): diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 34f246a29..ef5ffb7f6 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -422,7 +422,7 @@ def _materialize_as_table( @available.parse(lambda *a, **k: '') def copy_table(self, source, destination, materialization): - if materialization is 'incremental': + if materialization == 'incremental': write_disposition = WRITE_APPEND else: write_disposition = WRITE_TRUNCATE From e6a27fb25c1d4d64930bc9ea4471a329d5ce60ee Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 17 Aug 2020 13:03:33 -0600 Subject: [PATCH 254/860] make flake8 happy --- dbt/adapters/bigquery/connections.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 2ae350ade..eafc83d38 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -376,8 +376,8 @@ def copy_and_results(): return copy_job, iterator self._retry_and_handle( msg='copy table "{}" to "{}"'.format( - source_ref.path, destination_ref.path), conn=conn, - fn=copy_and_results) + source_ref.path, destination_ref.path), + conn=conn, fn=copy_and_results) @staticmethod def dataset(database, schema, conn): From cdcc32683646460430f2eab37f96705478a260c1 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 17 Aug 2020 14:00:10 -0600 Subject: [PATCH 255/860] Add time_to_expiration --- dbt/adapters/bigquery/impl.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 36049a46d..ad4731c8b 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -745,6 +745,10 @@ def get_table_options( expiration = 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)' opts['expiration_timestamp'] = expiration + if (config.get('time_to_expiration') is not None) and (not temporary): + expiration = ('TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL ' + '{} hour').format(config.get('time_to_expiration')) + if config.persist_relation_docs() and 'description' in node: description = sql_escape(node['description']) opts['description'] = '"""{}"""'.format(description) From bb165e07ffb07612661890a15e3421298425c526 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 17 Aug 2020 14:10:04 -0600 Subject: [PATCH 256/860] Fix macro compilation --- dbt/include/bigquery/macros/copy_table.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbt/include/bigquery/macros/copy_table.sql b/dbt/include/bigquery/macros/copy_table.sql index f87bcf589..0dd34f8eb 100644 --- a/dbt/include/bigquery/macros/copy_table.sql +++ b/dbt/include/bigquery/macros/copy_table.sql @@ -29,3 +29,5 @@ destination, config.get('copy_materialization')) }} + +{% endmacro %} From 56c511d729d1c782b91c712637a05ab4c5d1a964 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 17 Aug 2020 14:29:02 -0600 Subject: [PATCH 257/860] Add unit test --- dbt/adapters/bigquery/impl.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index ad4731c8b..0e4b32669 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -104,6 +104,7 @@ class BigqueryConfig(AdapterConfig): labels: Optional[Dict[str, str]] = None partitions: Optional[List[str]] = None grant_access_to: Optional[List[Dict[str, str]]] = None + time_to_expiration: Optional[int] = None class BigQueryAdapter(BaseAdapter): From b228cc3c33f56705da43b3525eda68265b6505b5 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 17 Aug 2020 15:04:59 -0600 Subject: [PATCH 258/860] Add integration test --- dbt/adapters/bigquery/impl.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 0e4b32669..c46444a38 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -749,6 +749,7 @@ def get_table_options( if (config.get('time_to_expiration') is not None) and (not temporary): expiration = ('TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL ' '{} hour').format(config.get('time_to_expiration')) + opts['expiration_timestamp'] = expiration if config.persist_relation_docs() and 'description' in node: description = sql_escape(node['description']) From b2118b150dbca67dd5c65eb8ab8bab9bd71bbb30 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 17 Aug 2020 16:05:44 -0600 Subject: [PATCH 259/860] Fix table options string --- dbt/adapters/bigquery/impl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index c46444a38..c3ce90737 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -748,7 +748,7 @@ def get_table_options( if (config.get('time_to_expiration') is not None) and (not temporary): expiration = ('TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL ' - '{} hour').format(config.get('time_to_expiration')) + '{} hour)').format(config.get('time_to_expiration')) opts['expiration_timestamp'] = expiration if config.persist_relation_docs() and 'description' in node: From 2f0655f62fadcfa4132cef5e2f932c4955d7e6dd Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Mon, 17 Aug 2020 16:16:46 -0600 Subject: [PATCH 260/860] plugins --- dbt/adapters/bigquery/impl.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index c3ce90737..9dddd52bf 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -747,8 +747,9 @@ def get_table_options( opts['expiration_timestamp'] = expiration if (config.get('time_to_expiration') is not None) and (not temporary): - expiration = ('TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL ' - '{} hour)').format(config.get('time_to_expiration')) + expiration = ( + 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL ' + '{} hour)').format(config.get('time_to_expiration')) opts['expiration_timestamp'] = expiration if config.persist_relation_docs() and 'description' in node: From 7323787011086e21884878409fc5435838a8b066 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Tue, 18 Aug 2020 10:17:15 -0600 Subject: [PATCH 261/860] Make copy a proper materialization --- dbt/adapters/bigquery/impl.py | 2 ++ dbt/include/bigquery/macros/copy_table.sql | 33 ------------------- .../bigquery/macros/materializations/copy.sql | 30 ++++++++++++++--- 3 files changed, 28 insertions(+), 37 deletions(-) delete mode 100644 dbt/include/bigquery/macros/copy_table.sql diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index ef5ffb7f6..4f1f3ee4d 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -430,6 +430,8 @@ def copy_table(self, source, destination, materialization): self.connections.copy_bq_table( source, destination, write_disposition) + return "COPY TABLE" + @classmethod def poll_until_job_completes(cls, job, timeout): retry_count = timeout diff --git a/dbt/include/bigquery/macros/copy_table.sql b/dbt/include/bigquery/macros/copy_table.sql deleted file mode 100644 index 0dd34f8eb..000000000 --- a/dbt/include/bigquery/macros/copy_table.sql +++ /dev/null @@ -1,33 +0,0 @@ -{# copy_table macro allows for a table copy job to be submitted to BigQuery, which should run - # faster than `select * from`. The macro takes in a BigQueryRelation (returned by a ref() or - # source()) defining the source table for the copy, which is forwarded to the adapter. If the - # materialization is set to table, creates a table or overwrites and existing one. If incremental, - # appends to the existing table. Other materializations are not supported, and throw an error. - #} -{% macro copy_table(source) %} - - {%- if not execute: -%} - {%- set materialized_method = model.get('config', '{}').get('materialized', '') -%} - {{ config(copy_materialization=materialized_method) }} - {%- if materialized_method not in ('table', 'incremental') -%} - {{ - exceptions.raise_not_implemented( - 'Copy must materialize as table or incremental, not %s' % - materialized_method) - }} - {%- endif -%} - {%- endif -%} - - {{ config(materialized='copy') }} - - {%- set destination = api.Relation.create( - database=database, schema=schema, identifier=model['alias'], type='table') -%} - - {{ - adapter.copy_table( - source, - destination, - config.get('copy_materialization')) - }} - -{% endmacro %} diff --git a/dbt/include/bigquery/macros/materializations/copy.sql b/dbt/include/bigquery/macros/materializations/copy.sql index 5d2207f62..3de25773f 100644 --- a/dbt/include/bigquery/macros/materializations/copy.sql +++ b/dbt/include/bigquery/macros/materializations/copy.sql @@ -3,13 +3,35 @@ {# Setup #} {{ run_hooks(pre_hooks) }} - {# execute the macro sql #} - {{ write(sql) }} - {{ store_result(name='main', status='COPY TABLE') }} + {# there should be exactly one ref or exactly one source #} + {% set destination = this.incorporate(type='table') %} + + {% set dependency_type = none %} + {% if (model.refs | length) == 1 and (model.sources | length) == 0 %} + {% set dependency_type = 'ref' %} + {% elif (model.refs | length) == 0 and (model.sources | length) == 1 %} + {% set dependency_type = 'source' %} + {% else %} + {% set msg %} + Expected exactly one ref or exactly one source, instead got {{ model.refs | length }} models and {{ model.sources | length }} sources. + {% endset %} + {% do exceptions.raise_compiler_error(msg) %} + {% endif %} + + {% if dependency_type == 'ref' %} + {% set src = ref(*model.refs[0]) %} + {% else %} + {% set src = source(*model.sources[0]) %} + {% endif %} + + {{ adapter.copy_table( + src, + destination, + config.get('copy_materialization')) }} {# Clean up #} {{ run_hooks(post_hooks) }} {{ adapter.commit() }} - { return({'relations': [context['destination']]}) } + {{ return({'relations': [destination]}) }} {%- endmaterialization %} From 326ef2340887215e6508e781f5a4c0b3cefb4545 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Tue, 18 Aug 2020 10:46:04 -0600 Subject: [PATCH 262/860] Return from copy_bq_table and fix test --- dbt/adapters/bigquery/connections.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index eafc83d38..4d1fe285a 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -374,7 +374,8 @@ def copy_and_results(): source_ref, destination_ref, job_config=job_config) iterator = copy_job.result(timeout=self.get_timeout(conn)) return copy_job, iterator - self._retry_and_handle( + + return self._retry_and_handle( msg='copy table "{}" to "{}"'.format( source_ref.path, destination_ref.path), conn=conn, fn=copy_and_results) From 222ec8b16e675de457b86cd422f854ac1c44c769 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Tue, 18 Aug 2020 13:08:58 -0600 Subject: [PATCH 263/860] Set status --- dbt/adapters/bigquery/connections.py | 2 +- dbt/adapters/bigquery/impl.py | 2 +- dbt/include/bigquery/macros/materializations/copy.sql | 6 ++++-- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 4d1fe285a..5fb2857a0 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -375,7 +375,7 @@ def copy_and_results(): iterator = copy_job.result(timeout=self.get_timeout(conn)) return copy_job, iterator - return self._retry_and_handle( + self._retry_and_handle( msg='copy table "{}" to "{}"'.format( source_ref.path, destination_ref.path), conn=conn, fn=copy_and_results) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 4f1f3ee4d..23cae7624 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -430,7 +430,7 @@ def copy_table(self, source, destination, materialization): self.connections.copy_bq_table( source, destination, write_disposition) - return "COPY TABLE" + return "COPY TABLE with materialization: {}".format(materialization) @classmethod def poll_until_job_completes(cls, job, timeout): diff --git a/dbt/include/bigquery/macros/materializations/copy.sql b/dbt/include/bigquery/macros/materializations/copy.sql index 3de25773f..2a2de7677 100644 --- a/dbt/include/bigquery/macros/materializations/copy.sql +++ b/dbt/include/bigquery/macros/materializations/copy.sql @@ -24,10 +24,12 @@ {% set src = source(*model.sources[0]) %} {% endif %} - {{ adapter.copy_table( + {%- set result_str = adapter.copy_table( src, destination, - config.get('copy_materialization')) }} + config.get('copy_materialization')) -%} + + {{ store_result('main', status=result_str) }} {# Clean up #} {{ run_hooks(post_hooks) }} From 8eb75528339ec485e5bc9188564380f0ac2a2b83 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Tue, 18 Aug 2020 13:41:00 -0600 Subject: [PATCH 264/860] Raise error for bad materializations and set a default --- dbt/adapters/bigquery/impl.py | 7 ++++++- dbt/include/bigquery/macros/materializations/copy.sql | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 23cae7624..70dc408ab 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -424,8 +424,13 @@ def _materialize_as_table( def copy_table(self, source, destination, materialization): if materialization == 'incremental': write_disposition = WRITE_APPEND - else: + elif materialization == 'table': write_disposition = WRITE_TRUNCATE + else: + dbt.exceptions.raise_compiler_error( + 'Copy table materialization must be "copy" or "table", but ' + f"config.get('copy_materialization', 'table') was " + f'{materialization}') self.connections.copy_bq_table( source, destination, write_disposition) diff --git a/dbt/include/bigquery/macros/materializations/copy.sql b/dbt/include/bigquery/macros/materializations/copy.sql index 2a2de7677..7b28975ef 100644 --- a/dbt/include/bigquery/macros/materializations/copy.sql +++ b/dbt/include/bigquery/macros/materializations/copy.sql @@ -27,7 +27,7 @@ {%- set result_str = adapter.copy_table( src, destination, - config.get('copy_materialization')) -%} + config.get('copy_materialization', 'table')) -%} {{ store_result('main', status=result_str) }} From 1b218f8fb7f5fda14aad0997a9fd019142feb910 Mon Sep 17 00:00:00 2001 From: Kurt Convey Date: Wed, 19 Aug 2020 10:08:52 -0600 Subject: [PATCH 265/860] hours_to_expiration --- dbt/adapters/bigquery/impl.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 9dddd52bf..cbd4bd6b6 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -104,7 +104,7 @@ class BigqueryConfig(AdapterConfig): labels: Optional[Dict[str, str]] = None partitions: Optional[List[str]] = None grant_access_to: Optional[List[Dict[str, str]]] = None - time_to_expiration: Optional[int] = None + hours_to_expiration: Optional[int] = None class BigQueryAdapter(BaseAdapter): @@ -746,10 +746,10 @@ def get_table_options( expiration = 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)' opts['expiration_timestamp'] = expiration - if (config.get('time_to_expiration') is not None) and (not temporary): + if (config.get('hours_to_expiration') is not None) and (not temporary): expiration = ( 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL ' - '{} hour)').format(config.get('time_to_expiration')) + '{} hour)').format(config.get('hours_to_expiration')) opts['expiration_timestamp'] = expiration if config.persist_relation_docs() and 'description' in node: From f79a115868739d490f4c5a2f7c83855783fc1952 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Wed, 19 Aug 2020 20:10:33 +0000 Subject: [PATCH 266/860] Release dbt v0.18.0rc1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 33d6ab18e..04748d3b5 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.18.0b2' +version = '0.18.0rc1' diff --git a/setup.py b/setup.py index ed01fd9b4..5ffa04f5f 100644 --- a/setup.py +++ b/setup.py @@ -14,7 +14,7 @@ package_name = "dbt-bigquery" -package_version = "0.18.0b2" +package_version = "0.18.0rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From b8374220741a9ff9a1e7e28debc48aa8ccf4160e Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 21 Aug 2020 07:54:57 -0600 Subject: [PATCH 267/860] add more test helper methods --- dbt/adapters/bigquery/impl.py | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 3d6bea532..eea5300fe 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -831,3 +831,20 @@ def get_rows_different_sql( column_names=column_names, except_operator=except_operator, ) + + def timestamp_add_sql( + self, add_to: str, number: int = 1, interval: str = 'hour' + ) -> str: + return f'timestamp_add({add_to}, interval {number} {interval})' + + def string_add_sql( + self, add_to: str, value: str, location='append', + ) -> str: + if location == 'append': + return f"concat({add_to}, '{value}')" + elif location == 'prepend': + return f"concat('{value}', {add_to})" + else: + raise dbt.exceptions.RuntimeException( + f'Got an unexpected location value of "{location}"' + ) From 07b704dd3ce3e49521a38612e210a49d85493cd1 Mon Sep 17 00:00:00 2001 From: genos Date: Sun, 23 Aug 2020 17:13:36 -0400 Subject: [PATCH 268/860] Fix for #2347 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit **Introduction** This PR attempts to fix #2347, wherein we wish `dbt` to complain about trying to install with a Python version < 3.6. **Changes** Per [the issue's suggestion](https://github.com/fishtown-analytics/dbt/issues/2347), I found every `setup.py` file I could: ``` -# If you have the fantastic `fd` utility installed: fd setup.py -# This also works find . -name setup.py -print ``` Then to each of these, I added the following after the `import sys`: ``` if sys.version_info < (3, 6): print('Error: dbt does not support this version of Python.') print('Please upgrade to Python 3.6 or higher.') sys.exit(1) ``` **Testing** I used the [`nix` package manager](https://nixos.org) to attempt installing this branch with both Python 2.7 and Python 3.8. _Python 2.7_ fails as expected: ``` ~/github/test2 ∃ cat default.nix let pkgs = import (fetchTarball "https://github.com/NixOS/nixpkgs/archive/20.03.tar.gz") { }; py = pkgs.python27Full.withPackages (p: [ p.setuptools ]); in pkgs.mkShell { name = "python-2-env"; buildInputs = [ py ]; } ~/github/test2 ∃ nix-shell --pure [nix-shell:~/github/test2]$ python ../dbt/setup.py build Error: dbt does not support this version of Python. Please upgrade to Python 3.6 or higher. [nix-shell:~/github/test2]$ echo $? 1 ``` _Python 3.8_ still works: ``` ~/github/test3 ∃ cat default.nix let pkgs = import (fetchTarball "https://github.com/NixOS/nixpkgs/archive/20.03.tar.gz") { }; py = pkgs.python38Full.withPackages (p: [ p.setuptools ]); in pkgs.mkShell { name = "python-3-env"; buildInputs = [ py ]; } ~/github/test3 ∃ nix-shell --pure [nix-shell:~/github/test3]$ python ../dbt/setup.py build running build [nix-shell:~/github/test3]$ echo $? 0 ``` --- setup.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/setup.py b/setup.py index 5ffa04f5f..b0f429cac 100644 --- a/setup.py +++ b/setup.py @@ -2,6 +2,12 @@ import os import sys +if sys.version_info < (3, 6): + print('Error: dbt does not support this version of Python.') + print('Please upgrade to Python 3.6 or higher.') + sys.exit(1) + + from setuptools import setup try: from setuptools import find_namespace_packages From 22efed2a59bff7b332a2bc8a8365cf280a58bf18 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Thu, 3 Sep 2020 15:49:09 +0000 Subject: [PATCH 269/860] Release dbt v0.18.0rc2 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 04748d3b5..f06e88e49 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.18.0rc1' +version = '0.18.0rc2' diff --git a/setup.py b/setup.py index b0f429cac..bff962623 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.18.0rc1" +package_version = "0.18.0rc2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 3cda55e7367ff9649b6bc82d3cfc3e398ccd045d Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Thu, 3 Sep 2020 16:02:36 +0000 Subject: [PATCH 270/860] Release dbt v0.18.0 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index f06e88e49..ec1bb8e9f 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.18.0rc2' +version = '0.18.0' diff --git a/setup.py b/setup.py index bff962623..4324d8982 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.18.0rc2" +package_version = "0.18.0" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 1f0604b61db1c97f54ac26dc68cad4e1761269a9 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Fri, 11 Sep 2020 14:21:03 -0600 Subject: [PATCH 271/860] bumpversion for 0.18.1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index ec1bb8e9f..3e662f036 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.18.0' +version = '0.18.1a1' diff --git a/setup.py b/setup.py index 4324d8982..a8ef1ccb9 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.18.0" +package_version = "0.18.1a1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 6fead0b9727f51057b3d0416fed63933702e2074 Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Wed, 16 Sep 2020 10:34:53 -0600 Subject: [PATCH 272/860] =?UTF-8?q?Bump=20version:=200.18.0=20=E2=86=92=20?= =?UTF-8?q?0.19.0a1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index ec1bb8e9f..7291f3ecd 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.18.0' +version = '0.19.0a1' diff --git a/setup.py b/setup.py index 4324d8982..54aad2b38 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.18.0" +package_version = "0.19.0a1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 7b1264815c40bfe10112da05051177d5812c7189 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Thu, 17 Sep 2020 18:54:44 +0000 Subject: [PATCH 273/860] Release dbt v0.18.1b1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 3e662f036..81b3034e7 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.18.1a1' +version = '0.18.1b1' diff --git a/setup.py b/setup.py index a8ef1ccb9..29c0c4005 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.18.1a1" +package_version = "0.18.1b1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From a86262a0c64cfa51e2875fb7033974f96c33d40a Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Tue, 22 Sep 2020 14:09:51 +0000 Subject: [PATCH 274/860] Release dbt v0.18.1b2 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 81b3034e7..27ea9715f 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.18.1b1' +version = '0.18.1b2' diff --git a/setup.py b/setup.py index 29c0c4005..3e01e344f 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.18.1b1" +package_version = "0.18.1b2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 7aa6a688c580b3e991ab5ce28a815e4c01403be1 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Fri, 25 Sep 2020 20:05:31 +0000 Subject: [PATCH 275/860] Release dbt v0.18.1b3 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 27ea9715f..0cde55ada 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.18.1b2' +version = '0.18.1b3' diff --git a/setup.py b/setup.py index 3e01e344f..dfdff2c5c 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.18.1b2" +package_version = "0.18.1b3" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From e9769ed6c5a4c1c672dc9325372188a607b7161e Mon Sep 17 00:00:00 2001 From: "Jared Champion (SYD)" Date: Wed, 30 Sep 2020 10:57:03 +1000 Subject: [PATCH 276/860] REBASED on dev/0.18.1; moved CHANGELOG entries --- dbt/adapters/bigquery/connections.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 5fb2857a0..bc6fca9c7 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -477,4 +477,7 @@ def _is_retryable(error): """Return true for errors that are unlikely to occur again if retried.""" if isinstance(error, RETRYABLE_ERRORS): return True + elif isinstance(error, google.api_core.exceptions.Forbidden) and any( + e['reason'] == 'rateLimitExceeded' for e in error.errors): + return True return False From 13543833a49fb948029e3e755e3c7fae06a8811e Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Wed, 30 Sep 2020 22:38:02 -0400 Subject: [PATCH 277/860] (#2344) Support BigQuery OAuth using a refesh token and client secrets --- dbt/adapters/bigquery/connections.py | 36 +++++++++++++++++++++++----- 1 file changed, 30 insertions(+), 6 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 5fb2857a0..336fc0648 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -9,7 +9,10 @@ import google.cloud.exceptions from google.api_core import retry, client_info from google.auth import impersonated_credentials -from google.oauth2 import service_account +from google.oauth2 import ( + credentials as GoogleCredentials, + service_account as GoogleServiceAccountCredentials +) from dbt.utils import format_bytes, format_rows_number from dbt.clients import agate_helper, gcloud @@ -50,19 +53,29 @@ class BigQueryConnectionMethod(StrEnum): OAUTH = 'oauth' SERVICE_ACCOUNT = 'service-account' SERVICE_ACCOUNT_JSON = 'service-account-json' + BEARER = 'bearer' @dataclass class BigQueryCredentials(Credentials): method: BigQueryConnectionMethod - keyfile: Optional[str] = None - keyfile_json: Optional[Dict[str, Any]] = None timeout_seconds: Optional[int] = 300 location: Optional[str] = None priority: Optional[Priority] = None retries: Optional[int] = 1 maximum_bytes_billed: Optional[int] = None impersonate_service_account: Optional[str] = None + + # Keyfile json creds + keyfile: Optional[str] = None + keyfile_json: Optional[Dict[str, Any]] = None + + # Bearer token creds + refresh_token: Optional[str] = None + client_id: Optional[str] = None + client_secret: Optional[str] = None + token_uri: Optional[str] = None + _ALIASES = { 'project': 'database', 'dataset': 'schema', @@ -110,12 +123,13 @@ def exception_handler(self, sql): message = "Access denied while running query" self.handle_error(e, message) - except google.auth.exceptions.RefreshError: + except google.auth.exceptions.RefreshError as e: message = "Unable to generate access token, if you're using " \ "impersonate_service_account, make sure your " \ 'initial account has the "roles/' \ 'iam.serviceAccountTokenCreator" role on the ' \ - 'account you are trying to impersonate.' + 'account you are trying to impersonate.\n\n' \ + f'{str(e)}' raise RuntimeException(message) except Exception as e: @@ -151,7 +165,7 @@ def commit(self): @classmethod def get_bigquery_credentials(cls, profile_credentials): method = profile_credentials.method - creds = service_account.Credentials + creds = GoogleServiceAccountCredentials.Credentials if method == BigQueryConnectionMethod.OAUTH: credentials, project_id = google.auth.default(scopes=cls.SCOPE) @@ -165,6 +179,16 @@ def get_bigquery_credentials(cls, profile_credentials): details = profile_credentials.keyfile_json return creds.from_service_account_info(details, scopes=cls.SCOPE) + elif method == BigQueryConnectionMethod.BEARER: + return GoogleCredentials.Credentials( + token=None, + refresh_token=profile_credentials.refresh_token, + client_id=profile_credentials.client_id, + client_secret=profile_credentials.client_secret, + token_uri=profile_credentials.token_uri, + scopes=cls.SCOPE + ) + error = ('Invalid `method` in profile: "{}"'.format(method)) raise FailedToConnectException(error) From b85c853e4e726e1a2648992ec01a7ce8db7b5047 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Thu, 1 Oct 2020 16:39:50 +0000 Subject: [PATCH 278/860] Release dbt v0.18.1rc1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 0cde55ada..464004f5f 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.18.1b3' +version = '0.18.1rc1' diff --git a/setup.py b/setup.py index dfdff2c5c..c0f4026b0 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.18.1b3" +package_version = "0.18.1rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 64485134196eb13a421859f37cbaae573f3b4431 Mon Sep 17 00:00:00 2001 From: Mark Scannell Date: Sat, 3 Oct 2020 13:01:25 +0100 Subject: [PATCH 279/860] Added dbt_invocation_id to BigQuery jobs --- dbt/adapters/bigquery/connections.py | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 5fb2857a0..cd50ca163 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -13,6 +13,7 @@ from dbt.utils import format_bytes, format_rows_number from dbt.clients import agate_helper, gcloud +from dbt.tracking import active_user from dbt.contracts.connection import ConnectionState from dbt.exceptions import ( FailedToConnectException, RuntimeException, DatabaseException @@ -249,7 +250,12 @@ def raw_execute(self, sql, fetch=False): logger.debug('On {}: {}', conn.name, sql) - job_params = {'use_legacy_sql': False} + job_params = { + 'use_legacy_sql': False, + 'labels': { + 'dbt_invocation_id': active_user.invocation_id, + }, + } priority = conn.credentials.priority if priority == Priority.Batch: From 1c331c7c3d1e957d0c658fe501aa23db3a253669 Mon Sep 17 00:00:00 2001 From: Mark Scannell Date: Sat, 3 Oct 2020 15:32:53 +0100 Subject: [PATCH 280/860] Only set invocation_id if tracking is enabled --- dbt/adapters/bigquery/connections.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index cd50ca163..c4b09eb2b 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -252,11 +252,13 @@ def raw_execute(self, sql, fetch=False): job_params = { 'use_legacy_sql': False, - 'labels': { - 'dbt_invocation_id': active_user.invocation_id, - }, } + if active_user: + job_params['labels'] = { + 'dbt_invocation_id': active_user.invocation_id + } + priority = conn.credentials.priority if priority == Priority.Batch: job_params['priority'] = google.cloud.bigquery.QueryPriority.BATCH From 22f5745e87f4f5fd45de04fb3fb2aa46facb6921 Mon Sep 17 00:00:00 2001 From: Drew Banin Date: Wed, 7 Oct 2020 15:34:57 -0400 Subject: [PATCH 281/860] support providing a token directly; update method name --- dbt/adapters/bigquery/connections.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 336fc0648..a13dbdd87 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -53,7 +53,7 @@ class BigQueryConnectionMethod(StrEnum): OAUTH = 'oauth' SERVICE_ACCOUNT = 'service-account' SERVICE_ACCOUNT_JSON = 'service-account-json' - BEARER = 'bearer' + OAUTH_SECRETS = 'oauth-secrets' @dataclass @@ -70,7 +70,8 @@ class BigQueryCredentials(Credentials): keyfile: Optional[str] = None keyfile_json: Optional[Dict[str, Any]] = None - # Bearer token creds + # oauth-secrets + token: Optional[str] = None refresh_token: Optional[str] = None client_id: Optional[str] = None client_secret: Optional[str] = None @@ -179,9 +180,9 @@ def get_bigquery_credentials(cls, profile_credentials): details = profile_credentials.keyfile_json return creds.from_service_account_info(details, scopes=cls.SCOPE) - elif method == BigQueryConnectionMethod.BEARER: + elif method == BigQueryConnectionMethod.OAUTH_SECRETS: return GoogleCredentials.Credentials( - token=None, + token=profile_credentials.token, refresh_token=profile_credentials.refresh_token, client_id=profile_credentials.client_id, client_secret=profile_credentials.client_secret, From 9d435f78b022d24b961b231ae8e278ac327c9dca Mon Sep 17 00:00:00 2001 From: Ran Ever-Hadani Date: Sat, 10 Oct 2020 17:44:07 -0700 Subject: [PATCH 282/860] Make partition metadata available to BigQuery users (rebased to dev/kiyoshi-kuromiya) --- dbt/adapters/bigquery/connections.py | 44 ++++++++++++++++++++++++++++ dbt/include/bigquery/macros/etc.sql | 8 +++++ 2 files changed, 52 insertions(+) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index efb2704fc..050c20adc 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -341,6 +341,50 @@ def execute(self, sql, auto_begin=False, fetch=None): return status, res + # The following method intentionaly violates DRY, in that it is mostly + # copy-pasted from raw_execute(). This is done in order to discorage + # use of legacySQL queries in DBT, except to obtain partition metadata. + # the method would be removed when partition metadata becomes available + # from standardSQL. + def _raw_execute_legacy_sql(self, sql, fetch=False): + conn = self.get_thread_connection() + client = conn.handle + + logger.debug('On {}: {}', conn.name, sql) + + job_params = {'use_legacy_sql': True} + + priority = conn.credentials.priority + if priority == Priority.Batch: + job_params['priority'] = google.cloud.bigquery.QueryPriority.BATCH + else: + job_params[ + 'priority'] = google.cloud.bigquery.QueryPriority.INTERACTIVE + + maximum_bytes_billed = conn.credentials.maximum_bytes_billed + if maximum_bytes_billed is not None and maximum_bytes_billed != 0: + job_params['maximum_bytes_billed'] = maximum_bytes_billed + + def fn(): + return self._query_and_results(client, sql, conn, job_params) + + query_job, iterator = self._retry_and_handle(msg=sql, conn=conn, fn=fn) + + return query_job, iterator + + def get_partitions_metadata(self, table_id): + def standard_to_legacy(table_id): + table_ref = google.cloud.bigquery.table.TableReference.from_string(table_id) + return (table_ref.project + ':' + table_ref.dataset_id + '.' + table_ref.table_id).replace('`','') + + legacy_sql = 'SELECT * FROM [' + standard_to_legacy(table_id) + '$__PARTITIONS_SUMMARY__]' + + sql = self._add_query_comment(legacy_sql) + # auto_begin is ignored on bigquery, and only included for consistency + _, iterator = self._raw_execute_legacy_sql(sql, fetch='fetch_result') + + return self.get_table_from_response(iterator) + def create_bigquery_table(self, database, schema, table_name, callback, sql): """Create a bigquery table. The caller must supply a callback diff --git a/dbt/include/bigquery/macros/etc.sql b/dbt/include/bigquery/macros/etc.sql index a10ad1a5b..2091133e8 100644 --- a/dbt/include/bigquery/macros/etc.sql +++ b/dbt/include/bigquery/macros/etc.sql @@ -5,3 +5,11 @@ {% macro grant_access_to(entity, entity_type, role, grant_target_dict) -%} {% do adapter.grant_access_to(entity, entity_type, role, grant_target_dict) %} {% endmacro %} + +{%- macro get_partitions_metadata(table) -%} + {%- if execute -%} + {%- set res = adapter.get_partitions_metadata(table) -%} + {{- return(res) -}} + {%- endif -%} + {{- return(None) -}} +{%- endmacro -%} \ No newline at end of file From 26a485edf3fafb4d350c35ff5ba15edc853da512 Mon Sep 17 00:00:00 2001 From: Ran Ever-Hadani Date: Sun, 11 Oct 2020 11:03:53 -0700 Subject: [PATCH 283/860] Accommodate first round of comments --- dbt/adapters/bigquery/connections.py | 50 +++++----------------------- dbt/include/bigquery/macros/etc.sql | 2 +- 2 files changed, 9 insertions(+), 43 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 050c20adc..e605c222a 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -269,15 +269,13 @@ def get_table_from_response(cls, resp): column_names = [field.name for field in resp.schema] return agate_helper.table_from_data_flat(resp, column_names) - def raw_execute(self, sql, fetch=False): + def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False): conn = self.get_thread_connection() client = conn.handle logger.debug('On {}: {}', conn.name, sql) - job_params = { - 'use_legacy_sql': False, - } + job_params = {'use_legacy_sql': use_legacy_sql} if active_user: job_params['labels'] = { @@ -341,48 +339,16 @@ def execute(self, sql, auto_begin=False, fetch=None): return status, res - # The following method intentionaly violates DRY, in that it is mostly - # copy-pasted from raw_execute(). This is done in order to discorage - # use of legacySQL queries in DBT, except to obtain partition metadata. - # the method would be removed when partition metadata becomes available - # from standardSQL. - def _raw_execute_legacy_sql(self, sql, fetch=False): - conn = self.get_thread_connection() - client = conn.handle - - logger.debug('On {}: {}', conn.name, sql) - - job_params = {'use_legacy_sql': True} - - priority = conn.credentials.priority - if priority == Priority.Batch: - job_params['priority'] = google.cloud.bigquery.QueryPriority.BATCH - else: - job_params[ - 'priority'] = google.cloud.bigquery.QueryPriority.INTERACTIVE - - maximum_bytes_billed = conn.credentials.maximum_bytes_billed - if maximum_bytes_billed is not None and maximum_bytes_billed != 0: - job_params['maximum_bytes_billed'] = maximum_bytes_billed - - def fn(): - return self._query_and_results(client, sql, conn, job_params) - - query_job, iterator = self._retry_and_handle(msg=sql, conn=conn, fn=fn) - - return query_job, iterator - - def get_partitions_metadata(self, table_id): - def standard_to_legacy(table_id): - table_ref = google.cloud.bigquery.table.TableReference.from_string(table_id) - return (table_ref.project + ':' + table_ref.dataset_id + '.' + table_ref.table_id).replace('`','') + def get_partitions_metadata(self, table): + def standard_to_legacy(table): + # table_ref = google.cloud.bigquery.table.TableReference.from_string(table) + return (table.project + ':' + table.dataset + '.' + table.identifier).replace('`','') - legacy_sql = 'SELECT * FROM [' + standard_to_legacy(table_id) + '$__PARTITIONS_SUMMARY__]' + legacy_sql = 'SELECT * FROM [' + standard_to_legacy(table) + '$__PARTITIONS_SUMMARY__]' sql = self._add_query_comment(legacy_sql) # auto_begin is ignored on bigquery, and only included for consistency - _, iterator = self._raw_execute_legacy_sql(sql, fetch='fetch_result') - + _, iterator = self.raw_execute(sql, fetch='fetch_result', use_legacy_sql=True) return self.get_table_from_response(iterator) def create_bigquery_table(self, database, schema, table_name, callback, diff --git a/dbt/include/bigquery/macros/etc.sql b/dbt/include/bigquery/macros/etc.sql index 2091133e8..59b61473e 100644 --- a/dbt/include/bigquery/macros/etc.sql +++ b/dbt/include/bigquery/macros/etc.sql @@ -12,4 +12,4 @@ {{- return(res) -}} {%- endif -%} {{- return(None) -}} -{%- endmacro -%} \ No newline at end of file +{%- endmacro -%} From 37a86644b525c67d610270b7a0a70139263a62e0 Mon Sep 17 00:00:00 2001 From: Ran Ever-Hadani Date: Sun, 11 Oct 2020 11:37:51 -0700 Subject: [PATCH 284/860] Eliminate pep8 errors --- dbt/adapters/bigquery/connections.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index e605c222a..0b96a73c3 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -341,14 +341,15 @@ def execute(self, sql, auto_begin=False, fetch=None): def get_partitions_metadata(self, table): def standard_to_legacy(table): - # table_ref = google.cloud.bigquery.table.TableReference.from_string(table) - return (table.project + ':' + table.dataset + '.' + table.identifier).replace('`','') - - legacy_sql = 'SELECT * FROM [' + standard_to_legacy(table) + '$__PARTITIONS_SUMMARY__]' + return table.project + ':' + table.dataset + '.' + table.identifier + + legacy_sql = 'SELECT * FROM ['\ + + standard_to_legacy(table) + '$__PARTITIONS_SUMMARY__]' sql = self._add_query_comment(legacy_sql) # auto_begin is ignored on bigquery, and only included for consistency - _, iterator = self.raw_execute(sql, fetch='fetch_result', use_legacy_sql=True) + _, iterator =\ + self.raw_execute(sql, fetch='fetch_result', use_legacy_sql=True) return self.get_table_from_response(iterator) def create_bigquery_table(self, database, schema, table_name, callback, From bbb65fa7b79b204de4c1e4b1765d5ecb12fc56b8 Mon Sep 17 00:00:00 2001 From: Ran Ever-Hadani Date: Sun, 11 Oct 2020 12:06:56 -0700 Subject: [PATCH 285/860] Eliminate asterisk from raw_execute to try an fix integration error --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 0b96a73c3..cf13aaab1 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -269,7 +269,7 @@ def get_table_from_response(cls, resp): column_names = [field.name for field in resp.schema] return agate_helper.table_from_data_flat(resp, column_names) - def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False): + def raw_execute(self, sql, fetch=False, use_legacy_sql=False): conn = self.get_thread_connection() client = conn.handle From 5452283fbc5bc2f8555a95576233dc4f3997a3f7 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Tue, 13 Oct 2020 21:31:54 +0000 Subject: [PATCH 286/860] Release dbt v0.18.1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 464004f5f..f3bd5fd8e 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.18.1rc1' +version = '0.18.1' diff --git a/setup.py b/setup.py index c0f4026b0..274dc4d27 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.18.1rc1" +package_version = "0.18.1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 970d7218ef784f33f6a136adf2f5d0ca53da4fc6 Mon Sep 17 00:00:00 2001 From: Gerda Shank Date: Thu, 15 Oct 2020 11:40:38 -0400 Subject: [PATCH 287/860] Remove injected_sql. Store non-ephemeral injected_sql in compiled_sql --- dbt/adapters/bigquery/impl.py | 4 ++-- dbt/include/bigquery/macros/materializations/table.sql | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index eea5300fe..364209a79 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -385,7 +385,7 @@ def _materialize_as_view(self, model: Dict[str, Any]) -> str: model_database = model.get('database') model_schema = model.get('schema') model_alias = model.get('alias') - model_sql = model.get('injected_sql') + model_sql = model.get('compiled_sql') logger.debug("Model SQL ({}):\n{}".format(model_alias, model_sql)) self.connections.create_view( @@ -505,7 +505,7 @@ def execute_model(self, model, materialization, sql_override=None, decorator=None): if sql_override is None: - sql_override = model.get('injected_sql') + sql_override = model.get('compiled_sql') if flags.STRICT_MODE: connection = self.connections.get_thread_connection() diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql index cbb97ea4e..53c7d2f66 100644 --- a/dbt/include/bigquery/macros/materializations/table.sql +++ b/dbt/include/bigquery/macros/materializations/table.sql @@ -11,7 +11,7 @@ {{ log(table_start_time ~ ' | -> Running for day ' ~ date, info=True) }} {% endif %} - {% set fixed_sql = model['injected_sql'] | replace('[DBT__PARTITION_DATE]', date) %} + {% set fixed_sql = model['compiled_sql'] | replace('[DBT__PARTITION_DATE]', date) %} {% set _ = adapter.execute_model(model, 'table', fixed_sql, decorator=date) %} {% endfor %} From d346f2e0d3a6906a0b60550e0b68cf99ac97e0c0 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Wed, 21 Oct 2020 00:35:44 +0000 Subject: [PATCH 288/860] Release dbt v0.19.0b1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 7291f3ecd..9fde2e4c6 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.19.0a1' +version = '0.19.0b1' diff --git a/setup.py b/setup.py index 54aad2b38..f84f57382 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.19.0a1" +package_version = "0.19.0b1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 6616136358bb8dc10a5fbfd56519ea7013136c0b Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Wed, 21 Oct 2020 19:00:10 -0400 Subject: [PATCH 289/860] Add tests for get_partitions_metadata (#1) * Add tests using get_partitions_metadata * Readd asterisk to raw_execute --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index cf13aaab1..0b96a73c3 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -269,7 +269,7 @@ def get_table_from_response(cls, resp): column_names = [field.name for field in resp.schema] return agate_helper.table_from_data_flat(resp, column_names) - def raw_execute(self, sql, fetch=False, use_legacy_sql=False): + def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False): conn = self.get_thread_connection() client = conn.handle From ec63bf0522f17dc2521ca25723fc0b3bf44c7980 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Mon, 9 Nov 2020 17:18:42 -0800 Subject: [PATCH 290/860] Unpin google-cloud dependencies --- setup.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/setup.py b/setup.py index f84f57382..33c54aeba 100644 --- a/setup.py +++ b/setup.py @@ -48,10 +48,10 @@ install_requires=[ 'dbt-core=={}'.format(package_version), 'protobuf>=3.6.0,<3.12', - 'google-cloud-core>=1.3.0,<1.4', - 'google-cloud-bigquery>=1.25.0,<1.26.0', - 'google-api-core>=1.16.0,<1.17.0', - 'googleapis-common-protos>=1.6.0,<1.7.0', + 'google-cloud-core>=1.3.0,<2', + 'google-cloud-bigquery>=1.25.0,<2', + 'google-api-core>=1.16.0,<2', + 'googleapis-common-protos>=1.6.0,<2', 'six>=1.14.0', ], zip_safe=False, From 2bffd4096f8e802aef867b80726670d6884451ec Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Mon, 9 Nov 2020 17:36:41 -0800 Subject: [PATCH 291/860] Bump protobuf too --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 33c54aeba..58afa6961 100644 --- a/setup.py +++ b/setup.py @@ -47,7 +47,7 @@ }, install_requires=[ 'dbt-core=={}'.format(package_version), - 'protobuf>=3.6.0,<3.12', + 'protobuf>=3.13.0,<4', 'google-cloud-core>=1.3.0,<2', 'google-cloud-bigquery>=1.25.0,<2', 'google-api-core>=1.16.0,<2', From 91aaac8521318f2b02d25839437645dece5f82c8 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Tue, 10 Nov 2020 23:51:15 -0800 Subject: [PATCH 292/860] google-cloud-bigquery goes to 3 --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 58afa6961..38fae2eb1 100644 --- a/setup.py +++ b/setup.py @@ -49,7 +49,7 @@ 'dbt-core=={}'.format(package_version), 'protobuf>=3.13.0,<4', 'google-cloud-core>=1.3.0,<2', - 'google-cloud-bigquery>=1.25.0,<2', + 'google-cloud-bigquery>=1.25.0,<3', 'google-api-core>=1.16.0,<2', 'googleapis-common-protos>=1.6.0,<2', 'six>=1.14.0', From 0ae7cc6b2b6c75d0166144b791d81fc2a1f14fc7 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Wed, 11 Nov 2020 13:48:41 -0800 Subject: [PATCH 293/860] _ --- setup.py | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/setup.py b/setup.py index 38fae2eb1..ed91f729d 100644 --- a/setup.py +++ b/setup.py @@ -48,10 +48,12 @@ install_requires=[ 'dbt-core=={}'.format(package_version), 'protobuf>=3.13.0,<4', - 'google-cloud-core>=1.3.0,<2', - 'google-cloud-bigquery>=1.25.0,<3', - 'google-api-core>=1.16.0,<2', - 'googleapis-common-protos>=1.6.0,<2', + # These are more tightly pinned, as they have a track record of + # breaking changes in minor releases. + 'google-cloud-core>=1.3.0,<1.5', + 'google-cloud-bigquery>=1.25.0,<2.4', + 'google-api-core>=1.16.0,<1.24', + 'googleapis-common-protos>=1.6.0,<1.53', 'six>=1.14.0', ], zip_safe=False, From 4eb1127cbcda99ad65c3d07c6c8f46c1dc441cb5 Mon Sep 17 00:00:00 2001 From: Magnus Fagertun Date: Thu, 19 Nov 2020 13:47:42 +0100 Subject: [PATCH 294/860] Added hour, year, month partitioning BQ --- dbt/adapters/bigquery/impl.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 364209a79..c53c0f0d7 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -50,15 +50,20 @@ def sql_escape(string): class PartitionConfig(JsonSchemaMixin): field: str data_type: str = 'date' + granularity: str = 'DAY' range: Optional[Dict[str, Any]] = None def render(self, alias: Optional[str] = None): column: str = self.field if alias: column = f'{alias}.{self.field}' - - if self.data_type in ('timestamp', 'datetime'): - return f'date({column})' + logger.debug(f'Render and field"{self.field}"') + logger.debug(f'Granularity is "{self.granularity}"') + if self.data_type == 'timestamp': + logger.debug(f'timestamp_trunc({column},{self.granularity}') + return f'timestamp_trunc({column},{self.granularity})' + elif self.data_type == 'datetime': + return f'datetime_trunc({column},{self.granularity})' else: return column From 07f53d29e17b018b365956342fae595355d53ef5 Mon Sep 17 00:00:00 2001 From: Magnus Fagertun Date: Thu, 19 Nov 2020 17:42:27 +0100 Subject: [PATCH 295/860] more BQ partitioning --- dbt/adapters/bigquery/impl.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index c53c0f0d7..8bbce18ef 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -57,13 +57,11 @@ def render(self, alias: Optional[str] = None): column: str = self.field if alias: column = f'{alias}.{self.field}' - logger.debug(f'Render and field"{self.field}"') - logger.debug(f'Granularity is "{self.granularity}"') + if self.data_type == 'timestamp': - logger.debug(f'timestamp_trunc({column},{self.granularity}') - return f'timestamp_trunc({column},{self.granularity})' + return f'timestamp_trunc({column}, {self.granularity})' elif self.data_type == 'datetime': - return f'datetime_trunc({column},{self.granularity})' + return f'datetime_trunc({column}, {self.granularity})' else: return column From d5b47326ffda0c525a8b04481fd117fab0c107f0 Mon Sep 17 00:00:00 2001 From: Magnus Fagertun Date: Thu, 19 Nov 2020 18:51:05 +0100 Subject: [PATCH 296/860] moving granularity to render, not to break tests --- dbt/adapters/bigquery/impl.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 8bbce18ef..9386dd6e5 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -50,7 +50,6 @@ def sql_escape(string): class PartitionConfig(JsonSchemaMixin): field: str data_type: str = 'date' - granularity: str = 'DAY' range: Optional[Dict[str, Any]] = None def render(self, alias: Optional[str] = None): @@ -58,6 +57,10 @@ def render(self, alias: Optional[str] = None): if alias: column = f'{alias}.{self.field}' + granularity: str = 'DAY' + if self.granularity: + granularity = self.granularity + if self.data_type == 'timestamp': return f'timestamp_trunc({column}, {self.granularity})' elif self.data_type == 'datetime': From b67f8128d1651df7eb3a54dd69ef86870e9c38f6 Mon Sep 17 00:00:00 2001 From: Magnus Fagertun Date: Fri, 20 Nov 2020 00:10:15 +0100 Subject: [PATCH 297/860] added tests for datetime and timestamp --- dbt/adapters/bigquery/impl.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 9386dd6e5..8bbce18ef 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -50,6 +50,7 @@ def sql_escape(string): class PartitionConfig(JsonSchemaMixin): field: str data_type: str = 'date' + granularity: str = 'DAY' range: Optional[Dict[str, Any]] = None def render(self, alias: Optional[str] = None): @@ -57,10 +58,6 @@ def render(self, alias: Optional[str] = None): if alias: column = f'{alias}.{self.field}' - granularity: str = 'DAY' - if self.granularity: - granularity = self.granularity - if self.data_type == 'timestamp': return f'timestamp_trunc({column}, {self.granularity})' elif self.data_type == 'datetime': From b539df0ee1110c6f1c2410b54a8f0465d3e46f6e Mon Sep 17 00:00:00 2001 From: Magnus Fagertun Date: Sat, 21 Nov 2020 00:24:20 +0100 Subject: [PATCH 298/860] added month,year to date partitioning, granularity comparison to _partitions_match --- dbt/adapters/bigquery/impl.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 8bbce18ef..84c18eb9e 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -62,6 +62,8 @@ def render(self, alias: Optional[str] = None): return f'timestamp_trunc({column}, {self.granularity})' elif self.data_type == 'datetime': return f'datetime_trunc({column}, {self.granularity})' + elif self.data_type == 'date' and self.granularity in ('MONTH','YEAR'): + return f'date_trunc({column}, {self.granularity})' else: return column @@ -550,7 +552,9 @@ def _partitions_match( return True elif conf_partition and table.time_partitioning is not None: table_field = table.time_partitioning.field - return table_field == conf_partition.field + table_granularity = table.partitioning.type + return table_field == conf_partition.field \ + and table_granularity == conf_partition.granularity elif conf_partition and table.range_partitioning is not None: dest_part = table.range_partitioning conf_part = conf_partition.range or {} From 011ecf68f927b12fe976444895304e64a202443b Mon Sep 17 00:00:00 2001 From: Magnus Fagertun Date: Sat, 21 Nov 2020 00:40:27 +0100 Subject: [PATCH 299/860] typo in _partitions_match --- dbt/adapters/bigquery/impl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 84c18eb9e..0b17054b9 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -552,7 +552,7 @@ def _partitions_match( return True elif conf_partition and table.time_partitioning is not None: table_field = table.time_partitioning.field - table_granularity = table.partitioning.type + table_granularity = table.partitioning_type return table_field == conf_partition.field \ and table_granularity == conf_partition.granularity elif conf_partition and table.range_partitioning is not None: From d8d6b229b3eb5fdd459362d09b707a9115ec1fed Mon Sep 17 00:00:00 2001 From: Magnus Fagertun Date: Sat, 21 Nov 2020 01:21:07 +0100 Subject: [PATCH 300/860] uppercase and lowercase for date partitions supported --- dbt/adapters/bigquery/impl.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 0b17054b9..53e788841 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -62,7 +62,8 @@ def render(self, alias: Optional[str] = None): return f'timestamp_trunc({column}, {self.granularity})' elif self.data_type == 'datetime': return f'datetime_trunc({column}, {self.granularity})' - elif self.data_type == 'date' and self.granularity in ('MONTH','YEAR'): + elif self.data_type == 'date' and \ + self.granularity in ('MONTH','YEAR','month','year'): return f'date_trunc({column}, {self.granularity})' else: return column From b6e579c531d7bb9e9bf602dce6e0b1cbb6e48081 Mon Sep 17 00:00:00 2001 From: Magnus Fagertun Date: Sun, 22 Nov 2020 10:00:10 +0100 Subject: [PATCH 301/860] whitespace cleanup --- dbt/adapters/bigquery/impl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 53e788841..2bfaa788d 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -63,7 +63,7 @@ def render(self, alias: Optional[str] = None): elif self.data_type == 'datetime': return f'datetime_trunc({column}, {self.granularity})' elif self.data_type == 'date' and \ - self.granularity in ('MONTH','YEAR','month','year'): + self.granularity in ('MONTH', 'YEAR', 'month', 'year'): return f'date_trunc({column}, {self.granularity})' else: return column From 174519495415b6da13b69e53bd642177cfb00ac8 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Mon, 23 Nov 2020 15:34:43 -0800 Subject: [PATCH 302/860] Remove duplicate raise --- dbt/adapters/bigquery/connections.py | 1 - 1 file changed, 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 738935d3e..95b4615b5 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -238,7 +238,6 @@ def open(cls, connection): handle = cls.get_bigquery_client(connection.credentials) except Exception as e: - raise logger.debug("Got an error when attempting to create a bigquery " "client: '{}'".format(e)) From 09e39791e56bb26bf30c893fcb5fac585ac1aa17 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Mon, 23 Nov 2020 16:55:37 -0800 Subject: [PATCH 303/860] Allow BigQuery to default on project name --- dbt/adapters/bigquery/connections.py | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 738935d3e..8ed98ac02 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -1,5 +1,6 @@ from contextlib import contextmanager from dataclasses import dataclass +from functools import lru_cache from requests.exceptions import ConnectionError from typing import Optional, Any, Dict @@ -78,6 +79,10 @@ class BigQueryCredentials(Credentials): client_secret: Optional[str] = None token_uri: Optional[str] = None + # BigQuery can evaluate this from the environment with most authentication + # mechanisms. + database: Optional[str] = None + _ALIASES = { 'project': 'database', 'dataset': 'schema', @@ -170,7 +175,7 @@ def get_bigquery_credentials(cls, profile_credentials): creds = GoogleServiceAccountCredentials.Credentials if method == BigQueryConnectionMethod.OAUTH: - credentials, project_id = google.auth.default(scopes=cls.SCOPE) + credentials, _ = cls.get_bigquery_defaults() return credentials elif method == BigQueryConnectionMethod.SERVICE_ACCOUNT: @@ -194,6 +199,13 @@ def get_bigquery_credentials(cls, profile_credentials): error = ('Invalid `method` in profile: "{}"'.format(method)) raise FailedToConnectException(error) + @classmethod + @lru_cache() + def get_bigquery_defaults(cls): + """ Returns (credentials, project_id) """ + # Cached, because the underlying implementation shells out. + return google.auth.default(scopes=cls.SCOPE) + @classmethod def get_impersonated_bigquery_credentials(cls, profile_credentials): source_credentials = cls.get_bigquery_credentials(profile_credentials) @@ -211,7 +223,11 @@ def get_bigquery_client(cls, profile_credentials): cls.get_impersonated_bigquery_credentials(profile_credentials) else: creds = cls.get_bigquery_credentials(profile_credentials) + # `database` is an alias of `project` in BigQuery database = profile_credentials.database + if database is None: + _, database = cls.get_bigquery_defaults() + location = getattr(profile_credentials, 'location', None) info = client_info.ClientInfo(user_agent=f'dbt-{dbt_version}') From 055b4ee3d4c770b01188d2d74bbffe454ca82bd9 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Mon, 23 Nov 2020 17:32:41 -0800 Subject: [PATCH 304/860] Definet database exclusively in contracts/connection.py --- dbt/adapters/bigquery/connections.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 8ed98ac02..4d9763956 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -79,10 +79,6 @@ class BigQueryCredentials(Credentials): client_secret: Optional[str] = None token_uri: Optional[str] = None - # BigQuery can evaluate this from the environment with most authentication - # mechanisms. - database: Optional[str] = None - _ALIASES = { 'project': 'database', 'dataset': 'schema', From ff343f2c84d3d934b88db83ae45878fb124beeb5 Mon Sep 17 00:00:00 2001 From: Magnus Fagertun Date: Wed, 25 Nov 2020 21:17:07 +0100 Subject: [PATCH 305/860] Update plugins/bigquery/dbt/adapters/bigquery/impl.py Co-authored-by: Jeremy Cohen --- dbt/adapters/bigquery/impl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 2bfaa788d..69d7474cb 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -50,7 +50,7 @@ def sql_escape(string): class PartitionConfig(JsonSchemaMixin): field: str data_type: str = 'date' - granularity: str = 'DAY' + granularity: str = 'day' range: Optional[Dict[str, Any]] = None def render(self, alias: Optional[str] = None): From 020cfce1a759f76b63c25352dc26ec80cd388e0e Mon Sep 17 00:00:00 2001 From: Magnus Fagertun Date: Wed, 25 Nov 2020 21:41:51 +0100 Subject: [PATCH 306/860] added test and enhancements from jtcohen6 --- dbt/adapters/bigquery/impl.py | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 69d7474cb..402ec7891 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -58,15 +58,11 @@ def render(self, alias: Optional[str] = None): if alias: column = f'{alias}.{self.field}' - if self.data_type == 'timestamp': - return f'timestamp_trunc({column}, {self.granularity})' - elif self.data_type == 'datetime': - return f'datetime_trunc({column}, {self.granularity})' - elif self.data_type == 'date' and \ - self.granularity in ('MONTH', 'YEAR', 'month', 'year'): - return f'date_trunc({column}, {self.granularity})' - else: + if self.data_type.lower() == 'date' and \ + self.granularity.lower() == 'day': return column + else: + return f'{self.data_type}_trunc({column}, {self.granularity})' @classmethod def parse(cls, raw_partition_by) -> Optional['PartitionConfig']: From b10ef3bb7e2d1ef0ad767e562a0609fb82cf7608 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Mon, 30 Nov 2020 16:54:21 -0800 Subject: [PATCH 307/860] _ --- dbt/adapters/bigquery/connections.py | 23 +++++++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 4d9763956..bbe611c78 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -92,6 +92,25 @@ def _connection_keys(self): return ('method', 'database', 'schema', 'location', 'priority', 'timeout_seconds', 'maximum_bytes_billed') + def __post_init__(self): + # We need to inject the correct value of the database (aka project) at + # this stage, ref + # https://github.com/fishtown-analytics/dbt/pull/2908#discussion_r532927436. + + # `database` is an alias of `project` in BigQuery + if self.database is None: + _, database = self.get_bigquery_defaults() + self.database = database + + @classmethod + @lru_cache() + def get_bigquery_defaults(cls): + """ Returns (credentials, project_id) """ + # This method is copied from ` BigQueryConnectionManager`, because it's + # required in both classes. + # We could move this & the scopes to the module level. + return google.auth.default() + class BigQueryConnectionManager(BaseConnectionManager): TYPE = 'bigquery' @@ -219,11 +238,7 @@ def get_bigquery_client(cls, profile_credentials): cls.get_impersonated_bigquery_credentials(profile_credentials) else: creds = cls.get_bigquery_credentials(profile_credentials) - # `database` is an alias of `project` in BigQuery database = profile_credentials.database - if database is None: - _, database = cls.get_bigquery_defaults() - location = getattr(profile_credentials, 'location', None) info = client_info.ClientInfo(user_agent=f'dbt-{dbt_version}') From 609d68088c915393fa3f459b2bcd38b6446e88b1 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Mon, 30 Nov 2020 19:30:07 -0800 Subject: [PATCH 308/860] _ --- dbt/adapters/bigquery/connections.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index bbe611c78..32020c406 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -61,6 +61,9 @@ class BigQueryConnectionMethod(StrEnum): @dataclass class BigQueryCredentials(Credentials): method: BigQueryConnectionMethod + # BigQuery allows an empty database / project, where it defers to the + # environment for the project + database: Optional[str] timeout_seconds: Optional[int] = 300 location: Optional[str] = None priority: Optional[Priority] = None From 2829695b6aa7418fcffc8612dc5b580e37b770c8 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Tue, 1 Dec 2020 16:19:20 -0800 Subject: [PATCH 309/860] Move method to module func --- dbt/adapters/bigquery/connections.py | 30 +++++++++++----------------- 1 file changed, 12 insertions(+), 18 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 32020c406..3eb4d9ba4 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -2,7 +2,7 @@ from dataclasses import dataclass from functools import lru_cache from requests.exceptions import ConnectionError -from typing import Optional, Any, Dict +from typing import Optional, Any, Dict, Tuple import google.auth import google.auth.exceptions @@ -45,6 +45,16 @@ ConnectionError, ) +@lru_cache() +def get_bigquery_defaults() -> Tuple[Any, Optional[str]]: + """ + Returns (credentials, project_id) + + project_id where available from the environment + """ + # Cached, because the underlying implementation shells out, taking ~1s + return google.auth.default() + class Priority(StrEnum): Interactive = 'interactive' @@ -105,15 +115,6 @@ def __post_init__(self): _, database = self.get_bigquery_defaults() self.database = database - @classmethod - @lru_cache() - def get_bigquery_defaults(cls): - """ Returns (credentials, project_id) """ - # This method is copied from ` BigQueryConnectionManager`, because it's - # required in both classes. - # We could move this & the scopes to the module level. - return google.auth.default() - class BigQueryConnectionManager(BaseConnectionManager): TYPE = 'bigquery' @@ -193,7 +194,7 @@ def get_bigquery_credentials(cls, profile_credentials): creds = GoogleServiceAccountCredentials.Credentials if method == BigQueryConnectionMethod.OAUTH: - credentials, _ = cls.get_bigquery_defaults() + credentials, _ = get_bigquery_defaults() return credentials elif method == BigQueryConnectionMethod.SERVICE_ACCOUNT: @@ -217,13 +218,6 @@ def get_bigquery_credentials(cls, profile_credentials): error = ('Invalid `method` in profile: "{}"'.format(method)) raise FailedToConnectException(error) - @classmethod - @lru_cache() - def get_bigquery_defaults(cls): - """ Returns (credentials, project_id) """ - # Cached, because the underlying implementation shells out. - return google.auth.default(scopes=cls.SCOPE) - @classmethod def get_impersonated_bigquery_credentials(cls, profile_credentials): source_credentials = cls.get_bigquery_credentials(profile_credentials) From c6cbb3239bcbed4e1e06b9d316b20a3f60ec88e9 Mon Sep 17 00:00:00 2001 From: Maximilian Roos <5635139+max-sixty@users.noreply.github.com> Date: Wed, 2 Dec 2020 10:41:20 -0800 Subject: [PATCH 310/860] Update plugins/bigquery/dbt/adapters/bigquery/connections.py Co-authored-by: Kyle Wigley --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 3eb4d9ba4..a7c941754 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -112,7 +112,7 @@ def __post_init__(self): # `database` is an alias of `project` in BigQuery if self.database is None: - _, database = self.get_bigquery_defaults() + _, database = get_bigquery_defaults() self.database = database From b735855519d74e955777555d2d269b7b827a0bf0 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Wed, 2 Dec 2020 11:15:28 -0800 Subject: [PATCH 311/860] Formatting --- dbt/adapters/bigquery/connections.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index a7c941754..743718869 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -45,12 +45,13 @@ ConnectionError, ) + @lru_cache() def get_bigquery_defaults() -> Tuple[Any, Optional[str]]: - """ + """ Returns (credentials, project_id) - - project_id where available from the environment + + project_id is returned available from the environment; otherwise None """ # Cached, because the underlying implementation shells out, taking ~1s return google.auth.default() From 9e0ab72dc3165956a59d6c27fc1a1d84f75fa66f Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Thu, 10 Dec 2020 13:22:06 -0500 Subject: [PATCH 312/860] address issue with py39 --- setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/setup.py b/setup.py index ed91f729d..377a30d1c 100644 --- a/setup.py +++ b/setup.py @@ -69,6 +69,7 @@ 'Programming Language :: Python :: 3.6', 'Programming Language :: Python :: 3.7', 'Programming Language :: Python :: 3.8', + 'Programming Language :: Python :: 3.9', ], python_requires=">=3.6.2", ) From 539b604881bba16f0938dbb0235cb1c9b3c27b49 Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Wed, 16 Dec 2020 11:24:31 -0500 Subject: [PATCH 313/860] first pass at adding query stats, naming tbd --- dbt/adapters/bigquery/connections.py | 55 ++++++++++++++++++++-------- 1 file changed, 39 insertions(+), 16 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 67fdd7e7c..c52084eb2 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -1,6 +1,7 @@ from contextlib import contextmanager from dataclasses import dataclass from functools import lru_cache +import agate from requests.exceptions import ConnectionError from typing import Optional, Any, Dict, Tuple @@ -18,7 +19,7 @@ from dbt.utils import format_bytes, format_rows_number from dbt.clients import agate_helper, gcloud from dbt.tracking import active_user -from dbt.contracts.connection import ConnectionState +from dbt.contracts.connection import ConnectionState, ExecutionStatus from dbt.exceptions import ( FailedToConnectException, RuntimeException, DatabaseException ) @@ -69,6 +70,11 @@ class BigQueryConnectionMethod(StrEnum): OAUTH_SECRETS = 'oauth-secrets' +@dataclass +class BigQueryExecutionStatus(ExecutionStatus): + bytes: Optional[str] = None + + @dataclass class BigQueryCredentials(Credentials): method: BigQueryConnectionMethod @@ -324,7 +330,9 @@ def fn(): return query_job, iterator - def execute(self, sql, auto_begin=False, fetch=None): + def execute( + self, sql, auto_begin=False, fetch=None + ) -> Tuple[BigQueryExecutionStatus, agate.Table]: sql = self._add_query_comment(sql) # auto_begin is ignored on bigquery, and only included for consistency query_job, iterator = self.raw_execute(sql, fetch=fetch) @@ -334,33 +342,48 @@ def execute(self, sql, auto_begin=False, fetch=None): else: res = agate_helper.empty_table() + message = 'OK' + state = None + num_rows = None + processed_bytes = None + if query_job.statement_type == 'CREATE_VIEW': - status = 'CREATE VIEW' + state = 'CREATE VIEW' elif query_job.statement_type == 'CREATE_TABLE_AS_SELECT': conn = self.get_thread_connection() client = conn.handle table = client.get_table(query_job.destination) - processed = format_bytes(query_job.total_bytes_processed) - status = 'CREATE TABLE ({} rows, {} processed)'.format( - format_rows_number(table.num_rows), - format_bytes(query_job.total_bytes_processed), + state = 'CREATE TABLE' + num_rows = format_rows_number(table.num_rows) + processed_bytes = format_bytes(query_job.total_bytes_processed) + message = '{} ({} rows, {} processed)'.format( + state, + num_rows, + processed_bytes ) elif query_job.statement_type == 'SCRIPT': - processed = format_bytes(query_job.total_bytes_processed) - status = f'SCRIPT ({processed} processed)' + state = 'SCRIPT' + processed_bytes = format_bytes(query_job.total_bytes_processed) + message = f'{state} ({processed_bytes} processed)' elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: - status = '{} ({} rows, {} processed)'.format( - query_job.statement_type, - format_rows_number(query_job.num_dml_affected_rows), - format_bytes(query_job.total_bytes_processed), + state = query_job.statement_type + num_rows = format_rows_number(query_job.num_dml_affected_rows) + processed_bytes = format_bytes(query_job.total_bytes_processed) + message = '{} ({} rows, {} processed)'.format( + state, + num_rows, + processed_bytes, ) - else: - status = 'OK' - + status = BigQueryExecutionStatus( + message=message, + rows=num_rows, + state=state, + bytes=processed_bytes + ) return status, res def get_partitions_metadata(self, table): From 65605f431bc0c1c89c3b8c0e9a0ea80ce408e894 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Thu, 3 Dec 2020 13:49:03 +0900 Subject: [PATCH 314/860] Support require_partition_filter and partition_expiration_days in BQ --- dbt/adapters/bigquery/impl.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 402ec7891..02b31adf9 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -788,6 +788,12 @@ def get_table_options( labels = config.get('labels', {}) opts['labels'] = list(labels.items()) + if config.get('partition_by_required'): + opts['partition_by_required'] = config.get('partition_by_required', False) + + if config.get('partition_expiration_days'): + opts['partition_expiration_days'] = config.get('partition_expiration_days', None) + return opts @available.parse_none From 85ffe72db565b05cbd77e941afa6188df6a0a4b8 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Thu, 3 Dec 2020 14:03:08 +0900 Subject: [PATCH 315/860] Format code --- dbt/adapters/bigquery/impl.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 02b31adf9..b90506c4d 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -789,10 +789,12 @@ def get_table_options( opts['labels'] = list(labels.items()) if config.get('partition_by_required'): - opts['partition_by_required'] = config.get('partition_by_required', False) + opts['partition_by_required'] = config.get( + 'partition_by_required', False) if config.get('partition_expiration_days'): - opts['partition_expiration_days'] = config.get('partition_expiration_days', None) + opts['partition_expiration_days'] = config.get( + 'partition_expiration_days', None) return opts From 75799406819fafd1d20e370f827e46a4a31de85b Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Thu, 3 Dec 2020 20:30:26 +0900 Subject: [PATCH 316/860] Update --- dbt/adapters/bigquery/impl.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index b90506c4d..32ebe1c31 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -789,12 +789,11 @@ def get_table_options( opts['labels'] = list(labels.items()) if config.get('partition_by_required'): - opts['partition_by_required'] = config.get( - 'partition_by_required', False) + opts['partition_by_required'] = config.get('partition_by_required') - if config.get('partition_expiration_days'): + if config.get('partition_expiration_days') is not None: opts['partition_expiration_days'] = config.get( - 'partition_expiration_days', None) + 'partition_expiration_days') return opts From 4b857e6f03df059adbcc8a7000be9f101c544077 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Thu, 3 Dec 2020 21:24:46 +0900 Subject: [PATCH 317/860] Update --- dbt/adapters/bigquery/impl.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 32ebe1c31..ed806bb1f 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -111,6 +111,8 @@ class BigqueryConfig(AdapterConfig): partitions: Optional[List[str]] = None grant_access_to: Optional[List[Dict[str, str]]] = None hours_to_expiration: Optional[int] = None + require_partition_filter: Optional[bool] = None + partition_expiration_days: Optional[int] = None class BigQueryAdapter(BaseAdapter): @@ -788,8 +790,8 @@ def get_table_options( labels = config.get('labels', {}) opts['labels'] = list(labels.items()) - if config.get('partition_by_required'): - opts['partition_by_required'] = config.get('partition_by_required') + if config.get('require_partition_filter'): + opts['require_partition_filter'] = config.get('require_partition_filter') if config.get('partition_expiration_days') is not None: opts['partition_expiration_days'] = config.get( From ad2601dfbc9ed7aa73be1c459bdc4805f34665a1 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Thu, 3 Dec 2020 21:53:40 +0900 Subject: [PATCH 318/860] Format --- dbt/adapters/bigquery/impl.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index ed806bb1f..72d56b9b5 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -791,7 +791,8 @@ def get_table_options( opts['labels'] = list(labels.items()) if config.get('require_partition_filter'): - opts['require_partition_filter'] = config.get('require_partition_filter') + opts['require_partition_filter'] = config.get( + 'require_partition_filter') if config.get('partition_expiration_days') is not None: opts['partition_expiration_days'] = config.get( From 1344d1fff23a212ded744dcd4d73f6c9b0bfac01 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Fri, 18 Dec 2020 11:04:44 +0900 Subject: [PATCH 319/860] Add a condition for require_partition_filter --- dbt/include/bigquery/macros/materializations/incremental.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index fb6cccb25..79dd112b0 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -54,6 +54,7 @@ set _dbt_max_partition = ( select max({{ partition_by.field }}) from {{ this }} + where {{ partition_by.field }} is not null ); -- 1. create a temp table From fddae50cae08a364a413ad1028dfb5932809ffcd Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Mon, 21 Dec 2020 13:24:09 -0500 Subject: [PATCH 320/860] update naming --- dbt/adapters/bigquery/connections.py | 61 ++++++++++--------- .../bigquery/macros/materializations/copy.sql | 2 +- .../macros/materializations/table.sql | 2 +- 3 files changed, 33 insertions(+), 32 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index c52084eb2..a06943fc5 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -19,7 +19,7 @@ from dbt.utils import format_bytes, format_rows_number from dbt.clients import agate_helper, gcloud from dbt.tracking import active_user -from dbt.contracts.connection import ConnectionState, ExecutionStatus +from dbt.contracts.connection import ConnectionState, AdapterResponse from dbt.exceptions import ( FailedToConnectException, RuntimeException, DatabaseException ) @@ -71,8 +71,8 @@ class BigQueryConnectionMethod(StrEnum): @dataclass -class BigQueryExecutionStatus(ExecutionStatus): - bytes: Optional[str] = None +class BigQueryAdapterResponse(AdapterResponse): + bytes_processed: Optional[int] = None @dataclass @@ -332,59 +332,60 @@ def fn(): def execute( self, sql, auto_begin=False, fetch=None - ) -> Tuple[BigQueryExecutionStatus, agate.Table]: + ) -> Tuple[BigQueryAdapterResponse, agate.Table]: sql = self._add_query_comment(sql) # auto_begin is ignored on bigquery, and only included for consistency query_job, iterator = self.raw_execute(sql, fetch=fetch) if fetch: - res = self.get_table_from_response(iterator) + table = self.get_table_from_response(iterator) else: - res = agate_helper.empty_table() + table = agate_helper.empty_table() message = 'OK' - state = None + code = None num_rows = None - processed_bytes = None + bytes_processed = None if query_job.statement_type == 'CREATE_VIEW': - state = 'CREATE VIEW' + code = 'CREATE VIEW' elif query_job.statement_type == 'CREATE_TABLE_AS_SELECT': conn = self.get_thread_connection() client = conn.handle - table = client.get_table(query_job.destination) - state = 'CREATE TABLE' - num_rows = format_rows_number(table.num_rows) - processed_bytes = format_bytes(query_job.total_bytes_processed) + query_table = client.get_table(query_job.destination) + code = 'CREATE TABLE' + num_rows = query_table.num_rows + bytes_processed = query_job.total_bytes_processed message = '{} ({} rows, {} processed)'.format( - state, - num_rows, - processed_bytes + code, + format_rows_number(num_rows), + format_bytes(bytes_processed) ) elif query_job.statement_type == 'SCRIPT': - state = 'SCRIPT' - processed_bytes = format_bytes(query_job.total_bytes_processed) - message = f'{state} ({processed_bytes} processed)' + code = 'SCRIPT' + bytes_processed = query_job.total_bytes_processed + message = f'{code} ({format_bytes(bytes_processed)} processed)' elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']: - state = query_job.statement_type - num_rows = format_rows_number(query_job.num_dml_affected_rows) - processed_bytes = format_bytes(query_job.total_bytes_processed) + code = query_job.statement_type + num_rows = query_job.num_dml_affected_rows + bytes_processed = query_job.total_bytes_processed message = '{} ({} rows, {} processed)'.format( - state, - num_rows, - processed_bytes, + code, + format_rows_number(num_rows), + format_bytes(bytes_processed), ) - status = BigQueryExecutionStatus( + response = BigQueryAdapterResponse( message=message, - rows=num_rows, - state=state, - bytes=processed_bytes + rows_affected=num_rows, + code=code, + bytes_processed=bytes_processed ) - return status, res + + return response, table def get_partitions_metadata(self, table): def standard_to_legacy(table): diff --git a/dbt/include/bigquery/macros/materializations/copy.sql b/dbt/include/bigquery/macros/materializations/copy.sql index 7b28975ef..0f24b50bc 100644 --- a/dbt/include/bigquery/macros/materializations/copy.sql +++ b/dbt/include/bigquery/macros/materializations/copy.sql @@ -29,7 +29,7 @@ destination, config.get('copy_materialization', 'table')) -%} - {{ store_result('main', status=result_str) }} + {{ store_result('main', response=result_str) }} {# Clean up #} {{ run_hooks(post_hooks) }} diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql index 53c7d2f66..bbfe54771 100644 --- a/dbt/include/bigquery/macros/materializations/table.sql +++ b/dbt/include/bigquery/macros/materializations/table.sql @@ -22,7 +22,7 @@ {% set result_str = 'CREATED ' ~ num_days ~ ' PARTITIONS' %} {% endif %} - {{ store_result('main', status=result_str) }} + {{ store_result('main', response=result_str) }} {% endmacro %} From b9d4ed7e626e7a7d035443f8303620241550e93b Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Tue, 22 Dec 2020 12:13:37 -0500 Subject: [PATCH 321/860] update api, fix tests, add placeholder for test/source results --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index a06943fc5..da3fc7d99 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -379,7 +379,7 @@ def execute( ) response = BigQueryAdapterResponse( - message=message, + _message=message, rows_affected=num_rows, code=code, bytes_processed=bytes_processed From 5d2ea1e5fe1301d747268dde6ec4321f6da3a9e1 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Tue, 29 Dec 2020 22:52:34 +0000 Subject: [PATCH 322/860] Release dbt v0.19.0rc1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 9fde2e4c6..3cdf668b4 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.19.0b1' +version = '0.19.0rc1' diff --git a/setup.py b/setup.py index 377a30d1c..67688e0d5 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.19.0b1" +package_version = "0.19.0rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From f02e4b85c3830aa59d9c4db84997269360f3c96d Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Thu, 14 Jan 2021 17:21:25 +0000 Subject: [PATCH 323/860] Release dbt v0.19.0rc2 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 3cdf668b4..23e02eb1c 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.19.0rc1' +version = '0.19.0rc2' diff --git a/setup.py b/setup.py index 67688e0d5..1400e3e0f 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.19.0rc1" +package_version = "0.19.0rc2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From cdb363f364c61c32dc5867999058f2e90e6bfd69 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Wed, 27 Jan 2021 16:07:41 +0000 Subject: [PATCH 324/860] Release dbt v0.19.0rc3 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 23e02eb1c..3ffc8be96 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.19.0rc2' +version = '0.19.0rc3' diff --git a/setup.py b/setup.py index 1400e3e0f..cb02750dd 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.19.0rc2" +package_version = "0.19.0rc3" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 9c0869e73c9b39958ca9676b84d7b955d0ee8acf Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Wed, 27 Jan 2021 17:39:48 +0000 Subject: [PATCH 325/860] Release dbt v0.19.0 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 3ffc8be96..f8bf85e8d 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.19.0rc3' +version = '0.19.0' diff --git a/setup.py b/setup.py index cb02750dd..634f658d3 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.19.0rc3" +package_version = "0.19.0" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 38a2681dd65d78f62bd2107c5ba61040922b3a24 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Fri, 29 Jan 2021 18:09:05 +0900 Subject: [PATCH 326/860] Pass the default scopes to the default BigQuery credentials --- dbt/adapters/bigquery/connections.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index da3fc7d99..9868d50ab 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -48,14 +48,14 @@ @lru_cache() -def get_bigquery_defaults() -> Tuple[Any, Optional[str]]: +def get_bigquery_defaults(scopes=None) -> Tuple[Any, Optional[str]]: """ Returns (credentials, project_id) project_id is returned available from the environment; otherwise None """ # Cached, because the underlying implementation shells out, taking ~1s - return google.auth.default() + return google.auth.default(scopes=scopes) class Priority(StrEnum): @@ -201,7 +201,7 @@ def get_bigquery_credentials(cls, profile_credentials): creds = GoogleServiceAccountCredentials.Credentials if method == BigQueryConnectionMethod.OAUTH: - credentials, _ = get_bigquery_defaults() + credentials, _ = get_bigquery_defaults(scopes=cls.SCOPE) return credentials elif method == BigQueryConnectionMethod.SERVICE_ACCOUNT: From 44d8757184b9c1cfcf846122391c3bf7df3fbc28 Mon Sep 17 00:00:00 2001 From: Gerda Shank Date: Fri, 20 Nov 2020 13:40:07 -0500 Subject: [PATCH 327/860] Performance fixes, including supporting libyaml, caching mapped_fields in the classes for 'from_dict', removing deepcopy on fqn_search, separating validation from 'from_dict', and special handling for dbt internal not_null and unique tests. Use TestMacroNamespace instead of original in order to limit the number of macros in the context. Integrate mashumaro into dbt to improve performance of 'from_dict' and 'to_dict' --- dbt/adapters/bigquery/connections.py | 2 +- dbt/adapters/bigquery/impl.py | 8 +++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index da3fc7d99..d660e75dd 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -27,7 +27,7 @@ from dbt.logger import GLOBAL_LOGGER as logger from dbt.version import __version__ as dbt_version -from hologram.helpers import StrEnum +from dbt.dataclass_schema import StrEnum BQ_QUERY_JOB_SPLIT = '-----Query Job SQL Follows-----' diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 72d56b9b5..d45dc2868 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -1,6 +1,6 @@ from dataclasses import dataclass from typing import Dict, List, Optional, Any, Set, Union -from hologram import JsonSchemaMixin, ValidationError +from dbt.dataclass_schema import dbtClassMixin, ValidationError import dbt.deprecations import dbt.exceptions @@ -47,7 +47,7 @@ def sql_escape(string): @dataclass -class PartitionConfig(JsonSchemaMixin): +class PartitionConfig(dbtClassMixin): field: str data_type: str = 'date' granularity: str = 'day' @@ -69,6 +69,7 @@ def parse(cls, raw_partition_by) -> Optional['PartitionConfig']: if raw_partition_by is None: return None try: + cls.validate(raw_partition_by) return cls.from_dict(raw_partition_by) except ValidationError as exc: msg = dbt.exceptions.validator_error_message(exc) @@ -84,7 +85,7 @@ def parse(cls, raw_partition_by) -> Optional['PartitionConfig']: @dataclass -class GrantTarget(JsonSchemaMixin): +class GrantTarget(dbtClassMixin): dataset: str project: str @@ -808,6 +809,7 @@ def grant_access_to(self, entity, entity_type, role, grant_target_dict): conn = self.connections.get_thread_connection() client = conn.handle + GrantTarget.validate(grant_target_dict) grant_target = GrantTarget.from_dict(grant_target_dict) dataset = client.get_dataset( self.connections.dataset_from_id(grant_target.render()) From 58a5bbad007ea44afaef0fbc6db76eb41f80d625 Mon Sep 17 00:00:00 2001 From: Gerda Shank Date: Fri, 20 Nov 2020 13:40:07 -0500 Subject: [PATCH 328/860] Performance fixes, including supporting libyaml, caching mapped_fields in the classes for 'from_dict', removing deepcopy on fqn_search, separating validation from 'from_dict', and special handling for dbt internal not_null and unique tests. Use TestMacroNamespace instead of original in order to limit the number of macros in the context. Integrate mashumaro into dbt to improve performance of 'from_dict' and 'to_dict' --- dbt/adapters/bigquery/connections.py | 2 +- dbt/adapters/bigquery/impl.py | 8 +++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index da3fc7d99..d660e75dd 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -27,7 +27,7 @@ from dbt.logger import GLOBAL_LOGGER as logger from dbt.version import __version__ as dbt_version -from hologram.helpers import StrEnum +from dbt.dataclass_schema import StrEnum BQ_QUERY_JOB_SPLIT = '-----Query Job SQL Follows-----' diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 402ec7891..d752a473c 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -1,6 +1,6 @@ from dataclasses import dataclass from typing import Dict, List, Optional, Any, Set, Union -from hologram import JsonSchemaMixin, ValidationError +from dbt.dataclass_schema import dbtClassMixin, ValidationError import dbt.deprecations import dbt.exceptions @@ -47,7 +47,7 @@ def sql_escape(string): @dataclass -class PartitionConfig(JsonSchemaMixin): +class PartitionConfig(dbtClassMixin): field: str data_type: str = 'date' granularity: str = 'day' @@ -69,6 +69,7 @@ def parse(cls, raw_partition_by) -> Optional['PartitionConfig']: if raw_partition_by is None: return None try: + cls.validate(raw_partition_by) return cls.from_dict(raw_partition_by) except ValidationError as exc: msg = dbt.exceptions.validator_error_message(exc) @@ -84,7 +85,7 @@ def parse(cls, raw_partition_by) -> Optional['PartitionConfig']: @dataclass -class GrantTarget(JsonSchemaMixin): +class GrantTarget(dbtClassMixin): dataset: str project: str @@ -798,6 +799,7 @@ def grant_access_to(self, entity, entity_type, role, grant_target_dict): conn = self.connections.get_thread_connection() client = conn.handle + GrantTarget.validate(grant_target_dict) grant_target = GrantTarget.from_dict(grant_target_dict) dataset = client.get_dataset( self.connections.dataset_from_id(grant_target.render()) From 403eaa703b85be7a3f20297301ba9cf0a2cf9acd Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Wed, 10 Feb 2021 00:31:40 -0500 Subject: [PATCH 329/860] update docs with permalinks and new git branching --- README.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index b7e967abe..d2e8e38b9 100644 --- a/README.md +++ b/README.md @@ -1,5 +1,5 @@

- dbt logo + dbt logo

**[dbt](https://www.getdbt.com/)** (data build tool) enables data analysts and engineers to transform their data using the same practices that software engineers use to build applications. @@ -14,8 +14,8 @@ more information on using dbt with BigQuery, consult [the docs](https://docs.get ## Find out more -- Check out the [Introduction to dbt](https://dbt.readme.io/docs/introduction). -- Read the [dbt Viewpoint](https://dbt.readme.io/docs/viewpoint). +- Check out the [Introduction to dbt](https://docs.getdbt.com/docs/introduction/). +- Read the [dbt Viewpoint](https://docs.getdbt.com/docs/about/viewpoint/). ## Join thousands of analysts in the dbt community @@ -25,8 +25,8 @@ more information on using dbt with BigQuery, consult [the docs](https://docs.get ## Reporting bugs and contributing code - Want to report a bug or request a feature? Let us know on [Slack](http://community.getdbt.com/), or open [an issue](https://github.com/fishtown-analytics/dbt/issues/new). -- Want to help us build dbt? Check out the [Contributing Getting Started Guide](/CONTRIBUTING.md) +- Want to help us build dbt? Check out the [Contributing Getting Started Guide](https://github.com/fishtown-analytics/dbt/blob/HEAD/CONTRIBUTING.md) ## Code of Conduct -Everyone interacting in the dbt project's codebases, issue trackers, chat rooms, and mailing lists is expected to follow the [PyPA Code of Conduct](https://www.pypa.io/en/latest/code-of-conduct/). +Everyone interacting in the dbt project's codebases, issue trackers, chat rooms, and mailing lists is expected to follow the [dbt Code of Conduct](https://community.getdbt.com/code-of-conduct). From 822295feda96493aec6df0b63ad70c4c994b10e1 Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Fri, 12 Feb 2021 14:03:03 +0100 Subject: [PATCH 330/860] Fix int64, ts partitions. Rework tests --- dbt/adapters/bigquery/impl.py | 14 ++++++++------ .../macros/materializations/incremental.sql | 5 +---- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index d45dc2868..7ccedde94 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -58,8 +58,10 @@ def render(self, alias: Optional[str] = None): if alias: column = f'{alias}.{self.field}' - if self.data_type.lower() == 'date' and \ - self.granularity.lower() == 'day': + if self.data_type.lower() == 'int64' or ( + self.data_type.lower() == 'date' and + self.granularity.lower() == 'day' + ): return column else: return f'{self.data_type}_trunc({column}, {self.granularity})' @@ -551,10 +553,10 @@ def _partitions_match( if not is_partitioned and not conf_partition: return True elif conf_partition and table.time_partitioning is not None: - table_field = table.time_partitioning.field - table_granularity = table.partitioning_type - return table_field == conf_partition.field \ - and table_granularity == conf_partition.granularity + table_field = table.time_partitioning.field.lower() + table_granularity = table.partitioning_type.lower() + return table_field == conf_partition.field.lower() \ + and table_granularity == conf_partition.granularity.lower() elif conf_partition and table.range_partitioning is not None: dest_part = table.range_partitioning conf_part = conf_partition.range or {} diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 79dd112b0..f4ad80d5a 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -16,9 +16,6 @@ {% macro bq_insert_overwrite(tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns) %} - {%- set partition_type = - 'date' if partition_by.data_type in ('timestamp, datetime') - else partition_by.data_type -%} {% if partitions is not none and partitions != [] %} {# static #} @@ -49,7 +46,7 @@ {%- endset -%} -- generated script to merge partitions into {{ target_relation }} - declare dbt_partitions_for_replacement array<{{ partition_type }}>; + declare dbt_partitions_for_replacement array<{{ partition_by.data_type }}>; declare _dbt_max_partition {{ partition_by.data_type }}; set _dbt_max_partition = ( From 8e79101b5ce8d8788de8203fe246b61fa561fcc9 Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Fri, 12 Feb 2021 14:03:03 +0100 Subject: [PATCH 331/860] Fix int64, ts partitions. Rework tests --- dbt/adapters/bigquery/impl.py | 14 ++++++++------ .../macros/materializations/incremental.sql | 5 +---- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index d752a473c..1d42c1eee 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -58,8 +58,10 @@ def render(self, alias: Optional[str] = None): if alias: column = f'{alias}.{self.field}' - if self.data_type.lower() == 'date' and \ - self.granularity.lower() == 'day': + if self.data_type.lower() == 'int64' or ( + self.data_type.lower() == 'date' and + self.granularity.lower() == 'day' + ): return column else: return f'{self.data_type}_trunc({column}, {self.granularity})' @@ -549,10 +551,10 @@ def _partitions_match( if not is_partitioned and not conf_partition: return True elif conf_partition and table.time_partitioning is not None: - table_field = table.time_partitioning.field - table_granularity = table.partitioning_type - return table_field == conf_partition.field \ - and table_granularity == conf_partition.granularity + table_field = table.time_partitioning.field.lower() + table_granularity = table.partitioning_type.lower() + return table_field == conf_partition.field.lower() \ + and table_granularity == conf_partition.granularity.lower() elif conf_partition and table.range_partitioning is not None: dest_part = table.range_partitioning conf_part = conf_partition.range or {} diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index fb6cccb25..3b4455138 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -16,9 +16,6 @@ {% macro bq_insert_overwrite(tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns) %} - {%- set partition_type = - 'date' if partition_by.data_type in ('timestamp, datetime') - else partition_by.data_type -%} {% if partitions is not none and partitions != [] %} {# static #} @@ -49,7 +46,7 @@ {%- endset -%} -- generated script to merge partitions into {{ target_relation }} - declare dbt_partitions_for_replacement array<{{ partition_type }}>; + declare dbt_partitions_for_replacement array<{{ partition_by.data_type }}>; declare _dbt_max_partition {{ partition_by.data_type }}; set _dbt_max_partition = ( From ff4f72d1d430176334c1c8996cfd7a356dadf0fd Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Fri, 12 Feb 2021 16:06:52 +0000 Subject: [PATCH 332/860] Release dbt v0.19.1b1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index f8bf85e8d..36a9bf684 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.19.0' +version = '0.19.1b1' diff --git a/setup.py b/setup.py index 634f658d3..64364fa91 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.19.0" +package_version = "0.19.1b1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From ffb762d454efcae05e93ff3fc22311772c9e061d Mon Sep 17 00:00:00 2001 From: prratek Date: Sat, 13 Feb 2021 11:09:25 -0500 Subject: [PATCH 333/860] get columns to update from config for BQ and Snowflake --- dbt/include/bigquery/macros/materializations/incremental.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index f4ad80d5a..1e131f495 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -112,7 +112,7 @@ {% endif %} {% set build_sql = create_table_as(False, target_relation, sql) %} {% else %} - {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %} + {% set dest_columns = config.get('update_columns') %} {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#} {% if strategy == 'insert_overwrite' %} From 561ef767a58fcb43a406925310a9224240fa8b36 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Mon, 15 Feb 2021 17:00:50 +0000 Subject: [PATCH 334/860] Release dbt v0.19.1b2 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 36a9bf684..290873c54 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.19.1b1' +version = '0.19.1b2' diff --git a/setup.py b/setup.py index 64364fa91..778137eb3 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.19.1b1" +package_version = "0.19.1b2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From bf7fc2fe7bfc623ca2b6bf14fe2b07bdc6804466 Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Mon, 15 Feb 2021 13:26:18 -0500 Subject: [PATCH 335/860] Revert "Merge branch '0.19.latest' into develop" This reverts commit 5c9f8a0cf0af9ef2f77d55c96f04d0e87c29680f, reversing changes made to 36d1bddc5b3ec7b0f503972d7138e8260693da38. --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 290873c54..f8bf85e8d 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.19.1b2' +version = '0.19.0' diff --git a/setup.py b/setup.py index 778137eb3..634f658d3 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.19.1b2" +package_version = "0.19.0" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From b5a2e6a6a909b366c40b20b7d52e135741021793 Mon Sep 17 00:00:00 2001 From: Prratek Ramchandani <34848565+prratek@users.noreply.github.com> Date: Mon, 15 Feb 2021 20:58:39 -0500 Subject: [PATCH 336/860] default to get_columns_in_relation if not specified in config Co-authored-by: Jeremy Cohen --- dbt/include/bigquery/macros/materializations/incremental.sql | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 1e131f495..a48178f88 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -112,7 +112,10 @@ {% endif %} {% set build_sql = create_table_as(False, target_relation, sql) %} {% else %} - {% set dest_columns = config.get('update_columns') %} + {% set dest_columns = config.get('update_columns', none) %} + {% if dest_columns is none %} + {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %} + {% endif %} {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#} {% if strategy == 'insert_overwrite' %} From a7a2c7d2d4e93f83e192c4a3d9807cac8d363f6f Mon Sep 17 00:00:00 2001 From: Pascal Casteran <15686972+pcasteran@users.noreply.github.com> Date: Thu, 24 Dec 2020 22:57:50 +0100 Subject: [PATCH 337/860] Fix `_dbt_max_partition` declaration and initialization for BigQuery incremental models --- dbt/include/bigquery/macros/materializations/incremental.sql | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index f4ad80d5a..0ac4c3823 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -47,9 +47,7 @@ -- generated script to merge partitions into {{ target_relation }} declare dbt_partitions_for_replacement array<{{ partition_by.data_type }}>; - declare _dbt_max_partition {{ partition_by.data_type }}; - - set _dbt_max_partition = ( + declare _dbt_max_partition {{ partition_by.data_type }} default ( select max({{ partition_by.field }}) from {{ this }} where {{ partition_by.field }} is not null ); From 2401c1e0e0c7574ace6db50126bf3f55e75d5390 Mon Sep 17 00:00:00 2001 From: prratek Date: Fri, 19 Feb 2021 21:13:05 -0500 Subject: [PATCH 338/860] rename config field to incremental_update_columns --- dbt/include/bigquery/macros/materializations/incremental.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index a48178f88..48d8ac3c4 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -112,7 +112,7 @@ {% endif %} {% set build_sql = create_table_as(False, target_relation, sql) %} {% else %} - {% set dest_columns = config.get('update_columns', none) %} + {% set dest_columns = config.get('incremental_update_columns', none) %} {% if dest_columns is none %} {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %} {% endif %} From 94336b41ea3448d0c9c937bcf7e18c4129ccce91 Mon Sep 17 00:00:00 2001 From: prratek Date: Fri, 19 Feb 2021 21:17:39 -0500 Subject: [PATCH 339/860] add incremental_update_columns to Snowflake & BQ config schemas --- dbt/adapters/bigquery/impl.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 7ccedde94..6fa20c3d9 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -116,6 +116,7 @@ class BigqueryConfig(AdapterConfig): hours_to_expiration: Optional[int] = None require_partition_filter: Optional[bool] = None partition_expiration_days: Optional[int] = None + incrmental_update_columns: Optional[str] = None class BigQueryAdapter(BaseAdapter): From 567e5b5fda1535593eaaff0596199b6b29541837 Mon Sep 17 00:00:00 2001 From: Prratek Ramchandani <34848565+prratek@users.noreply.github.com> Date: Tue, 2 Mar 2021 21:59:21 -0500 Subject: [PATCH 340/860] fix typo Co-authored-by: Jeremy Cohen --- dbt/adapters/bigquery/impl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 6fa20c3d9..53201e49a 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -116,7 +116,7 @@ class BigqueryConfig(AdapterConfig): hours_to_expiration: Optional[int] = None require_partition_filter: Optional[bool] = None partition_expiration_days: Optional[int] = None - incrmental_update_columns: Optional[str] = None + incremental_update_columns: Optional[str] = None class BigQueryAdapter(BaseAdapter): From eb63118f4e1400d3afe957ace5d498eb8ad94714 Mon Sep 17 00:00:00 2001 From: prratek Date: Tue, 2 Mar 2021 22:11:12 -0500 Subject: [PATCH 341/860] rename new config var to merge_update_columns --- dbt/adapters/bigquery/impl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 53201e49a..f97e00848 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -116,7 +116,7 @@ class BigqueryConfig(AdapterConfig): hours_to_expiration: Optional[int] = None require_partition_filter: Optional[bool] = None partition_expiration_days: Optional[int] = None - incremental_update_columns: Optional[str] = None + merge_update_columns: Optional[str] = None class BigQueryAdapter(BaseAdapter): From ddf9927aeeb113bcecbf193148e5202bafa94fc3 Mon Sep 17 00:00:00 2001 From: prratek Date: Tue, 2 Mar 2021 22:14:58 -0500 Subject: [PATCH 342/860] revert changes to incremental materializations --- dbt/include/bigquery/macros/materializations/incremental.sql | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 48d8ac3c4..f4ad80d5a 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -112,10 +112,7 @@ {% endif %} {% set build_sql = create_table_as(False, target_relation, sql) %} {% else %} - {% set dest_columns = config.get('incremental_update_columns', none) %} - {% if dest_columns is none %} - {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %} - {% endif %} + {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %} {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#} {% if strategy == 'insert_overwrite' %} From 569f68db00cb45bde7a2526d5b0de9a0489497b1 Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Thu, 4 Mar 2021 00:06:59 -0500 Subject: [PATCH 343/860] Parse query comment and use as bigquery job labels. --- dbt/adapters/bigquery/connections.py | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index d660e75dd..3e52abc67 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -1,3 +1,4 @@ +import json from contextlib import contextmanager from dataclasses import dataclass from functools import lru_cache @@ -305,12 +306,17 @@ def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False): logger.debug('On {}: {}', conn.name, sql) - job_params = {'use_legacy_sql': use_legacy_sql} + job_params = {'use_legacy_sql': use_legacy_sql, 'labels': {}} if active_user: - job_params['labels'] = { - 'dbt_invocation_id': active_user.invocation_id - } + job_params['labels']['dbt_invocation_id'] = active_user.invocation_id + + if self.profile.query_comment.job_label: + try: + labels = json.loads(self.query_header.comment.query_comment) + job_params['labels'].update(labels) + except (TypeError, ValueError): + pass priority = conn.credentials.priority if priority == Priority.Batch: From ddd2c8e7de58291be048f6e0fb528eed066bde0b Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Thu, 4 Mar 2021 00:14:50 -0500 Subject: [PATCH 344/860] Sanitize bigquery labels. --- dbt/adapters/bigquery/connections.py | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 3e52abc67..7c13e5f4b 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -1,4 +1,5 @@ import json +import re from contextlib import contextmanager from dataclasses import dataclass from functools import lru_cache @@ -314,7 +315,10 @@ def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False): if self.profile.query_comment.job_label: try: labels = json.loads(self.query_header.comment.query_comment) - job_params['labels'].update(labels) + job_params['labels'].update({ + _sanitize_bigquery_label(key): _sanitize_bigquery_label(str(value)) + for key, value in labels.items() + }) except (TypeError, ValueError): pass @@ -579,3 +583,14 @@ def _is_retryable(error): e['reason'] == 'rateLimitExceeded' for e in error.errors): return True return False + + +_SANITIZE_BIGQUERY_LABEL_PATTERN = re.compile(r"[^a-z0-9_-]") + + +def _sanitize_bigquery_label(value: str, max_length: int = 63) -> str: + """Return a legal value for a BigQuery label.""" + value = value.lower() + value = _SANITIZE_BIGQUERY_LABEL_PATTERN.sub("_", value) + value = value[: max_length - 1] + return value From bee763f09f903db4ef638254516d5b63a855c00b Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Thu, 4 Mar 2021 00:34:46 -0500 Subject: [PATCH 345/860] Fix lint. --- dbt/adapters/bigquery/connections.py | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 7c13e5f4b..99e68f731 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -307,21 +307,25 @@ def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False): logger.debug('On {}: {}', conn.name, sql) - job_params = {'use_legacy_sql': use_legacy_sql, 'labels': {}} + labels = {} if active_user: - job_params['labels']['dbt_invocation_id'] = active_user.invocation_id + labels['dbt_invocation_id'] = active_user.invocation_id if self.profile.query_comment.job_label: try: - labels = json.loads(self.query_header.comment.query_comment) - job_params['labels'].update({ - _sanitize_bigquery_label(key): _sanitize_bigquery_label(str(value)) - for key, value in labels.items() + comment_labels = json.loads( + self.query_header.comment.query_comment + ) + labels.update({ + _sanitize_label(key): _sanitize_label(str(value)) + for key, value in comment_labels.items() }) except (TypeError, ValueError): pass + job_params = {'use_legacy_sql': use_legacy_sql, 'labels': labels} + priority = conn.credentials.priority if priority == Priority.Batch: job_params['priority'] = google.cloud.bigquery.QueryPriority.BATCH @@ -585,12 +589,12 @@ def _is_retryable(error): return False -_SANITIZE_BIGQUERY_LABEL_PATTERN = re.compile(r"[^a-z0-9_-]") +_SANITIZE_LABEL_PATTERN = re.compile(r"[^a-z0-9_-]") -def _sanitize_bigquery_label(value: str, max_length: int = 63) -> str: +def _sanitize_label(value: str, max_length: int = 63) -> str: """Return a legal value for a BigQuery label.""" value = value.lower() - value = _SANITIZE_BIGQUERY_LABEL_PATTERN.sub("_", value) + value = _SANITIZE_LABEL_PATTERN.sub("_", value) value = value[: max_length - 1] return value From 43cdfae92aa820be95b7b078b46aa02efd516c5d Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Thu, 4 Mar 2021 10:06:52 -0500 Subject: [PATCH 346/860] Address comments from code review. --- dbt/adapters/bigquery/connections.py | 28 +++++++++++++++------------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 99e68f731..69ae21785 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -307,23 +307,15 @@ def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False): logger.debug('On {}: {}', conn.name, sql) - labels = {} + if self.profile.query_comment.job_label: + query_comment = self.query_header.comment.query_comment + labels = self._labels_from_query_comment(query_comment) + else: + labels = {} if active_user: labels['dbt_invocation_id'] = active_user.invocation_id - if self.profile.query_comment.job_label: - try: - comment_labels = json.loads( - self.query_header.comment.query_comment - ) - labels.update({ - _sanitize_label(key): _sanitize_label(str(value)) - for key, value in comment_labels.items() - }) - except (TypeError, ValueError): - pass - job_params = {'use_legacy_sql': use_legacy_sql, 'labels': labels} priority = conn.credentials.priority @@ -558,6 +550,16 @@ def _retry_generator(self): initial=self.DEFAULT_INITIAL_DELAY, maximum=self.DEFAULT_MAXIMUM_DELAY) + def _labels_from_query_comment(self, comment: str) -> Dict: + try: + comment_labels = json.loads(comment) + except (TypeError, ValueError): + return {'query_comment': _sanitize_label(comment)} + return { + _sanitize_label(key): _sanitize_label(str(value)) + for key, value in comment_labels.items() + } + class _ErrorCounter(object): """Counts errors seen up to a threshold then raises the next error.""" From 2d9de128a9946c604500e61bd712d3607df2f901 Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Mon, 8 Mar 2021 15:39:02 -0500 Subject: [PATCH 347/860] Add tests for bigquery label sanitize helper. --- dbt/adapters/bigquery/connections.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 69ae21785..acbecfa84 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -596,7 +596,7 @@ def _is_retryable(error): def _sanitize_label(value: str, max_length: int = 63) -> str: """Return a legal value for a BigQuery label.""" - value = value.lower() + value = value.strip().lower() value = _SANITIZE_LABEL_PATTERN.sub("_", value) - value = value[: max_length - 1] + value = value[: max_length] return value From 00488db500bf5b787226ee6e6e4465008803bede Mon Sep 17 00:00:00 2001 From: Maximilian Roos <5635139+max-sixty@users.noreply.github.com> Date: Wed, 10 Mar 2021 19:57:57 -0800 Subject: [PATCH 348/860] Update google cloud dependencies --- setup.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/setup.py b/setup.py index 634f658d3..a75b022f9 100644 --- a/setup.py +++ b/setup.py @@ -50,10 +50,10 @@ 'protobuf>=3.13.0,<4', # These are more tightly pinned, as they have a track record of # breaking changes in minor releases. - 'google-cloud-core>=1.3.0,<1.5', - 'google-cloud-bigquery>=1.25.0,<2.4', - 'google-api-core>=1.16.0,<1.24', - 'googleapis-common-protos>=1.6.0,<1.53', + 'google-cloud-core>=1.3.0,<1.7', + 'google-cloud-bigquery>=1.25.0,<2.12', + 'google-api-core>=1.16.0,<1.27', + 'googleapis-common-protos>=1.6.0,<1.54', 'six>=1.14.0', ], zip_safe=False, From cb5d2da25b2825c608c407cab64479a1a8232b8f Mon Sep 17 00:00:00 2001 From: Joshua Carp Date: Fri, 19 Mar 2021 23:50:45 -0400 Subject: [PATCH 349/860] Cleanups from code review. --- dbt/adapters/bigquery/connections.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index acbecfa84..a3ff5d0e6 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -594,9 +594,8 @@ def _is_retryable(error): _SANITIZE_LABEL_PATTERN = re.compile(r"[^a-z0-9_-]") -def _sanitize_label(value: str, max_length: int = 63) -> str: +def _sanitize_label(value: str) -> str: """Return a legal value for a BigQuery label.""" value = value.strip().lower() value = _SANITIZE_LABEL_PATTERN.sub("_", value) - value = value[: max_length] return value From 5c987a3dc1e85da0af8c13d32dc08a010bed1be9 Mon Sep 17 00:00:00 2001 From: Maximilian Roos <5635139+max-sixty@users.noreply.github.com> Date: Wed, 31 Mar 2021 18:51:53 -0700 Subject: [PATCH 350/860] Pin to major versions --- setup.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/setup.py b/setup.py index a75b022f9..e39251ebc 100644 --- a/setup.py +++ b/setup.py @@ -50,10 +50,10 @@ 'protobuf>=3.13.0,<4', # These are more tightly pinned, as they have a track record of # breaking changes in minor releases. - 'google-cloud-core>=1.3.0,<1.7', - 'google-cloud-bigquery>=1.25.0,<2.12', - 'google-api-core>=1.16.0,<1.27', - 'googleapis-common-protos>=1.6.0,<1.54', + 'google-cloud-core>=1.3.0,<2', + 'google-cloud-bigquery>=1.25.0,<3', + 'google-api-core>=1.16.0,<2', + 'googleapis-common-protos>=1.6.0,<2', 'six>=1.14.0', ], zip_safe=False, From 4e6c333ad9ccbc2a4db0f2b4b0bb0c68051ba4f4 Mon Sep 17 00:00:00 2001 From: Maximilian Roos <5635139+max-sixty@users.noreply.github.com> Date: Fri, 2 Apr 2021 10:05:16 -0700 Subject: [PATCH 351/860] Update setup.py --- setup.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/setup.py b/setup.py index e39251ebc..70494b16a 100644 --- a/setup.py +++ b/setup.py @@ -48,8 +48,6 @@ install_requires=[ 'dbt-core=={}'.format(package_version), 'protobuf>=3.13.0,<4', - # These are more tightly pinned, as they have a track record of - # breaking changes in minor releases. 'google-cloud-core>=1.3.0,<2', 'google-cloud-bigquery>=1.25.0,<3', 'google-api-core>=1.16.0,<2', From 58d9c2009d54e4eaff3cf277a66f2df66739f75f Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Wed, 28 Apr 2021 13:37:23 -0400 Subject: [PATCH 352/860] Merge pull request #3305 from fishtown-analytics/fix/bigquery-no-project add unit test and move default logic to mashumaro hook --- dbt/adapters/bigquery/connections.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 84fa58624..e0140c38c 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -114,15 +114,17 @@ def _connection_keys(self): return ('method', 'database', 'schema', 'location', 'priority', 'timeout_seconds', 'maximum_bytes_billed') - def __post_init__(self): + @classmethod + def __pre_deserialize__(cls, d: Dict[Any, Any]) -> Dict[Any, Any]: # We need to inject the correct value of the database (aka project) at # this stage, ref # https://github.com/fishtown-analytics/dbt/pull/2908#discussion_r532927436. # `database` is an alias of `project` in BigQuery - if self.database is None: + if 'database' not in d: _, database = get_bigquery_defaults() - self.database = database + d['database'] = database + return d class BigQueryConnectionManager(BaseConnectionManager): From a0fae239eb825abab06c5a5b844b6615c8db2b10 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Mon, 3 May 2021 11:08:53 +0000 Subject: [PATCH 353/860] Release dbt v0.20.0b1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index f8bf85e8d..64d4c3082 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.19.0' +version = '0.20.0b1' diff --git a/setup.py b/setup.py index 70494b16a..b403f8f30 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.19.0" +package_version = "0.20.0b1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From b2ca2386cab820f5e4083cda76f8e5a34de95121 Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Wed, 12 May 2021 18:35:30 -0400 Subject: [PATCH 354/860] fix debug task for bigquery connections --- dbt/adapters/bigquery/connections.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index e0140c38c..137b02d4f 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -309,7 +309,7 @@ def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False): logger.debug('On {}: {}', conn.name, sql) - if self.profile.query_comment.job_label: + if self.profile.query_comment and self.profile.query_comment.job_label: query_comment = self.query_header.comment.query_comment labels = self._labels_from_query_comment(query_comment) else: From 94d4ffc8aa1417a4e229695268638be85061a709 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Fri, 4 Jun 2021 18:31:00 +0000 Subject: [PATCH 355/860] Release dbt v0.20.0rc1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 64d4c3082..de34f3567 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.20.0b1' +version = '0.20.0rc1' diff --git a/setup.py b/setup.py index b403f8f30..6c33c2835 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.20.0b1" +package_version = "0.20.0rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 2b1f2eabb3290652210193205edee6d31e4aa735 Mon Sep 17 00:00:00 2001 From: leahwicz <60146280+leahwicz@users.noreply.github.com> Date: Thu, 15 Jul 2021 16:46:22 -0400 Subject: [PATCH 356/860] Adding missing v0.20.0 files back to `develop` (#3566) --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index de34f3567..0c81391a3 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.20.0rc1' +version = '0.21.0a1' diff --git a/setup.py b/setup.py index 6c33c2835..eb48850f6 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.20.0rc1" +package_version = "0.21.0a1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 2629613b3c237710cc9d9c3c8d2d604b0df9dc13 Mon Sep 17 00:00:00 2001 From: matt-winkler <75497565+matt-winkler@users.noreply.github.com> Date: Wed, 21 Jul 2021 13:49:19 -0600 Subject: [PATCH 357/860] Feature: on_schema_change for incremental models (#3387) * detect and act on schema changes * update incremental helpers code * update changelog * fix error in diff_columns from testing * abstract code a bit further * address matching names vs. data types * Update CHANGELOG.md Co-authored-by: Jeremy Cohen * updates from Jeremy's feedback * multi-column add / remove with full_refresh * simple changes from JC's feedback * updated for snowflake * reorganize postgres code * reorganize approach * updated full refresh trigger logic * fixed unintentional wipe behavior * catch final else condition * remove WHERE string replace * touch ups * port core to snowflake * added bigquery code * updated impacted unit tests * updates from linting tests * updates from linting again * snowflake updates from further testing * fix logging * clean up incremental logic * updated for bigquery * update postgres with new strategy * update nodeconfig * starting integration tests * integration test for ignore case * add test for append_new_columns * add integration test for sync * remove extra tests * add unique key and snowflake test * move incremental integration test dir * update integration tests * update integration tests * Suggestions for #3387 (#3558) * PR feedback: rationalize macros + logging, fix + expand tests * Rm alter_column_types, always true for sync_all_columns * update logging and integration test on sync * update integration tests * test fix SF integration tests Co-authored-by: Matt Winkler * rename integration test folder * Update core/dbt/include/global_project/macros/materializations/incremental/incremental.sql Accept Jeremy's suggested change Co-authored-by: Jeremy Cohen * Update changelog [skip ci] Co-authored-by: Jeremy Cohen --- dbt/include/bigquery/macros/adapters.sql | 32 ++++++ .../macros/materializations/incremental.sql | 99 ++++++++++++------- 2 files changed, 96 insertions(+), 35 deletions(-) diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index 3bb7cd606..d21d5f04e 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -128,6 +128,38 @@ {% do adapter.rename_relation(from_relation, to_relation) %} {% endmacro %} +{% macro bigquery__alter_relation_add_columns(relation, add_columns) %} + + {% set sql -%} + + alter {{ relation.type }} {{ relation }} + {% for column in add_columns %} + add column {{ column.name }} {{ column.data_type }}{{ ',' if not loop.last }} + {% endfor %} + + {%- endset -%} + + {{ return(run_query(sql)) }} + +{% endmacro %} + +{% macro bigquery__alter_relation_drop_columns(relation, drop_columns) %} + + {% set sql -%} + + alter {{ relation.type }} {{ relation }} + + {% for column in drop_columns %} + drop column {{ column.name }}{{ ',' if not loop.last }} + {% endfor %} + + {%- endset -%} + + {{ return(run_query(sql)) }} + +{% endmacro %} + + {% macro bigquery__alter_column_type(relation, column_name, new_column_type) -%} {# Changing a column's data type using a query requires you to scan the entire table. diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql index 0ac4c3823..0b8e7bac5 100644 --- a/dbt/include/bigquery/macros/materializations/incremental.sql +++ b/dbt/include/bigquery/macros/materializations/incremental.sql @@ -15,7 +15,9 @@ {% endmacro %} -{% macro bq_insert_overwrite(tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns) %} +{% macro bq_insert_overwrite( + tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists +) %} {% if partitions is not none and partitions != [] %} {# static #} @@ -52,8 +54,13 @@ where {{ partition_by.field }} is not null ); - -- 1. create a temp table - {{ create_table_as(True, tmp_relation, sql) }} + {# have we already created the temp table to check for schema changes? #} + {% if not tmp_relation_exists %} + -- 1. create a temp table + {{ create_table_as(True, tmp_relation, sql) }} + {% else %} + -- 1. temp table already exists, we used it to check for schema changes + {% endif %} -- 2. define partitions to update set (dbt_partitions_for_replacement) = ( @@ -77,6 +84,44 @@ {% endmacro %} +{% macro bq_generate_incremental_build_sql( + strategy, tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists +) %} + {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#} + {% if strategy == 'insert_overwrite' %} + + {% set missing_partition_msg -%} + The 'insert_overwrite' strategy requires the `partition_by` config. + {%- endset %} + {% if partition_by is none %} + {% do exceptions.raise_compiler_error(missing_partition_msg) %} + {% endif %} + + {% set build_sql = bq_insert_overwrite( + tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, on_schema_change + ) %} + + {% else %} {# strategy == 'merge' #} + {%- set source_sql -%} + {%- if tmp_relation_exists -%} + ( + select * from {{ tmp_relation }} + ) + {%- else -%} {#-- wrap sql in parens to make it a subquery --#} + ( + {{sql}} + ) + {%- endif -%} + {%- endset -%} + + {% set build_sql = get_merge_sql(target_relation, source_sql, unique_key, dest_columns) %} + + {% endif %} + + {{ return(build_sql) }} + +{% endmacro %} + {% materialization incremental, adapter='bigquery' -%} {%- set unique_key = config.get('unique_key') -%} @@ -94,14 +139,18 @@ {%- set partitions = config.get('partitions', none) -%} {%- set cluster_by = config.get('cluster_by', none) -%} + {% set on_schema_change = incremental_validate_on_schema_change(config.get('on_schema_change'), default='ignore') %} + {{ run_hooks(pre_hooks) }} {% if existing_relation is none %} {% set build_sql = create_table_as(False, target_relation, sql) %} + {% elif existing_relation.is_view %} {#-- There's no way to atomically replace a view with a table on BQ --#} {{ adapter.drop_relation(existing_relation) }} {% set build_sql = create_table_as(False, target_relation, sql) %} + {% elif full_refresh_mode %} {#-- If the partition/cluster config has changed, then we must drop and recreate --#} {% if not adapter.is_replaceable(existing_relation, partition_by, cluster_by) %} @@ -109,39 +158,19 @@ {{ adapter.drop_relation(existing_relation) }} {% endif %} {% set build_sql = create_table_as(False, target_relation, sql) %} + {% else %} - {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %} - - {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#} - {% if strategy == 'insert_overwrite' %} - - {% set missing_partition_msg -%} - The 'insert_overwrite' strategy requires the `partition_by` config. - {%- endset %} - {% if partition_by is none %} - {% do exceptions.raise_compiler_error(missing_partition_msg) %} - {% endif %} - - {% set build_sql = bq_insert_overwrite( - tmp_relation, - target_relation, - sql, - unique_key, - partition_by, - partitions, - dest_columns) %} - - {% else %} - {#-- wrap sql in parens to make it a subquery --#} - {%- set source_sql -%} - ( - {{sql}} - ) - {%- endset -%} - - {% set build_sql = get_merge_sql(target_relation, source_sql, unique_key, dest_columns) %} - - {% endif %} + {% set tmp_relation_exists = false %} + {% if on_schema_change != 'ignore' %} {# Check first, since otherwise we may not build a temp table #} + {% do run_query(create_table_as(True, tmp_relation, sql)) %} + {% set tmp_relation_exists = true %} + {% do process_schema_changes(on_schema_change, tmp_relation, existing_relation) %} + {% endif %} + + {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %} + {% set build_sql = bq_generate_incremental_build_sql( + strategy, tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists + ) %} {% endif %} From be572db73f4d5aed50f7468b3ef4e08f5bca013a Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Tue, 27 Jul 2021 18:13:35 -0400 Subject: [PATCH 358/860] Snowflake: no transactions, except for DML (#3510) * Rm Snowflake txnal logic. Explicit for DML * Be less clever. Update create_or_replace_view() * Seed DML as well * Changelog entry * Fix unit test * One semicolon can change the world --- dbt/include/bigquery/macros/materializations/view.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/include/bigquery/macros/materializations/view.sql b/dbt/include/bigquery/macros/materializations/view.sql index 887905129..97e3d2761 100644 --- a/dbt/include/bigquery/macros/materializations/view.sql +++ b/dbt/include/bigquery/macros/materializations/view.sql @@ -9,7 +9,7 @@ {% materialization view, adapter='bigquery' -%} - {% set to_return = create_or_replace_view(run_outside_transaction_hooks=False) %} + {% set to_return = create_or_replace_view() %} {% set target_relation = this.incorporate(type='view') %} {% do persist_docs(target_relation, model) %} From bf461f7ac8f50b80ab42aa596895bd57e7700af0 Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Fri, 30 Jul 2021 08:46:58 -0400 Subject: [PATCH 359/860] update links to dbt repo (#3521) --- setup.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/setup.py b/setup.py index eb48850f6..d99511ce4 100644 --- a/setup.py +++ b/setup.py @@ -33,9 +33,9 @@ description=description, long_description=long_description, long_description_content_type='text/markdown', - author="Fishtown Analytics", - author_email="info@fishtownanalytics.com", - url="https://github.com/fishtown-analytics/dbt", + author="dbt Labs", + author_email="info@dbtlabs.com", + url="https://github.com/dbt-labs/dbt", packages=find_namespace_packages(include=['dbt', 'dbt.*']), package_data={ 'dbt': [ From aa844e31cbe42c9eea4a7d23643209170505d198 Mon Sep 17 00:00:00 2001 From: Github Build Bot Date: Tue, 3 Aug 2021 17:30:32 +0000 Subject: [PATCH 360/860] Release dbt v0.21.0b1 --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 0c81391a3..278d33a8e 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.21.0a1' +version = '0.21.0b1' diff --git a/setup.py b/setup.py index d99511ce4..2a4655baf 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.21.0a1" +package_version = "0.21.0b1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 031247447f12e1a98f64d41b2025ae8ef3d1fae6 Mon Sep 17 00:00:00 2001 From: sungchun12 Date: Tue, 17 Aug 2021 13:54:31 -0500 Subject: [PATCH 361/860] Fix/bigquery job label length (#3703) * add blueprints to resolve issue * revert to previous version * intentionally failing test * add imports * add validation in existing function * add passing test for length validation * add current sanitized label * remove duplicate var * Make logging output 2 lines Co-authored-by: Jeremy Cohen * Raise RuntimeException to better handle error Co-authored-by: Jeremy Cohen * update test * fix flake8 errors * update changelog Co-authored-by: Jeremy Cohen --- dbt/adapters/bigquery/connections.py | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 137b02d4f..62de829b2 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -595,9 +595,20 @@ def _is_retryable(error): _SANITIZE_LABEL_PATTERN = re.compile(r"[^a-z0-9_-]") +_VALIDATE_LABEL_LENGTH_LIMIT = 63 + def _sanitize_label(value: str) -> str: """Return a legal value for a BigQuery label.""" value = value.strip().lower() value = _SANITIZE_LABEL_PATTERN.sub("_", value) - return value + value_length = len(value) + if value_length > _VALIDATE_LABEL_LENGTH_LIMIT: + error_msg = ( + f"Job label length {value_length} is greater than length limit: " + f"{_VALIDATE_LABEL_LENGTH_LIMIT}\n" + f"Current sanitized label: {value}" + ) + raise RuntimeException(error_msg) + else: + return value From 25cdbd803ee7d6a78d03ddd649cb0b855e0a33e4 Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Wed, 25 Aug 2021 09:24:41 -0400 Subject: [PATCH 362/860] =?UTF-8?q?Remove=20fishtown-analytics=20reference?= =?UTF-8?q?s=20=F0=9F=98=A2=20(#3801)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 6 +++--- dbt/adapters/bigquery/connections.py | 2 +- dbt/adapters/bigquery/impl.py | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index d2e8e38b9..7a1b5f509 100644 --- a/README.md +++ b/README.md @@ -1,5 +1,5 @@

- dbt logo + dbt logo

**[dbt](https://www.getdbt.com/)** (data build tool) enables data analysts and engineers to transform their data using the same practices that software engineers use to build applications. @@ -24,8 +24,8 @@ more information on using dbt with BigQuery, consult [the docs](https://docs.get ## Reporting bugs and contributing code -- Want to report a bug or request a feature? Let us know on [Slack](http://community.getdbt.com/), or open [an issue](https://github.com/fishtown-analytics/dbt/issues/new). -- Want to help us build dbt? Check out the [Contributing Getting Started Guide](https://github.com/fishtown-analytics/dbt/blob/HEAD/CONTRIBUTING.md) +- Want to report a bug or request a feature? Let us know on [Slack](http://community.getdbt.com/), or open [an issue](https://github.com/dbt-labs/dbt/issues/new). +- Want to help us build dbt? Check out the [Contributing Getting Started Guide](https://github.com/dbt-labs/dbt/blob/HEAD/CONTRIBUTING.md) ## Code of Conduct diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 62de829b2..2c90297cd 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -118,7 +118,7 @@ def _connection_keys(self): def __pre_deserialize__(cls, d: Dict[Any, Any]) -> Dict[Any, Any]: # We need to inject the correct value of the database (aka project) at # this stage, ref - # https://github.com/fishtown-analytics/dbt/pull/2908#discussion_r532927436. + # https://github.com/dbt-labs/dbt/pull/2908#discussion_r532927436. # `database` is an alias of `project` in BigQuery if 'database' not in d: diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index f97e00848..2c5d94c66 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -274,7 +274,7 @@ def list_relations_without_caching( # next page. If that key table gets dropped before we run # list_relations, then this will 404. So, we avoid this # situation by making the page size sufficiently large. - # see: https://github.com/fishtown-analytics/dbt/issues/726 + # see: https://github.com/dbt-labs/dbt/issues/726 # TODO: cache the list of relations up front, and then we # won't need to do this max_results=100000) From 6d867e3169ce2b1939f7a4efd76b02a7ec08e9b0 Mon Sep 17 00:00:00 2001 From: Slava Kalashnikov Date: Mon, 30 Aug 2021 17:28:35 +0300 Subject: [PATCH 363/860] BigQuery copy materialization enhancement (#3606) * Change BigQuery copy materialization Change BigQuery copy materialization macros to copy data from several sources into single target * Change BigQuery copy materialization Change BigQuery connections.py to copy data from several sources into single target via copy materialization * Change BigQuery copy materialization Test to check default value of `copy_materialization` if it is absent in config * Change BigQuery copy materialization Update changelog * Update changelog * Var renaming + test addition * Changelog updated * Changelog updated * Fix test for copy table * Update test_bigquery_adapter.py * Update test_bigquery_adapter.py * Update impl.py * Update connections.py * Update test_bigquery_adapter.py * Update test_bigquery_adapter.py * Update connections.py * Align calls from mock and from adapter * Split long code ilnes * Create additional.sql * Update copy_as_several_tables.sql * Update schema.yml * Update copy.sql * Update connections.py * Update test_bigquery_copy_models.py * Add contributor --- dbt/adapters/bigquery/connections.py | 26 +++++++++++---- .../bigquery/macros/materializations/copy.sql | 33 ++++++++----------- 2 files changed, 33 insertions(+), 26 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 2c90297cd..1d2c5c609 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -458,26 +458,40 @@ def copy_bq_table(self, source, destination, write_disposition): conn = self.get_thread_connection() client = conn.handle - source_ref = self.table_ref( - source.database, source.schema, source.table, conn) +# ------------------------------------------------------------------------------- +# BigQuery allows to use copy API using two different formats: +# 1. client.copy_table(source_table_id, destination_table_id) +# where source_table_id = "your-project.source_dataset.source_table" +# 2. client.copy_table(source_table_ids, destination_table_id) +# where source_table_ids = ["your-project.your_dataset.your_table_name", ...] +# Let's use uniform function call and always pass list there +# ------------------------------------------------------------------------------- + if type(source) is not list: + source = [source] + + source_ref_array = [self.table_ref( + src_table.database, src_table.schema, src_table.table, conn) + for src_table in source] destination_ref = self.table_ref( destination.database, destination.schema, destination.table, conn) logger.debug( - 'Copying table "{}" to "{}" with disposition: "{}"', - source_ref.path, destination_ref.path, write_disposition) + 'Copying table(s) "{}" to "{}" with disposition: "{}"', + ', '.join(source_ref.path for source_ref in source_ref_array), + destination_ref.path, write_disposition) def copy_and_results(): job_config = google.cloud.bigquery.CopyJobConfig( write_disposition=write_disposition) copy_job = client.copy_table( - source_ref, destination_ref, job_config=job_config) + source_ref_array, destination_ref, job_config=job_config) iterator = copy_job.result(timeout=self.get_timeout(conn)) return copy_job, iterator self._retry_and_handle( msg='copy table "{}" to "{}"'.format( - source_ref.path, destination_ref.path), + ', '.join(source_ref.path for source_ref in source_ref_array), + destination_ref.path), conn=conn, fn=copy_and_results) @staticmethod diff --git a/dbt/include/bigquery/macros/materializations/copy.sql b/dbt/include/bigquery/macros/materializations/copy.sql index 0f24b50bc..6a86fbe44 100644 --- a/dbt/include/bigquery/macros/materializations/copy.sql +++ b/dbt/include/bigquery/macros/materializations/copy.sql @@ -3,31 +3,24 @@ {# Setup #} {{ run_hooks(pre_hooks) }} - {# there should be exactly one ref or exactly one source #} {% set destination = this.incorporate(type='table') %} - {% set dependency_type = none %} - {% if (model.refs | length) == 1 and (model.sources | length) == 0 %} - {% set dependency_type = 'ref' %} - {% elif (model.refs | length) == 0 and (model.sources | length) == 1 %} - {% set dependency_type = 'source' %} - {% else %} - {% set msg %} - Expected exactly one ref or exactly one source, instead got {{ model.refs | length }} models and {{ model.sources | length }} sources. - {% endset %} - {% do exceptions.raise_compiler_error(msg) %} - {% endif %} - - {% if dependency_type == 'ref' %} - {% set src = ref(*model.refs[0]) %} - {% else %} - {% set src = source(*model.sources[0]) %} - {% endif %} + {# there can be several ref() or source() according to BQ copy API docs #} + {# cycle over ref() and source() to create source tables array #} + {% set source_array = [] %} + {% for ref_table in model.refs %} + {{ source_array.append(ref(*ref_table)) }} + {% endfor %} + {% for src_table in model.sources %} + {{ source_array.append(source(*src_table)) }} + {% endfor %} + + {# Call adapter's copy_table function #} {%- set result_str = adapter.copy_table( - src, + source_array, destination, - config.get('copy_materialization', 'table')) -%} + config.get('copy_materialization', default = 'table')) -%} {{ store_result('main', response=result_str) }} From 75522768a7254132d92ab2012190d64da9b71eca Mon Sep 17 00:00:00 2001 From: Daniel Bartley Date: Wed, 1 Sep 2021 00:08:20 +1000 Subject: [PATCH 364/860] Add target_project and target_dataset config aliases for snapshots on BigQuery (#3834) * add bq alias for target_project and target_dataset * Update CHANGELOG.md add #3694 to changelog * Update CHANGELOG.md Be more specific about the change to bigquery synonym for schema only. * Set integration test bigquery configs to use alias Co-authored-by: Jeremy Cohen --- dbt/adapters/bigquery/connections.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 1d2c5c609..c6923ddc2 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -104,6 +104,8 @@ class BigQueryCredentials(Credentials): _ALIASES = { 'project': 'database', 'dataset': 'schema', + 'target_project': 'target_database', + 'target_dataset': 'target_schema', } @property From a883bcd8b5338f76819bbecb0a275c1d7c9b3c10 Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Fri, 3 Sep 2021 16:38:20 +0200 Subject: [PATCH 365/860] Add adapter_unique_id to invocation tracking (#3796) * Add properties + methods for adapter_unique_id * Turn on tracking --- dbt/adapters/bigquery/connections.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index c6923ddc2..010bcd74c 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -112,6 +112,10 @@ class BigQueryCredentials(Credentials): def type(self): return 'bigquery' + @property + def unique_field(self): + return self.database + def _connection_keys(self): return ('method', 'database', 'schema', 'location', 'priority', 'timeout_seconds', 'maximum_bytes_billed') From b408f03dcbb060998ef30b1de5db415b890e711a Mon Sep 17 00:00:00 2001 From: AndreasTA-AW <85551349+AndreasTA-AW@users.noreply.github.com> Date: Thu, 9 Sep 2021 12:01:02 +0200 Subject: [PATCH 366/860] =?UTF-8?q?#3682=20Changed=20how=20tables=20and=20?= =?UTF-8?q?views=20are=20generated=20to=20be=20able=20to=20use=20differen?= =?UTF-8?q?=E2=80=A6=20(#3691)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Changed how tables and views are generated to be able to use different options * 3682 added unit tests * 3682 had conflict in changelog and became a bit messy * 3682 Tested to add default kms to dataset and accidently pushed the changes --- dbt/adapters/bigquery/impl.py | 32 ++++++++++++++++++------ dbt/include/bigquery/macros/adapters.sql | 17 +++++++++---- 2 files changed, 36 insertions(+), 13 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index 2c5d94c66..b6580b0ff 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -769,13 +769,10 @@ def _get_catalog_schemas(self, manifest: Manifest) -> SchemaSearchMap: return result @available.parse(lambda *a, **k: {}) - def get_table_options( - self, config: Dict[str, Any], node: Dict[str, Any], temporary: bool + def get_common_options( + self, config: Dict[str, Any], node: Dict[str, Any], temporary: bool = False ) -> Dict[str, Any]: opts = {} - if temporary: - expiration = 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)' - opts['expiration_timestamp'] = expiration if (config.get('hours_to_expiration') is not None) and (not temporary): expiration = ( @@ -787,13 +784,25 @@ def get_table_options( description = sql_escape(node['description']) opts['description'] = '"""{}"""'.format(description) - if config.get('kms_key_name') is not None: - opts['kms_key_name'] = "'{}'".format(config.get('kms_key_name')) - if config.get('labels'): labels = config.get('labels', {}) opts['labels'] = list(labels.items()) + return opts + + @available.parse(lambda *a, **k: {}) + def get_table_options( + self, config: Dict[str, Any], node: Dict[str, Any], temporary: bool + ) -> Dict[str, Any]: + opts = self.get_common_options(config, node, temporary) + + if temporary: + expiration = 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)' + opts['expiration_timestamp'] = expiration + + if config.get('kms_key_name') is not None: + opts['kms_key_name'] = "'{}'".format(config.get('kms_key_name')) + if config.get('require_partition_filter'): opts['require_partition_filter'] = config.get( 'require_partition_filter') @@ -804,6 +813,13 @@ def get_table_options( return opts + @available.parse(lambda *a, **k: {}) + def get_view_options( + self, config: Dict[str, Any], node: Dict[str, Any] + ) -> Dict[str, Any]: + opts = self.get_common_options(config, node) + return opts + @available.parse_none def grant_access_to(self, entity, entity_type, role, grant_target_dict): """ diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql index d21d5f04e..03f70a1d3 100644 --- a/dbt/include/bigquery/macros/adapters.sql +++ b/dbt/include/bigquery/macros/adapters.sql @@ -27,10 +27,7 @@ {%- endmacro -%} - -{% macro bigquery_table_options(config, node, temporary) %} - {% set opts = adapter.get_table_options(config, node, temporary) %} - +{% macro bigquery_options(opts) %} {% set options -%} OPTIONS({% for opt_key, opt_val in opts.items() %} {{ opt_key }}={{ opt_val }}{{ "," if not loop.last }} @@ -39,6 +36,11 @@ {%- do return(options) -%} {%- endmacro -%} +{% macro bigquery_table_options(config, node, temporary) %} + {% set opts = adapter.get_table_options(config, node, temporary) %} + {%- do return(bigquery_options(opts)) -%} +{%- endmacro -%} + {% macro bigquery__create_table_as(temporary, relation, sql) -%} {%- set raw_partition_by = config.get('partition_by', none) -%} {%- set raw_cluster_by = config.get('cluster_by', none) -%} @@ -58,13 +60,18 @@ {%- endmacro -%} +{% macro bigquery_view_options(config, node) %} + {% set opts = adapter.get_view_options(config, node) %} + {%- do return(bigquery_options(opts)) -%} +{%- endmacro -%} + {% macro bigquery__create_view_as(relation, sql) -%} {%- set sql_header = config.get('sql_header', none) -%} {{ sql_header if sql_header is not none }} create or replace view {{ relation }} - {{ bigquery_table_options(config, model, temporary=false) }} + {{ bigquery_view_options(config, model) }} as {{ sql }}; {% endmacro %} From 42cd89d08e38a871743ffd49716708475bb48d1f Mon Sep 17 00:00:00 2001 From: Christophe Oudar Date: Wed, 15 Sep 2021 14:09:08 +0200 Subject: [PATCH 367/860] add support for execution project for BigQuery adapter (#3707) * add support for execution project for BigQuery adapter * Add integration test for BigQuery execution_project * review changes * Update changelog --- dbt/adapters/bigquery/connections.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py index 010bcd74c..9a9fed58c 100644 --- a/dbt/adapters/bigquery/connections.py +++ b/dbt/adapters/bigquery/connections.py @@ -83,6 +83,7 @@ class BigQueryCredentials(Credentials): # BigQuery allows an empty database / project, where it defers to the # environment for the project database: Optional[str] + execution_project: Optional[str] = None timeout_seconds: Optional[int] = 300 location: Optional[str] = None priority: Optional[Priority] = None @@ -130,6 +131,9 @@ def __pre_deserialize__(cls, d: Dict[Any, Any]) -> Dict[Any, Any]: if 'database' not in d: _, database = get_bigquery_defaults() d['database'] = database + # `execution_project` default to dataset/project + if 'execution_project' not in d: + d['execution_project'] = d['database'] return d @@ -252,12 +256,12 @@ def get_bigquery_client(cls, profile_credentials): cls.get_impersonated_bigquery_credentials(profile_credentials) else: creds = cls.get_bigquery_credentials(profile_credentials) - database = profile_credentials.database + execution_project = profile_credentials.execution_project location = getattr(profile_credentials, 'location', None) info = client_info.ClientInfo(user_agent=f'dbt-{dbt_version}') return google.cloud.bigquery.Client( - database, + execution_project, creds, location=location, client_info=info, From 3d461a526fe5f133f58a1c17a1c9c26524ce4f4a Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Fri, 17 Sep 2021 14:07:42 -0400 Subject: [PATCH 368/860] update develop (#3914) Co-authored-by: Jeremy Cohen Co-authored-by: sungchun12 Co-authored-by: Drew Banin Co-authored-by: Github Build Bot --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 278d33a8e..9aea90fcb 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.21.0b1' +version = '0.21.0b2' diff --git a/setup.py b/setup.py index 2a4655baf..a47da89e6 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.21.0b1" +package_version = "0.21.0b2" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From e7ab1874112e3e230dceb3c063a96fc2febf3fdb Mon Sep 17 00:00:00 2001 From: Gerda Shank Date: Tue, 24 Aug 2021 11:30:58 -0400 Subject: [PATCH 369/860] [#2990] Normalize global CLI args/flags --- dbt/adapters/bigquery/impl.py | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py index b6580b0ff..24aa1cb36 100644 --- a/dbt/adapters/bigquery/impl.py +++ b/dbt/adapters/bigquery/impl.py @@ -4,7 +4,6 @@ import dbt.deprecations import dbt.exceptions -import dbt.flags as flags import dbt.clients.gcloud import dbt.clients.agate_helper @@ -15,7 +14,6 @@ from dbt.adapters.bigquery.relation import BigQueryRelation from dbt.adapters.bigquery import BigQueryColumn from dbt.adapters.bigquery import BigQueryConnectionManager -from dbt.contracts.connection import Connection from dbt.contracts.graph.manifest import Manifest from dbt.logger import GLOBAL_LOGGER as logger, print_timestamped_line from dbt.utils import filter_null_values @@ -515,19 +513,6 @@ def execute_model(self, model, materialization, sql_override=None, if sql_override is None: sql_override = model.get('compiled_sql') - if flags.STRICT_MODE: - connection = self.connections.get_thread_connection() - if not isinstance(connection, Connection): - dbt.exceptions.raise_compiler_error( - f'Got {connection} - not a Connection!' - ) - model_uid = model.get('unique_id') - if connection.name != model_uid: - raise dbt.exceptions.InternalException( - f'Connection had name "{connection.name}", expected model ' - f'unique id of "{model_uid}"' - ) - if materialization == 'view': res = self._materialize_as_view(model) elif materialization == 'table': From 3ef6ce4d1aaab773ede7b30040f9d2f89785afcb Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Wed, 22 Sep 2021 13:25:46 -0400 Subject: [PATCH 370/860] Pull in changes for the `0.21.0rc1` release (#3935) Co-authored-by: Github Build Bot --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 9aea90fcb..cebffb393 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.21.0b2' +version = '0.21.0rc1' diff --git a/setup.py b/setup.py index a47da89e6..038033f5d 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.21.0b2" +package_version = "0.21.0rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From efa25ce85616a5209b901263716a4328099772cb Mon Sep 17 00:00:00 2001 From: Kyle Wigley Date: Wed, 22 Sep 2021 13:25:46 -0400 Subject: [PATCH 371/860] Pull in changes for the `0.21.0rc1` release (#3935) Co-authored-by: Github Build Bot --- dbt/adapters/bigquery/__version__.py | 2 +- setup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py index 9aea90fcb..cebffb393 100644 --- a/dbt/adapters/bigquery/__version__.py +++ b/dbt/adapters/bigquery/__version__.py @@ -1 +1 @@ -version = '0.21.0b2' +version = '0.21.0rc1' diff --git a/setup.py b/setup.py index a47da89e6..038033f5d 100644 --- a/setup.py +++ b/setup.py @@ -20,7 +20,7 @@ package_name = "dbt-bigquery" -package_version = "0.21.0b2" +package_version = "0.21.0rc1" description = """The bigquery adapter plugin for dbt (data build tool)""" this_directory = os.path.abspath(os.path.dirname(__file__)) From 7f02de1b43c5b4a99f6cd21ead3efa74508c4f3c Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Wed, 29 Sep 2021 11:26:19 +0200 Subject: [PATCH 372/860] Consolidate dbt-bigquery --- tests/__init__.py | 0 tests/integration/__init__.py | 0 .../bigquery-models/renamed_model.sql | 1 + .../bigquery-models/sources.yml | 6 + .../macros/rename_named_relation.sql | 6 + .../adapter_methods_test/models/expected.sql | 3 + .../adapter_methods_test/models/model.sql | 19 + .../adapter_methods_test/models/upstream.sql | 1 + .../adapter_methods_test/seed_bq.sql | 32 + .../test_adapter_methods.py | 103 + .../tests/get_columns_in_relation.sql | 7 + .../integration/aliases_test/macros/cast.sql | 13 + .../aliases_test/macros/expect_value.sql | 10 + .../models-dupe-custom-database/README.md | 2 + .../models-dupe-custom-database/model_a.sql | 1 + .../models-dupe-custom-database/model_b.sql | 1 + .../models-dupe-custom-database/schema.yml | 12 + .../aliases_test/models/alias_in_project.sql | 2 + .../models/alias_in_project_with_override.sql | 4 + .../aliases_test/models/foo_alias.sql | 9 + .../aliases_test/models/ref_foo_alias.sql | 16 + .../aliases_test/models/schema.yml | 22 + .../integration/aliases_test/test_aliases.py | 70 + tests/integration/base.py | 920 ++++++++ .../bigquery_test/adapter-models/schema.yml | 39 + .../bigquery_test/adapter-models/source.sql | 41 + .../test_flattened_get_columns_in_table.sql | 22 + .../test_get_columns_in_table.sql | 10 + .../expiring_table.sql | 1 + .../case-sensitive-models/fUnKyCaSe.sql | 5 + .../case-sensitive-schemas/model.sql | 5 + .../copy_bad_materialization.sql | 2 + .../copy-failing-models/original.sql | 1 + .../bigquery_test/copy-models/additional.sql | 1 + .../copy-models/copy_as_incremental.sql | 2 + .../copy-models/copy_as_several_tables.sql | 1 + .../copy-models/copy_as_table.sql | 1 + .../bigquery_test/copy-models/original.sql | 1 + .../bigquery_test/copy-models/schema.yml | 6 + .../bigquery_test/data/data_seed.csv | 5 + .../incremental_overwrite_date_expected.csv | 5 + .../incremental_overwrite_day_expected.csv | 5 + .../incremental_overwrite_range_expected.csv | 5 + .../incremental_overwrite_time_expected.csv | 5 + .../bigquery_test/data/merge_expected.csv | 7 + .../bigquery_test/dp-models/confirmation.sql | 20 + .../dp-models/confirmation_noconfig.sql | 20 + .../dp-models/events_20180101.sql | 4 + .../dp-models/events_20180102.sql | 4 + .../dp-models/events_20180103.sql | 4 + .../bigquery_test/dp-models/partitioned.sql | 16 + .../dp-models/partitioned_noconfig.sql | 7 + .../dp-models/partitioned_simple.sql | 14 + .../bigquery_test/dp-models/schema.yml | 33 + .../execution-project-models/model.sql | 1 + .../execution-project-models/schema.yml | 10 + .../incremental_merge_range.sql | 46 + .../incremental_merge_time.sql | 42 + .../incremental_overwrite_date.sql | 41 + .../incremental_overwrite_day.sql | 41 + .../incremental_overwrite_partitions.sql | 42 + .../incremental_overwrite_range.sql | 46 + .../incremental_overwrite_time.sql | 42 + .../bigquery_test/location-models/model.sql | 1 + .../macros/partition_metadata.sql | 19 + .../bigquery_test/macros/test_creation.sql | 18 + .../macros/test_int_inference.sql | 36 + .../macros/test_project_for_job_id.sql | 7 + .../bigquery_test/macros/wrapped_macros.sql | 43 + .../bigquery_test/models/clustered_model.sql | 10 + .../bigquery_test/models/fUnKyCaSe.sql | 1 + .../bigquery_test/models/labeled_model.sql | 8 + .../models/multi_clustered_model.sql | 10 + .../models/partitioned_model.sql | 9 + .../bigquery_test/models/schema.yml | 60 + .../bigquery_test/models/sql_header_model.sql | 14 + .../models/sql_header_model_incr.sql | 17 + ...sql_header_model_incr_insert_overwrite.sql | 33 + ...der_model_incr_insert_overwrite_static.sql | 32 + .../bigquery_test/models/table_model.sql | 8 + .../bigquery_test/models/view_model.sql | 14 + .../partition-models/my_model.sql | 15 + .../bigquery_test/partition-models/schema.yml | 6 + .../test_bigquery_adapter_functions.py | 77 + .../test_bigquery_adapter_specific.py | 40 + .../test_bigquery_case_sensitive.py | 34 + .../test_bigquery_changing_partitions.py | 183 ++ .../test_bigquery_copy_failing_models.py | 36 + .../test_bigquery_copy_models.py | 41 + .../test_bigquery_date_partitioning.py | 46 + .../test_bigquery_execution_project.py | 23 + .../test_bigquery_location_change.py | 31 + .../test_bigquery_query_results.py | 24 + .../test_bigquery_repeated_records.py | 68 + .../test_bigquery_update_columns.py | 85 + .../test_incremental_strategies.py | 29 + .../test_simple_bigquery_view.py | 71 + .../description_table.sql | 9 + .../description_table.yml | 7 + .../policy_tag_table.sql | 9 + .../policy_tag_table.yml | 8 + .../models/model.sql | 8 + .../test_changing_relation_type.py | 45 + .../integration/column_quoting/data/seed.csv | 4 + .../column_quoting/models-unquoted/model.sql | 12 + .../column_quoting/models/model.sql | 12 + .../column_quoting/test_column_quotes.py | 78 + .../column_type_tests/bq_models/model.sql | 5 + .../column_type_tests/bq_models/schema.yml | 10 + .../bq_models_alter_type/altered_schema.yml | 10 + .../bq_models_alter_type/model.sql | 6 + .../macros/test_alter_column_type.sql | 5 + .../column_type_tests/macros/test_is_type.sql | 72 + .../test_alter_column_types.py | 28 + .../column_type_tests/test_column_types.py | 21 + .../dbt_debug_test/models/model.sql | 1 + .../integration/dbt_debug_test/test_debug.py | 33 + .../defer_state_test/data/seed.csv | 3 + .../macros/infinite_macros.sql | 13 + .../defer_state_test/macros/macros.sql | 3 + .../models/ephemeral_model.sql | 2 + .../defer_state_test/models/exposures.yml | 8 + .../defer_state_test/models/schema.yml | 9 + .../defer_state_test/models/table_model.sql | 5 + .../defer_state_test/models/view_model.sql | 4 + .../defer_state_test/test_defer_state.py | 85 + .../adapter-macro-macros/macros.sql | 17 + .../adapter-macro-models-package/model.sql | 4 + .../adapter-macro-models/model.sql | 4 + .../deprecation_tests/test_deprecations.py | 62 + .../assets/lorem-ipsum.txt | 1 + .../bq_models/clustered.sql | 9 + .../bq_models/multi_clustered.sql | 9 + .../bq_models/nested_table.sql | 15 + .../bq_models/nested_view.sql | 7 + .../docs_generate_tests/bq_models/schema.yml | 44 + .../bq_models_noschema/disabled.sql | 2 + .../bq_models_noschema/model.sql | 2 + .../cross_db_models/model.sql | 8 + .../cross_db_models/schema.yml | 21 + .../fail_macros/failure.sql | 3 + .../docs_generate_tests/macros/dummy_test.sql | 9 + .../docs_generate_tests/macros/macro.md | 8 + .../docs_generate_tests/macros/schema.yml | 10 + .../docs_generate_tests/models/model.sql | 8 + .../docs_generate_tests/models/readme.md | 1 + .../docs_generate_tests/models/schema.yml | 82 + .../models/second_model.sql | 8 + .../docs_generate_tests/ref_models/docs.md | 31 + .../ref_models/ephemeral_copy.sql | 7 + .../ref_models/ephemeral_summary.sql | 9 + .../docs_generate_tests/ref_models/schema.yml | 48 + .../ref_models/view_summary.sql | 8 + .../docs_generate_tests/rs_models/model.sql | 7 + .../docs_generate_tests/rs_models/schema.yml | 16 + .../docs_generate_tests/seed/schema.yml | 15 + .../docs_generate_tests/seed/seed.csv | 2 + .../snapshot/snapshot_seed.sql | 11 + .../docs_generate_tests/test_docs_generate.py | 1981 +++++++++++++++++ .../trivial_models/model.sql | 1 + .../hook_tests/data/example_seed.csv | 4 + .../hook_tests/macros/before-and-after-bq.sql | 30 + tests/integration/hook_tests/macros/hook.sql | 3 + tests/integration/hook_tests/models/hooks.sql | 2 + .../hook_tests/seed-models-bq/schema.yml | 7 + .../hook_tests/seed_model_bigquery.sql | 18 + .../hook_tests/seed_run_bigquery.sql | 18 + .../hook_tests/test_model_hooks_bq.py | 150 ++ .../hook_tests/test_run_hooks_bq.py | 102 + .../models/incremental_append_new_columns.sql | 29 + .../incremental_append_new_columns_target.sql | 19 + .../models/incremental_fail.sql | 19 + .../models/incremental_ignore.sql | 19 + .../models/incremental_ignore_target.sql | 15 + .../models/incremental_sync_all_columns.sql | 31 + .../incremental_sync_all_columns_target.sql | 20 + .../models/model_a.sql | 22 + .../models/schema.yml | 54 + .../test_incremental_schema.py | 130 ++ .../tests/select_from_a.sql | 1 + ...ct_from_incremental_append_new_columns.sql | 1 + ..._incremental_append_new_columns_target.sql | 1 + .../tests/select_from_incremental_ignore.sql | 1 + .../select_from_incremental_ignore_target.sql | 1 + ...lect_from_incremental_sync_all_columns.sql | 1 + ...om_incremental_sync_all_columns_target.sql | 1 + .../override_database_test/data/seed.csv | 6 + .../models/subfolder/view_3.sql | 1 + .../models/subfolder/view_4.sql | 5 + .../override_database_test/models/view_1.sql | 7 + .../override_database_test/models/view_2.sql | 6 + .../test_override_database.py | 176 ++ .../persist_docs_tests/data/seed.csv | 3 + .../models-bigquery-nested/schema.yml | 19 + .../table_model_nested.sql | 8 + .../view_model_nested.sql | 8 + .../models-column-missing/missing_column.sql | 2 + .../models-column-missing/schema.yml | 8 + .../persist_docs_tests/models/my_fun_docs.md | 10 + .../models/no_docs_model.sql | 1 + .../persist_docs_tests/models/schema.yml | 70 + .../persist_docs_tests/models/table_model.sql | 2 + .../persist_docs_tests/models/view_model.sql | 2 + .../persist_docs_tests/test_persist_docs.py | 209 ++ .../query_comments_test/macros/macro.sql | 25 + .../query_comments_test/models/x.sql | 25 + .../test_query_comments.py | 167 ++ .../models-v2/bq-models/ephemeral_copy.sql | 8 + .../models-v2/bq-models/schema.yml | 30 + .../models-v2/bq-models/table_summary.sql | 9 + .../schema_tests_test/models-v2/seed/seed.csv | 5 + .../schema_tests_test/test_schema_v2_tests.py | 55 + .../incremental_update_cols.sql | 17 + .../models/advanced_incremental.sql | 17 + .../simple_copy_test/models/compound_sort.sql | 9 + .../simple_copy_test/models/disabled.sql | 8 + .../simple_copy_test/models/empty.sql | 5 + .../simple_copy_test/models/get_and_ref.sql | 3 + .../simple_copy_test/models/incremental.sql | 11 + .../models/interleaved_sort.sql | 9 + .../simple_copy_test/models/materialized.sql | 12 + .../simple_copy_test/models/schema.yml | 7 + .../simple_copy_test/models/view_model.sql | 7 + .../simple_copy_test/seed-initial/seed.csv | 101 + .../simple_copy_test/seed-update/seed.csv | 201 ++ .../seeds-merge-cols-initial/seed.csv | 101 + .../expected_result.csv | 201 ++ .../seeds-merge-cols-update/seed.csv | 204 ++ .../simple_copy_test/test_simple_copy.py | 96 + .../data-config/seed_disabled.csv | 21 + .../data-config/seed_enabled.csv | 21 + .../data-config/seed_tricky.csv | 7 + .../simple_seed_test/macros/schema_test.sql | 22 + .../simple_seed_test/models-bq/schema.yml | 47 + .../test_seed_type_override.py | 60 + .../add_column_to_source_bq.sql | 56 + .../check_snapshots_test_current.sql | 51 + .../check-snapshots/check_cols_cycle.sql | 33 + .../custom-snapshot-macros/custom.sql | 18 + .../simple_snapshot_test/data/seed.csv | 4 + .../simple_snapshot_test/data/seed_newcol.csv | 4 + .../invalidate_bigquery.sql | 12 + .../macros/test_no_overlaps.sql | 85 + .../simple_snapshot_test/models/.gitkeep | 0 .../models/ref_snapshot.sql | 1 + .../simple_snapshot_test/models/schema.yml | 5 + .../integration/simple_snapshot_test/seed.sql | 220 ++ .../simple_snapshot_test/seed_bq.sql | 81 + .../test-check-col-snapshots-bq/snapshot.sql | 29 + .../snapshot.sql | 9 + .../test-check-col-snapshots/snapshot.sql | 28 + .../test-snapshots-bq/snapshot.sql | 19 + .../test-snapshots-checkall/snapshot.sql | 4 + .../test-snapshots-pg/snapshot.sql | 19 + .../snapshot.sql | 41 + .../test-snapshots-select/snapshot.sql | 44 + .../test_simple_snapshot.py | 566 +++++ .../test_snapshot_check_cols.py | 40 + .../simple_snapshot_test/update_bq.sql | 78 + .../data/expected_multi_source.csv | 4 + .../sources_test/data/other_source_table.csv | 4 + .../sources_test/data/other_table.csv | 4 + .../integration/sources_test/data/source.csv | 101 + .../sources_test/error_models/model.sql | 1 + .../sources_test/error_models/schema.yml | 12 + .../sources_test/filtered_models/schema.yml | 18 + .../integration/sources_test/macros/macro.sql | 14 + .../malformed_models/descendant_model.sql | 1 + .../sources_test/malformed_models/schema.yml | 14 + .../malformed_schema_tests/model.sql | 1 + .../malformed_schema_tests/schema.yml | 14 + .../sources_test/models/descendant_model.sql | 1 + .../sources_test/models/ephemeral_model.sql | 3 + .../models/multi_source_model.sql | 2 + .../models/nonsource_descendant.sql | 1 + .../sources_test/models/schema.yml | 77 + .../sources_test/models/view_model.sql | 3 + tests/integration/sources_test/seed.sql | 113 + .../integration/sources_test/test_sources.py | 182 ++ .../models-bq/statement_actual.sql | 23 + .../integration/statement_test/seed/seed.csv | 101 + .../seed/statement_expected.csv | 3 + .../statement_test/test_statements.py | 36 + .../expected/expected_accepted_values.csv | 3 + .../data/expected/expected_failing_test.csv | 11 + ...expected_not_null_problematic_model_id.csv | 3 + .../expected_unique_problematic_model_id.csv | 3 + .../store_test_failures_tests/data/people.csv | 11 + .../models/fine_model.sql | 1 + ...odel_but_with_a_no_good_very_long_name.sql | 1 + .../models/problematic_model.sql | 11 + .../models/schema.yml | 40 + .../test_store_test_failures.py | 60 + .../tests/failing_test.sql | 1 + .../tests/passing_test.sql | 2 + tests/unit/__init__.py | 0 tests/unit/mock_adapter.py | 81 + tests/unit/test_bigquery_adapter.py | 1018 +++++++++ tests/unit/utils.py | 371 +++ 299 files changed, 12547 insertions(+) create mode 100644 tests/__init__.py create mode 100644 tests/integration/__init__.py create mode 100644 tests/integration/adapter_methods_test/bigquery-models/renamed_model.sql create mode 100644 tests/integration/adapter_methods_test/bigquery-models/sources.yml create mode 100644 tests/integration/adapter_methods_test/macros/rename_named_relation.sql create mode 100644 tests/integration/adapter_methods_test/models/expected.sql create mode 100644 tests/integration/adapter_methods_test/models/model.sql create mode 100644 tests/integration/adapter_methods_test/models/upstream.sql create mode 100644 tests/integration/adapter_methods_test/seed_bq.sql create mode 100644 tests/integration/adapter_methods_test/test_adapter_methods.py create mode 100644 tests/integration/adapter_methods_test/tests/get_columns_in_relation.sql create mode 100644 tests/integration/aliases_test/macros/cast.sql create mode 100644 tests/integration/aliases_test/macros/expect_value.sql create mode 100644 tests/integration/aliases_test/models-dupe-custom-database/README.md create mode 100644 tests/integration/aliases_test/models-dupe-custom-database/model_a.sql create mode 100644 tests/integration/aliases_test/models-dupe-custom-database/model_b.sql create mode 100644 tests/integration/aliases_test/models-dupe-custom-database/schema.yml create mode 100644 tests/integration/aliases_test/models/alias_in_project.sql create mode 100644 tests/integration/aliases_test/models/alias_in_project_with_override.sql create mode 100644 tests/integration/aliases_test/models/foo_alias.sql create mode 100644 tests/integration/aliases_test/models/ref_foo_alias.sql create mode 100644 tests/integration/aliases_test/models/schema.yml create mode 100644 tests/integration/aliases_test/test_aliases.py create mode 100644 tests/integration/base.py create mode 100644 tests/integration/bigquery_test/adapter-models/schema.yml create mode 100644 tests/integration/bigquery_test/adapter-models/source.sql create mode 100644 tests/integration/bigquery_test/adapter-models/test_flattened_get_columns_in_table.sql create mode 100644 tests/integration/bigquery_test/adapter-models/test_get_columns_in_table.sql create mode 100644 tests/integration/bigquery_test/adapter-specific-models/expiring_table.sql create mode 100644 tests/integration/bigquery_test/case-sensitive-models/fUnKyCaSe.sql create mode 100644 tests/integration/bigquery_test/case-sensitive-schemas/model.sql create mode 100644 tests/integration/bigquery_test/copy-failing-models/copy_bad_materialization.sql create mode 100644 tests/integration/bigquery_test/copy-failing-models/original.sql create mode 100644 tests/integration/bigquery_test/copy-models/additional.sql create mode 100644 tests/integration/bigquery_test/copy-models/copy_as_incremental.sql create mode 100644 tests/integration/bigquery_test/copy-models/copy_as_several_tables.sql create mode 100644 tests/integration/bigquery_test/copy-models/copy_as_table.sql create mode 100644 tests/integration/bigquery_test/copy-models/original.sql create mode 100644 tests/integration/bigquery_test/copy-models/schema.yml create mode 100644 tests/integration/bigquery_test/data/data_seed.csv create mode 100644 tests/integration/bigquery_test/data/incremental_overwrite_date_expected.csv create mode 100644 tests/integration/bigquery_test/data/incremental_overwrite_day_expected.csv create mode 100644 tests/integration/bigquery_test/data/incremental_overwrite_range_expected.csv create mode 100644 tests/integration/bigquery_test/data/incremental_overwrite_time_expected.csv create mode 100644 tests/integration/bigquery_test/data/merge_expected.csv create mode 100644 tests/integration/bigquery_test/dp-models/confirmation.sql create mode 100644 tests/integration/bigquery_test/dp-models/confirmation_noconfig.sql create mode 100644 tests/integration/bigquery_test/dp-models/events_20180101.sql create mode 100644 tests/integration/bigquery_test/dp-models/events_20180102.sql create mode 100644 tests/integration/bigquery_test/dp-models/events_20180103.sql create mode 100644 tests/integration/bigquery_test/dp-models/partitioned.sql create mode 100644 tests/integration/bigquery_test/dp-models/partitioned_noconfig.sql create mode 100644 tests/integration/bigquery_test/dp-models/partitioned_simple.sql create mode 100644 tests/integration/bigquery_test/dp-models/schema.yml create mode 100644 tests/integration/bigquery_test/execution-project-models/model.sql create mode 100644 tests/integration/bigquery_test/execution-project-models/schema.yml create mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_range.sql create mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_time.sql create mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_date.sql create mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day.sql create mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_partitions.sql create mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_range.sql create mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_time.sql create mode 100644 tests/integration/bigquery_test/location-models/model.sql create mode 100644 tests/integration/bigquery_test/macros/partition_metadata.sql create mode 100644 tests/integration/bigquery_test/macros/test_creation.sql create mode 100644 tests/integration/bigquery_test/macros/test_int_inference.sql create mode 100644 tests/integration/bigquery_test/macros/test_project_for_job_id.sql create mode 100644 tests/integration/bigquery_test/macros/wrapped_macros.sql create mode 100644 tests/integration/bigquery_test/models/clustered_model.sql create mode 100644 tests/integration/bigquery_test/models/fUnKyCaSe.sql create mode 100644 tests/integration/bigquery_test/models/labeled_model.sql create mode 100644 tests/integration/bigquery_test/models/multi_clustered_model.sql create mode 100644 tests/integration/bigquery_test/models/partitioned_model.sql create mode 100644 tests/integration/bigquery_test/models/schema.yml create mode 100644 tests/integration/bigquery_test/models/sql_header_model.sql create mode 100644 tests/integration/bigquery_test/models/sql_header_model_incr.sql create mode 100644 tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite.sql create mode 100644 tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite_static.sql create mode 100644 tests/integration/bigquery_test/models/table_model.sql create mode 100644 tests/integration/bigquery_test/models/view_model.sql create mode 100644 tests/integration/bigquery_test/partition-models/my_model.sql create mode 100644 tests/integration/bigquery_test/partition-models/schema.yml create mode 100644 tests/integration/bigquery_test/test_bigquery_adapter_functions.py create mode 100644 tests/integration/bigquery_test/test_bigquery_adapter_specific.py create mode 100644 tests/integration/bigquery_test/test_bigquery_case_sensitive.py create mode 100644 tests/integration/bigquery_test/test_bigquery_changing_partitions.py create mode 100644 tests/integration/bigquery_test/test_bigquery_copy_failing_models.py create mode 100644 tests/integration/bigquery_test/test_bigquery_copy_models.py create mode 100644 tests/integration/bigquery_test/test_bigquery_date_partitioning.py create mode 100644 tests/integration/bigquery_test/test_bigquery_execution_project.py create mode 100644 tests/integration/bigquery_test/test_bigquery_location_change.py create mode 100644 tests/integration/bigquery_test/test_bigquery_query_results.py create mode 100644 tests/integration/bigquery_test/test_bigquery_repeated_records.py create mode 100644 tests/integration/bigquery_test/test_bigquery_update_columns.py create mode 100644 tests/integration/bigquery_test/test_incremental_strategies.py create mode 100644 tests/integration/bigquery_test/test_simple_bigquery_view.py create mode 100644 tests/integration/bigquery_test/update-column-description/description_table.sql create mode 100644 tests/integration/bigquery_test/update-column-description/description_table.yml create mode 100644 tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.sql create mode 100644 tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.yml create mode 100644 tests/integration/changing_relation_type_test/models/model.sql create mode 100644 tests/integration/changing_relation_type_test/test_changing_relation_type.py create mode 100644 tests/integration/column_quoting/data/seed.csv create mode 100644 tests/integration/column_quoting/models-unquoted/model.sql create mode 100644 tests/integration/column_quoting/models/model.sql create mode 100644 tests/integration/column_quoting/test_column_quotes.py create mode 100644 tests/integration/column_type_tests/bq_models/model.sql create mode 100644 tests/integration/column_type_tests/bq_models/schema.yml create mode 100644 tests/integration/column_type_tests/bq_models_alter_type/altered_schema.yml create mode 100644 tests/integration/column_type_tests/bq_models_alter_type/model.sql create mode 100644 tests/integration/column_type_tests/macros/test_alter_column_type.sql create mode 100644 tests/integration/column_type_tests/macros/test_is_type.sql create mode 100644 tests/integration/column_type_tests/test_alter_column_types.py create mode 100644 tests/integration/column_type_tests/test_column_types.py create mode 100644 tests/integration/dbt_debug_test/models/model.sql create mode 100644 tests/integration/dbt_debug_test/test_debug.py create mode 100644 tests/integration/defer_state_test/data/seed.csv create mode 100644 tests/integration/defer_state_test/macros/infinite_macros.sql create mode 100644 tests/integration/defer_state_test/macros/macros.sql create mode 100644 tests/integration/defer_state_test/models/ephemeral_model.sql create mode 100644 tests/integration/defer_state_test/models/exposures.yml create mode 100644 tests/integration/defer_state_test/models/schema.yml create mode 100644 tests/integration/defer_state_test/models/table_model.sql create mode 100644 tests/integration/defer_state_test/models/view_model.sql create mode 100644 tests/integration/defer_state_test/test_defer_state.py create mode 100644 tests/integration/deprecation_tests/adapter-macro-macros/macros.sql create mode 100644 tests/integration/deprecation_tests/adapter-macro-models-package/model.sql create mode 100644 tests/integration/deprecation_tests/adapter-macro-models/model.sql create mode 100644 tests/integration/deprecation_tests/test_deprecations.py create mode 100644 tests/integration/docs_generate_tests/assets/lorem-ipsum.txt create mode 100644 tests/integration/docs_generate_tests/bq_models/clustered.sql create mode 100644 tests/integration/docs_generate_tests/bq_models/multi_clustered.sql create mode 100644 tests/integration/docs_generate_tests/bq_models/nested_table.sql create mode 100644 tests/integration/docs_generate_tests/bq_models/nested_view.sql create mode 100644 tests/integration/docs_generate_tests/bq_models/schema.yml create mode 100644 tests/integration/docs_generate_tests/bq_models_noschema/disabled.sql create mode 100644 tests/integration/docs_generate_tests/bq_models_noschema/model.sql create mode 100644 tests/integration/docs_generate_tests/cross_db_models/model.sql create mode 100644 tests/integration/docs_generate_tests/cross_db_models/schema.yml create mode 100644 tests/integration/docs_generate_tests/fail_macros/failure.sql create mode 100644 tests/integration/docs_generate_tests/macros/dummy_test.sql create mode 100644 tests/integration/docs_generate_tests/macros/macro.md create mode 100644 tests/integration/docs_generate_tests/macros/schema.yml create mode 100644 tests/integration/docs_generate_tests/models/model.sql create mode 100644 tests/integration/docs_generate_tests/models/readme.md create mode 100644 tests/integration/docs_generate_tests/models/schema.yml create mode 100644 tests/integration/docs_generate_tests/models/second_model.sql create mode 100644 tests/integration/docs_generate_tests/ref_models/docs.md create mode 100644 tests/integration/docs_generate_tests/ref_models/ephemeral_copy.sql create mode 100644 tests/integration/docs_generate_tests/ref_models/ephemeral_summary.sql create mode 100644 tests/integration/docs_generate_tests/ref_models/schema.yml create mode 100644 tests/integration/docs_generate_tests/ref_models/view_summary.sql create mode 100644 tests/integration/docs_generate_tests/rs_models/model.sql create mode 100644 tests/integration/docs_generate_tests/rs_models/schema.yml create mode 100644 tests/integration/docs_generate_tests/seed/schema.yml create mode 100644 tests/integration/docs_generate_tests/seed/seed.csv create mode 100644 tests/integration/docs_generate_tests/snapshot/snapshot_seed.sql create mode 100644 tests/integration/docs_generate_tests/test_docs_generate.py create mode 100644 tests/integration/docs_generate_tests/trivial_models/model.sql create mode 100644 tests/integration/hook_tests/data/example_seed.csv create mode 100644 tests/integration/hook_tests/macros/before-and-after-bq.sql create mode 100644 tests/integration/hook_tests/macros/hook.sql create mode 100644 tests/integration/hook_tests/models/hooks.sql create mode 100644 tests/integration/hook_tests/seed-models-bq/schema.yml create mode 100644 tests/integration/hook_tests/seed_model_bigquery.sql create mode 100644 tests/integration/hook_tests/seed_run_bigquery.sql create mode 100644 tests/integration/hook_tests/test_model_hooks_bq.py create mode 100644 tests/integration/hook_tests/test_run_hooks_bq.py create mode 100644 tests/integration/incremental_schema_tests/models/incremental_append_new_columns.sql create mode 100644 tests/integration/incremental_schema_tests/models/incremental_append_new_columns_target.sql create mode 100644 tests/integration/incremental_schema_tests/models/incremental_fail.sql create mode 100644 tests/integration/incremental_schema_tests/models/incremental_ignore.sql create mode 100644 tests/integration/incremental_schema_tests/models/incremental_ignore_target.sql create mode 100644 tests/integration/incremental_schema_tests/models/incremental_sync_all_columns.sql create mode 100644 tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_target.sql create mode 100644 tests/integration/incremental_schema_tests/models/model_a.sql create mode 100644 tests/integration/incremental_schema_tests/models/schema.yml create mode 100644 tests/integration/incremental_schema_tests/test_incremental_schema.py create mode 100644 tests/integration/incremental_schema_tests/tests/select_from_a.sql create mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns.sql create mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_target.sql create mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore.sql create mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore_target.sql create mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns.sql create mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns_target.sql create mode 100644 tests/integration/override_database_test/data/seed.csv create mode 100644 tests/integration/override_database_test/models/subfolder/view_3.sql create mode 100644 tests/integration/override_database_test/models/subfolder/view_4.sql create mode 100644 tests/integration/override_database_test/models/view_1.sql create mode 100644 tests/integration/override_database_test/models/view_2.sql create mode 100644 tests/integration/override_database_test/test_override_database.py create mode 100644 tests/integration/persist_docs_tests/data/seed.csv create mode 100644 tests/integration/persist_docs_tests/models-bigquery-nested/schema.yml create mode 100644 tests/integration/persist_docs_tests/models-bigquery-nested/table_model_nested.sql create mode 100644 tests/integration/persist_docs_tests/models-bigquery-nested/view_model_nested.sql create mode 100644 tests/integration/persist_docs_tests/models-column-missing/missing_column.sql create mode 100644 tests/integration/persist_docs_tests/models-column-missing/schema.yml create mode 100644 tests/integration/persist_docs_tests/models/my_fun_docs.md create mode 100644 tests/integration/persist_docs_tests/models/no_docs_model.sql create mode 100644 tests/integration/persist_docs_tests/models/schema.yml create mode 100644 tests/integration/persist_docs_tests/models/table_model.sql create mode 100644 tests/integration/persist_docs_tests/models/view_model.sql create mode 100644 tests/integration/persist_docs_tests/test_persist_docs.py create mode 100644 tests/integration/query_comments_test/macros/macro.sql create mode 100644 tests/integration/query_comments_test/models/x.sql create mode 100644 tests/integration/query_comments_test/test_query_comments.py create mode 100644 tests/integration/schema_tests_test/models-v2/bq-models/ephemeral_copy.sql create mode 100644 tests/integration/schema_tests_test/models-v2/bq-models/schema.yml create mode 100644 tests/integration/schema_tests_test/models-v2/bq-models/table_summary.sql create mode 100644 tests/integration/schema_tests_test/models-v2/seed/seed.csv create mode 100644 tests/integration/schema_tests_test/test_schema_v2_tests.py create mode 100644 tests/integration/simple_copy_test/models-merge-update/incremental_update_cols.sql create mode 100644 tests/integration/simple_copy_test/models/advanced_incremental.sql create mode 100644 tests/integration/simple_copy_test/models/compound_sort.sql create mode 100644 tests/integration/simple_copy_test/models/disabled.sql create mode 100644 tests/integration/simple_copy_test/models/empty.sql create mode 100644 tests/integration/simple_copy_test/models/get_and_ref.sql create mode 100644 tests/integration/simple_copy_test/models/incremental.sql create mode 100644 tests/integration/simple_copy_test/models/interleaved_sort.sql create mode 100644 tests/integration/simple_copy_test/models/materialized.sql create mode 100644 tests/integration/simple_copy_test/models/schema.yml create mode 100644 tests/integration/simple_copy_test/models/view_model.sql create mode 100644 tests/integration/simple_copy_test/seed-initial/seed.csv create mode 100644 tests/integration/simple_copy_test/seed-update/seed.csv create mode 100644 tests/integration/simple_copy_test/seeds-merge-cols-initial/seed.csv create mode 100644 tests/integration/simple_copy_test/seeds-merge-cols-update/expected_result.csv create mode 100644 tests/integration/simple_copy_test/seeds-merge-cols-update/seed.csv create mode 100644 tests/integration/simple_copy_test/test_simple_copy.py create mode 100644 tests/integration/simple_seed_test/data-config/seed_disabled.csv create mode 100644 tests/integration/simple_seed_test/data-config/seed_enabled.csv create mode 100644 tests/integration/simple_seed_test/data-config/seed_tricky.csv create mode 100644 tests/integration/simple_seed_test/macros/schema_test.sql create mode 100644 tests/integration/simple_seed_test/models-bq/schema.yml create mode 100644 tests/integration/simple_seed_test/test_seed_type_override.py create mode 100644 tests/integration/simple_snapshot_test/add_column_to_source_bq.sql create mode 100644 tests/integration/simple_snapshot_test/check-snapshots-expected/check_snapshots_test_current.sql create mode 100644 tests/integration/simple_snapshot_test/check-snapshots/check_cols_cycle.sql create mode 100644 tests/integration/simple_snapshot_test/custom-snapshot-macros/custom.sql create mode 100644 tests/integration/simple_snapshot_test/data/seed.csv create mode 100644 tests/integration/simple_snapshot_test/data/seed_newcol.csv create mode 100644 tests/integration/simple_snapshot_test/invalidate_bigquery.sql create mode 100644 tests/integration/simple_snapshot_test/macros/test_no_overlaps.sql create mode 100644 tests/integration/simple_snapshot_test/models/.gitkeep create mode 100644 tests/integration/simple_snapshot_test/models/ref_snapshot.sql create mode 100644 tests/integration/simple_snapshot_test/models/schema.yml create mode 100644 tests/integration/simple_snapshot_test/seed.sql create mode 100644 tests/integration/simple_snapshot_test/seed_bq.sql create mode 100644 tests/integration/simple_snapshot_test/test-check-col-snapshots-bq/snapshot.sql create mode 100644 tests/integration/simple_snapshot_test/test-check-col-snapshots-noconfig/snapshot.sql create mode 100644 tests/integration/simple_snapshot_test/test-check-col-snapshots/snapshot.sql create mode 100644 tests/integration/simple_snapshot_test/test-snapshots-bq/snapshot.sql create mode 100644 tests/integration/simple_snapshot_test/test-snapshots-checkall/snapshot.sql create mode 100644 tests/integration/simple_snapshot_test/test-snapshots-pg/snapshot.sql create mode 100644 tests/integration/simple_snapshot_test/test-snapshots-select-noconfig/snapshot.sql create mode 100644 tests/integration/simple_snapshot_test/test-snapshots-select/snapshot.sql create mode 100644 tests/integration/simple_snapshot_test/test_simple_snapshot.py create mode 100644 tests/integration/simple_snapshot_test/test_snapshot_check_cols.py create mode 100644 tests/integration/simple_snapshot_test/update_bq.sql create mode 100644 tests/integration/sources_test/data/expected_multi_source.csv create mode 100644 tests/integration/sources_test/data/other_source_table.csv create mode 100644 tests/integration/sources_test/data/other_table.csv create mode 100644 tests/integration/sources_test/data/source.csv create mode 100644 tests/integration/sources_test/error_models/model.sql create mode 100644 tests/integration/sources_test/error_models/schema.yml create mode 100644 tests/integration/sources_test/filtered_models/schema.yml create mode 100644 tests/integration/sources_test/macros/macro.sql create mode 100644 tests/integration/sources_test/malformed_models/descendant_model.sql create mode 100644 tests/integration/sources_test/malformed_models/schema.yml create mode 100644 tests/integration/sources_test/malformed_schema_tests/model.sql create mode 100644 tests/integration/sources_test/malformed_schema_tests/schema.yml create mode 100644 tests/integration/sources_test/models/descendant_model.sql create mode 100644 tests/integration/sources_test/models/ephemeral_model.sql create mode 100644 tests/integration/sources_test/models/multi_source_model.sql create mode 100644 tests/integration/sources_test/models/nonsource_descendant.sql create mode 100644 tests/integration/sources_test/models/schema.yml create mode 100644 tests/integration/sources_test/models/view_model.sql create mode 100644 tests/integration/sources_test/seed.sql create mode 100644 tests/integration/sources_test/test_sources.py create mode 100644 tests/integration/statement_test/models-bq/statement_actual.sql create mode 100644 tests/integration/statement_test/seed/seed.csv create mode 100644 tests/integration/statement_test/seed/statement_expected.csv create mode 100644 tests/integration/statement_test/test_statements.py create mode 100644 tests/integration/store_test_failures_tests/data/expected/expected_accepted_values.csv create mode 100644 tests/integration/store_test_failures_tests/data/expected/expected_failing_test.csv create mode 100644 tests/integration/store_test_failures_tests/data/expected/expected_not_null_problematic_model_id.csv create mode 100644 tests/integration/store_test_failures_tests/data/expected/expected_unique_problematic_model_id.csv create mode 100644 tests/integration/store_test_failures_tests/data/people.csv create mode 100644 tests/integration/store_test_failures_tests/models/fine_model.sql create mode 100644 tests/integration/store_test_failures_tests/models/fine_model_but_with_a_no_good_very_long_name.sql create mode 100644 tests/integration/store_test_failures_tests/models/problematic_model.sql create mode 100644 tests/integration/store_test_failures_tests/models/schema.yml create mode 100644 tests/integration/store_test_failures_tests/test_store_test_failures.py create mode 100644 tests/integration/store_test_failures_tests/tests/failing_test.sql create mode 100644 tests/integration/store_test_failures_tests/tests/passing_test.sql create mode 100644 tests/unit/__init__.py create mode 100644 tests/unit/mock_adapter.py create mode 100644 tests/unit/test_bigquery_adapter.py create mode 100644 tests/unit/utils.py diff --git a/tests/__init__.py b/tests/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/tests/integration/__init__.py b/tests/integration/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/tests/integration/adapter_methods_test/bigquery-models/renamed_model.sql b/tests/integration/adapter_methods_test/bigquery-models/renamed_model.sql new file mode 100644 index 000000000..3c030d920 --- /dev/null +++ b/tests/integration/adapter_methods_test/bigquery-models/renamed_model.sql @@ -0,0 +1 @@ +select * from {{ source('test_source', 'renamed_seed') }} \ No newline at end of file diff --git a/tests/integration/adapter_methods_test/bigquery-models/sources.yml b/tests/integration/adapter_methods_test/bigquery-models/sources.yml new file mode 100644 index 000000000..cb74cde9c --- /dev/null +++ b/tests/integration/adapter_methods_test/bigquery-models/sources.yml @@ -0,0 +1,6 @@ +version: 2 +sources: + - name: test_source + schema: "{{ target.schema }}" + tables: + - name: renamed_seed diff --git a/tests/integration/adapter_methods_test/macros/rename_named_relation.sql b/tests/integration/adapter_methods_test/macros/rename_named_relation.sql new file mode 100644 index 000000000..253e1e0ad --- /dev/null +++ b/tests/integration/adapter_methods_test/macros/rename_named_relation.sql @@ -0,0 +1,6 @@ +-- Macro to rename a relation +{% macro rename_named_relation(from_name, to_name) %} +{%- set from_relation = api.Relation.create(database=target.database, schema=target.schema, identifier=from_name, type='table') -%} +{%- set to_relation = api.Relation.create(database=target.database, schema=target.schema, identifier=to_name, type='table') -%} +{% do adapter.rename_relation(from_relation, to_relation) %} +{% endmacro %} \ No newline at end of file diff --git a/tests/integration/adapter_methods_test/models/expected.sql b/tests/integration/adapter_methods_test/models/expected.sql new file mode 100644 index 000000000..ddebb5f5d --- /dev/null +++ b/tests/integration/adapter_methods_test/models/expected.sql @@ -0,0 +1,3 @@ +-- make sure this runs after 'model' +-- {{ ref('model') }} +select 2 as id diff --git a/tests/integration/adapter_methods_test/models/model.sql b/tests/integration/adapter_methods_test/models/model.sql new file mode 100644 index 000000000..bcf11d81c --- /dev/null +++ b/tests/integration/adapter_methods_test/models/model.sql @@ -0,0 +1,19 @@ + +{% set upstream = ref('upstream') %} + +{% if execute %} + {# don't ever do any of this #} + {%- do adapter.drop_schema(upstream) -%} + {% set existing = adapter.get_relation(upstream.database, upstream.schema, upstream.identifier) %} + {% if existing is not none %} + {% do exceptions.raise_compiler_error('expected ' ~ ' to not exist, but it did') %} + {% endif %} + + {%- do adapter.create_schema(upstream) -%} + + {% set sql = create_view_as(upstream, 'select 2 as id') %} + {% do run_query(sql) %} +{% endif %} + + +select * from {{ upstream }} diff --git a/tests/integration/adapter_methods_test/models/upstream.sql b/tests/integration/adapter_methods_test/models/upstream.sql new file mode 100644 index 000000000..43258a714 --- /dev/null +++ b/tests/integration/adapter_methods_test/models/upstream.sql @@ -0,0 +1 @@ +select 1 as id diff --git a/tests/integration/adapter_methods_test/seed_bq.sql b/tests/integration/adapter_methods_test/seed_bq.sql new file mode 100644 index 000000000..71a9a78c6 --- /dev/null +++ b/tests/integration/adapter_methods_test/seed_bq.sql @@ -0,0 +1,32 @@ +create table {database}.{schema}.seed ( + id INT64, + first_name STRING, + last_name STRING, + email STRING, + gender STRING, + ip_address STRING, + updated_at TIMESTAMP +); + +-- seed inserts +insert {database}.{schema}.seed (id, first_name, last_name, email, gender, ip_address, updated_at) values +(1, 'Judith', 'Kennedy', 'jkennedy0@phpbb.com', 'Female', '54.60.24.128', '2015-12-24 12:19:28'), +(2, 'Arthur', 'Kelly', 'akelly1@eepurl.com', 'Male', '62.56.24.215', '2015-10-28 16:22:15'), +(3, 'Rachel', 'Moreno', 'rmoreno2@msu.edu', 'Female', '31.222.249.23', '2016-04-05 02:05:30'), +(4, 'Ralph', 'Turner', 'rturner3@hp.com', 'Male', '157.83.76.114', '2016-08-08 00:06:51'), +(5, 'Laura', 'Gonzales', 'lgonzales4@howstuffworks.com', 'Female', '30.54.105.168', '2016-09-01 08:25:38'), +(6, 'Katherine', 'Lopez', 'klopez5@yahoo.co.jp', 'Female', '169.138.46.89', '2016-08-30 18:52:11'), +(7, 'Jeremy', 'Hamilton', 'jhamilton6@mozilla.org', 'Male', '231.189.13.133', '2016-07-17 02:09:46'), +(8, 'Heather', 'Rose', 'hrose7@goodreads.com', 'Female', '87.165.201.65', '2015-12-29 22:03:56'), +(9, 'Gregory', 'Kelly', 'gkelly8@trellian.com', 'Male', '154.209.99.7', '2016-03-24 21:18:16'), +(10, 'Rachel', 'Lopez', 'rlopez9@themeforest.net', 'Female', '237.165.82.71', '2016-08-20 15:44:49'), +(11, 'Donna', 'Welch', 'dwelcha@shutterfly.com', 'Female', '103.33.110.138', '2016-02-27 01:41:48'), +(12, 'Russell', 'Lawrence', 'rlawrenceb@qq.com', 'Male', '189.115.73.4', '2016-06-11 03:07:09'), +(13, 'Michelle', 'Montgomery', 'mmontgomeryc@scientificamerican.com', 'Female', '243.220.95.82', '2016-06-18 16:27:19'), +(14, 'Walter', 'Castillo', 'wcastillod@pagesperso-orange.fr', 'Male', '71.159.238.196', '2016-10-06 01:55:44'), +(15, 'Robin', 'Mills', 'rmillse@vkontakte.ru', 'Female', '172.190.5.50', '2016-10-31 11:41:21'), +(16, 'Raymond', 'Holmes', 'rholmesf@usgs.gov', 'Male', '148.153.166.95', '2016-10-03 08:16:38'), +(17, 'Gary', 'Bishop', 'gbishopg@plala.or.jp', 'Male', '161.108.182.13', '2016-08-29 19:35:20'), +(18, 'Anna', 'Riley', 'arileyh@nasa.gov', 'Female', '253.31.108.22', '2015-12-11 04:34:27'), +(19, 'Sarah', 'Knight', 'sknighti@foxnews.com', 'Female', '222.220.3.177', '2016-09-26 00:49:06'), +(20, 'Phyllis', 'Fox', null, 'Female', '163.191.232.95', '2016-08-21 10:35:19'); diff --git a/tests/integration/adapter_methods_test/test_adapter_methods.py b/tests/integration/adapter_methods_test/test_adapter_methods.py new file mode 100644 index 000000000..a7613ca1e --- /dev/null +++ b/tests/integration/adapter_methods_test/test_adapter_methods.py @@ -0,0 +1,103 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import yaml + + +class TestBaseCaching(DBTIntegrationTest): + @property + def schema(self): + return "adapter_methods_caching" + + @property + def models(self): + return "models" + + @property + def project_config(self): + return { + 'config-version': 2, + 'test-paths': ['tests'] + } + + @use_profile('bigquery') + def test_bigquery_adapter_methods(self): + self.run_dbt(['compile']) # trigger any compile-time issues + self.run_dbt() + self.assertTablesEqual('model', 'expected') + + +class TestRenameRelation(DBTIntegrationTest): + @property + def schema(self): + return "adapter_methods_rename_relation" + + @property + def models(self): + return 'bigquery-models' + + @property + def project_config(self): + return { + 'config-version': 2, + 'source-paths': ['models'] + } + + @use_profile('bigquery') + def test_bigquery_adapter_methods(self): + self.run_dbt(['compile']) # trigger any compile-time issues + self.run_sql_file("seed_bq.sql") + self.run_dbt(['seed']) + rename_relation_args = yaml.safe_dump({ + 'from_name': 'seed', + 'to_name': 'renamed_seed', + }) + self.run_dbt(['run-operation', 'rename_named_relation', '--args', rename_relation_args]) + self.run_dbt() + + +class TestGrantAccess(DBTIntegrationTest): + @property + def schema(self): + return "adapter_methods_grant_access" + + @property + def models(self): + return 'bigquery-models' + + @property + def project_config(self): + return { + 'config-version': 2, + 'source-paths': ['models'] + } + + @use_profile('bigquery') + def test_bigquery_adapter_methods(self): + from dbt.adapters.bigquery import GrantTarget + from google.cloud.bigquery import AccessEntry + + self.run_dbt(['compile']) # trigger any compile-time issues + self.run_sql_file("seed_bq.sql") + self.run_dbt(['seed']) + + ae_role = "READER" + ae_entity = "user@email.com" + ae_entity_type = "userByEmail" + ae_grant_target_dict = { + 'project': self.default_database, + 'dataset': self.unique_schema() + } + self.adapter.grant_access_to(ae_entity, ae_entity_type, ae_role, ae_grant_target_dict) + + conn = self.adapter.connections.get_thread_connection() + client = conn.handle + + grant_target = GrantTarget.from_dict(ae_grant_target_dict) + dataset = client.get_dataset( + self.adapter.connections.dataset_from_id(grant_target.render()) + ) + + expected_access_entry = AccessEntry(ae_role, ae_entity_type, ae_entity) + self.assertTrue(expected_access_entry in dataset.access_entries) + + unexpected_access_entry = AccessEntry(ae_role, ae_entity_type, "unexpected@email.com") + self.assertFalse(unexpected_access_entry in dataset.access_entries) diff --git a/tests/integration/adapter_methods_test/tests/get_columns_in_relation.sql b/tests/integration/adapter_methods_test/tests/get_columns_in_relation.sql new file mode 100644 index 000000000..6507b52ec --- /dev/null +++ b/tests/integration/adapter_methods_test/tests/get_columns_in_relation.sql @@ -0,0 +1,7 @@ +{% set columns = adapter.get_columns_in_relation(ref('model')) %} +{% set limit_query = 0 %} +{% if (columns | length) == 0 %} + {% set limit_query = 1 %} +{% endif %} + +select 1 as id limit {{ limit_query }} diff --git a/tests/integration/aliases_test/macros/cast.sql b/tests/integration/aliases_test/macros/cast.sql new file mode 100644 index 000000000..584492077 --- /dev/null +++ b/tests/integration/aliases_test/macros/cast.sql @@ -0,0 +1,13 @@ + + +{% macro string_literal(s) -%} + {{ adapter.dispatch('string_literal', macro_namespace='test')(s) }} +{%- endmacro %} + +{% macro default__string_literal(s) %} + '{{ s }}'::text +{% endmacro %} + +{% macro bigquery__string_literal(s) %} + cast('{{ s }}' as string) +{% endmacro %} diff --git a/tests/integration/aliases_test/macros/expect_value.sql b/tests/integration/aliases_test/macros/expect_value.sql new file mode 100644 index 000000000..0ee66151f --- /dev/null +++ b/tests/integration/aliases_test/macros/expect_value.sql @@ -0,0 +1,10 @@ + +-- cross-db compatible test, similar to accepted_values + +{% test expect_value(model, field, value) %} + +select * +from {{ model }} +where {{ field }} != '{{ value }}' + +{% endtest %} diff --git a/tests/integration/aliases_test/models-dupe-custom-database/README.md b/tests/integration/aliases_test/models-dupe-custom-database/README.md new file mode 100644 index 000000000..7d0add229 --- /dev/null +++ b/tests/integration/aliases_test/models-dupe-custom-database/README.md @@ -0,0 +1,2 @@ +these should succeed, as both models have the same alias, +but they are configured to be built in _different_ schemas diff --git a/tests/integration/aliases_test/models-dupe-custom-database/model_a.sql b/tests/integration/aliases_test/models-dupe-custom-database/model_a.sql new file mode 100644 index 000000000..9ce91382c --- /dev/null +++ b/tests/integration/aliases_test/models-dupe-custom-database/model_a.sql @@ -0,0 +1 @@ +select {{ string_literal(this.name) }} as tablename diff --git a/tests/integration/aliases_test/models-dupe-custom-database/model_b.sql b/tests/integration/aliases_test/models-dupe-custom-database/model_b.sql new file mode 100644 index 000000000..9ce91382c --- /dev/null +++ b/tests/integration/aliases_test/models-dupe-custom-database/model_b.sql @@ -0,0 +1 @@ +select {{ string_literal(this.name) }} as tablename diff --git a/tests/integration/aliases_test/models-dupe-custom-database/schema.yml b/tests/integration/aliases_test/models-dupe-custom-database/schema.yml new file mode 100644 index 000000000..a7613882c --- /dev/null +++ b/tests/integration/aliases_test/models-dupe-custom-database/schema.yml @@ -0,0 +1,12 @@ +version: 2 +models: +- name: model_a + tests: + - expect_value: + field: tablename + value: duped_alias +- name: model_b + tests: + - expect_value: + field: tablename + value: duped_alias diff --git a/tests/integration/aliases_test/models/alias_in_project.sql b/tests/integration/aliases_test/models/alias_in_project.sql new file mode 100644 index 000000000..aa9ecd0bf --- /dev/null +++ b/tests/integration/aliases_test/models/alias_in_project.sql @@ -0,0 +1,2 @@ + +select {{ string_literal(this.name) }} as tablename diff --git a/tests/integration/aliases_test/models/alias_in_project_with_override.sql b/tests/integration/aliases_test/models/alias_in_project_with_override.sql new file mode 100644 index 000000000..67cb93d93 --- /dev/null +++ b/tests/integration/aliases_test/models/alias_in_project_with_override.sql @@ -0,0 +1,4 @@ + +{{ config(alias='override_alias') }} + +select {{ string_literal(this.name) }} as tablename diff --git a/tests/integration/aliases_test/models/foo_alias.sql b/tests/integration/aliases_test/models/foo_alias.sql new file mode 100644 index 000000000..3f7d16467 --- /dev/null +++ b/tests/integration/aliases_test/models/foo_alias.sql @@ -0,0 +1,9 @@ + +{{ + config( + alias='foo', + materialized='table' + ) +}} + +select {{ string_literal(this.name) }} as tablename diff --git a/tests/integration/aliases_test/models/ref_foo_alias.sql b/tests/integration/aliases_test/models/ref_foo_alias.sql new file mode 100644 index 000000000..e01463bb7 --- /dev/null +++ b/tests/integration/aliases_test/models/ref_foo_alias.sql @@ -0,0 +1,16 @@ + +{{ + config( + materialized='table' + ) +}} + +with trigger_ref as ( + + -- we should still be able to ref a model by its filepath + select * from {{ ref('foo_alias') }} + +) + +-- this name should still be the filename +select {{ string_literal(this.name) }} as tablename diff --git a/tests/integration/aliases_test/models/schema.yml b/tests/integration/aliases_test/models/schema.yml new file mode 100644 index 000000000..b3a82faad --- /dev/null +++ b/tests/integration/aliases_test/models/schema.yml @@ -0,0 +1,22 @@ +version: 2 +models: +- name: foo_alias + tests: + - expect_value: + field: tablename + value: foo +- name: ref_foo_alias + tests: + - expect_value: + field: tablename + value: ref_foo_alias +- name: alias_in_project + tests: + - expect_value: + field: tablename + value: project_alias +- name: alias_in_project_with_override + tests: + - expect_value: + field: tablename + value: override_alias diff --git a/tests/integration/aliases_test/test_aliases.py b/tests/integration/aliases_test/test_aliases.py new file mode 100644 index 000000000..404c89499 --- /dev/null +++ b/tests/integration/aliases_test/test_aliases.py @@ -0,0 +1,70 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + + +class TestAliases(DBTIntegrationTest): + @property + def schema(self): + return "aliases" + + @property + def models(self): + return "models" + + @property + def project_config(self): + return { + 'config-version': 2, + "macro-paths": ['macros'], + "models": { + "test": { + "alias_in_project": { + "alias": 'project_alias', + }, + "alias_in_project_with_override": { + "alias": 'project_alias', + }, + } + } + } + + @use_profile('bigquery') + def test__alias_model_name_bigquery(self): + results = self.run_dbt(['run']) + self.assertEqual(len(results), 4) + self.run_dbt(['test']) + + +class TestSameAliasDifferentDatabases(DBTIntegrationTest): + setup_alternate_db = True + + @property + def schema(self): + return "aliases_026" + + @property + def models(self): + return "models-dupe-custom-database" + + @property + def project_config(self): + return { + 'config-version': 2, + "macro-paths": ['macros'], + 'models': { + 'test': { + 'alias': 'duped_alias', + 'model_b': { + 'database': self.alternative_database, + }, + }, + } + } + + @use_profile('bigquery') + def test__bigquery_same_alias_succeeds_in_different_schemas(self): + results = self.run_dbt(['run']) + self.assertEqual(len(results), 2) + res = self.run_dbt(['test']) + + # Make extra sure the tests ran + self.assertTrue(len(res) > 0) diff --git a/tests/integration/base.py b/tests/integration/base.py new file mode 100644 index 000000000..2f4c0fd35 --- /dev/null +++ b/tests/integration/base.py @@ -0,0 +1,920 @@ +import json +import os +import io +import random +import shutil +import sys +import tempfile +import traceback +import unittest +from contextlib import contextmanager +from datetime import datetime +from functools import wraps + +import pytest +import yaml +from unittest.mock import patch + +import dbt.main as dbt +from dbt import flags +from dbt.deprecations import reset_deprecations +from dbt.adapters.factory import get_adapter, reset_adapters, register_adapter +from dbt.clients.jinja import template_cache +from dbt.config import RuntimeConfig +from dbt.context import providers +from dbt.logger import GLOBAL_LOGGER as logger, log_manager +from dbt.contracts.graph.manifest import Manifest + + +INITIAL_ROOT = os.getcwd() + + +def normalize(path): + """On windows, neither is enough on its own: + + >>> normcase('C:\\documents/ALL CAPS/subdir\\..') + 'c:\\documents\\all caps\\subdir\\..' + >>> normpath('C:\\documents/ALL CAPS/subdir\\..') + 'C:\\documents\\ALL CAPS' + >>> normpath(normcase('C:\\documents/ALL CAPS/subdir\\..')) + 'c:\\documents\\all caps' + """ + return os.path.normcase(os.path.normpath(path)) + + +class Normalized: + def __init__(self, value): + self.value = value + + def __repr__(self): + return f'Normalized({self.value!r})' + + def __str__(self): + return f'Normalized({self.value!s})' + + def __eq__(self, other): + return normalize(self.value) == normalize(other) + + +class FakeArgs: + def __init__(self): + self.threads = 1 + self.data = False + self.defer = False + self.schema = True + self.full_refresh = False + self.models = None + self.select = None + self.exclude = None + self.single_threaded = False + self.selector_name = None + self.state = None + self.defer = None + + +class TestArgs: + def __init__(self, kwargs): + self.which = 'run' + self.single_threaded = False + self.profiles_dir = None + self.project_dir = None + self.__dict__.update(kwargs) + + +def _profile_from_test_name(test_name): + adapter_names = ('bigquery',) + adapters_in_name = sum(x in test_name for x in adapter_names) + if adapters_in_name != 1: + raise ValueError( + 'test names must have exactly 1 profile choice embedded, {} has {}' + .format(test_name, adapters_in_name) + ) + + for adapter_name in adapter_names: + if adapter_name in test_name: + return adapter_name + + raise ValueError( + 'could not find adapter name in test name {}'.format(test_name) + ) + + +def _pytest_test_name(): + return os.environ['PYTEST_CURRENT_TEST'].split()[0] + + +def _pytest_get_test_root(): + test_path = _pytest_test_name().split('::')[0] + relative_to = INITIAL_ROOT + head = os.path.relpath(test_path, relative_to) + + path_parts = [] + while head: + head, tail = os.path.split(head) + path_parts.append(tail) + path_parts.reverse() + # dbt tests are all of the form 'tests/integration/suite_name' + target = os.path.join(*path_parts[:5]) # TODO: try to not hard code this + return os.path.join(relative_to, target) + + +def _really_makedirs(path): + while not os.path.exists(path): + try: + os.makedirs(path) + except EnvironmentError: + raise + + +class DBTIntegrationTest(unittest.TestCase): + CREATE_SCHEMA_STATEMENT = 'CREATE SCHEMA {}' + DROP_SCHEMA_STATEMENT = 'DROP SCHEMA IF EXISTS {} CASCADE' + + _randint = random.randint(0, 9999) + _runtime_timedelta = (datetime.utcnow() - datetime(1970, 1, 1, 0, 0, 0)) + _runtime = ( + (int(_runtime_timedelta.total_seconds() * 1e6)) + + _runtime_timedelta.microseconds + ) + + prefix = f'test{_runtime}{_randint:04}' + setup_alternate_db = False + + def bigquery_profile(self): + credentials_json_str = os.getenv('BIGQUERY_TEST_SERVICE_ACCOUNT_JSON').replace("'", '') + credentials = json.loads(credentials_json_str) + project_id = credentials.get('project_id') + + return { + 'config': { + 'send_anonymous_usage_stats': False + }, + 'test': { + 'outputs': { + 'default2': { + 'type': 'bigquery', + 'method': 'service-account-json', + 'threads': 1, + 'project': project_id, + 'keyfile_json': credentials, + 'schema': self.unique_schema(), + }, + 'alternate': { + 'type': 'bigquery', + 'method': 'service-account-json', + 'threads': 1, + 'project': project_id, + 'keyfile_json': credentials, + 'schema': self.unique_schema(), + 'execution_project': self.alternative_database, + }, + }, + 'target': 'default2' + } + } + + @property + def packages_config(self): + return None + + @property + def selectors_config(self): + return None + + def unique_schema(self): + schema = self.schema + + to_return = "{}_{}".format(self.prefix, schema) + + return to_return.lower() + + @property + def default_database(self): + database = self.config.credentials.database + return database + + @property + def alternative_database(self): + return os.environ['BIGQUERY_TEST_ALT_DATABASE'] + + def get_profile(self, adapter_type): + if adapter_type == 'bigquery': + return self.bigquery_profile() + else: + raise ValueError('invalid adapter type {}'.format(adapter_type)) + + def _pick_profile(self): + test_name = self.id().split('.')[-1] + return _profile_from_test_name(test_name) + + def _symlink_test_folders(self): + for entry in os.listdir(self.test_original_source_path): + src = os.path.join(self.test_original_source_path, entry) + tst = os.path.join(self.test_root_dir, entry) + if os.path.isdir(src) or src.endswith('.sql'): + # symlink all sql files and all directories. + os.symlink(src, tst) + os.symlink(self._logs_dir, os.path.join(self.test_root_dir, 'logs')) + + @property + def test_root_realpath(self): + if sys.platform == 'darwin': + return os.path.realpath(self.test_root_dir) + else: + return self.test_root_dir + + def _generate_test_root_dir(self): + return normalize(tempfile.mkdtemp(prefix='dbt-int-test-')) + + def setUp(self): + self.dbt_core_install_root = os.path.dirname(dbt.__file__) + log_manager.reset_handlers() + self.initial_dir = INITIAL_ROOT + os.chdir(self.initial_dir) + # before we go anywhere, collect the initial path info + self._logs_dir = os.path.join(self.initial_dir, 'logs', self.prefix) + _really_makedirs(self._logs_dir) + self.test_original_source_path = _pytest_get_test_root() + self.test_root_dir = self._generate_test_root_dir() + + os.chdir(self.test_root_dir) + try: + self._symlink_test_folders() + except Exception as exc: + msg = '\n\t'.join(( + 'Failed to symlink test folders!', + 'initial_dir={0.initial_dir}', + 'test_original_source_path={0.test_original_source_path}', + 'test_root_dir={0.test_root_dir}' + )).format(self) + logger.exception(msg) + + # if logging isn't set up, I still really want this message. + print(msg) + traceback.print_exc() + + raise + + self._created_schemas = set() + reset_deprecations() + template_cache.clear() + + self.use_profile(self._pick_profile()) + self.use_default_project() + self.set_packages() + self.set_selectors() + self.load_config() + + def use_default_project(self, overrides=None): + # create a dbt_project.yml + base_project_config = { + 'name': 'test', + 'version': '1.0', + 'config-version': 2, + 'test-paths': [], + 'source-paths': [self.models], + 'profile': 'test', + } + + project_config = {} + project_config.update(base_project_config) + project_config.update(self.project_config) + project_config.update(overrides or {}) + + with open("dbt_project.yml", 'w') as f: + yaml.safe_dump(project_config, f, default_flow_style=True) + + def use_profile(self, adapter_type): + self.adapter_type = adapter_type + + profile_config = {} + default_profile_config = self.get_profile(adapter_type) + + profile_config.update(default_profile_config) + profile_config.update(self.profile_config) + + if not os.path.exists(self.test_root_dir): + os.makedirs(self.test_root_dir) + + flags.PROFILES_DIR = self.test_root_dir + profiles_path = os.path.join(self.test_root_dir, 'profiles.yml') + with open(profiles_path, 'w') as f: + yaml.safe_dump(profile_config, f, default_flow_style=True) + self._profile_config = profile_config + + def set_packages(self): + if self.packages_config is not None: + with open('packages.yml', 'w') as f: + yaml.safe_dump(self.packages_config, f, default_flow_style=True) + + def set_selectors(self): + if self.selectors_config is not None: + with open('selectors.yml', 'w') as f: + yaml.safe_dump(self.selectors_config, f, default_flow_style=True) + + def load_config(self): + # we've written our profile and project. Now we want to instantiate a + # fresh adapter for the tests. + # it's important to use a different connection handle here so + # we don't look into an incomplete transaction + kwargs = { + 'profile': None, + 'profiles_dir': self.test_root_dir, + 'target': None, + } + + config = RuntimeConfig.from_args(TestArgs(kwargs)) + + register_adapter(config) + adapter = get_adapter(config) + adapter.cleanup_connections() + self.adapter_type = adapter.type() + self.adapter = adapter + self.config = config + + self._drop_schemas() + self._create_schemas() + + def quote_as_configured(self, value, quote_key): + return self.adapter.quote_as_configured(value, quote_key) + + def tearDown(self): + # get any current run adapter and clean up its connections before we + # reset them. It'll probably be different from ours because + # handle_and_check() calls reset_adapters(). + register_adapter(self.config) + adapter = get_adapter(self.config) + if adapter is not self.adapter: + adapter.cleanup_connections() + if not hasattr(self, 'adapter'): + self.adapter = adapter + + self._drop_schemas() + + self.adapter.cleanup_connections() + reset_adapters() + os.chdir(INITIAL_ROOT) + try: + shutil.rmtree(self.test_root_dir) + except EnvironmentError: + logger.exception('Could not clean up after test - {} not removable' + .format(self.test_root_dir)) + + def _get_schema_fqn(self, database, schema): + schema_fqn = self.quote_as_configured(schema, 'schema') + return schema_fqn + + def _create_schema_named(self, database, schema): + relation = self.adapter.Relation.create(database=database, schema=schema) + self.adapter.create_schema(relation) + + def _drop_schema_named(self, database, schema): + relation = self.adapter.Relation.create(database=database, schema=schema) + self.adapter.drop_schema(relation) + + def _create_schemas(self): + schema = self.unique_schema() + with self.adapter.connection_named('__test'): + self._create_schema_named(self.default_database, schema) + + def _drop_schemas(self): + with self.adapter.connection_named('__test'): + schema = self.unique_schema() + self._drop_schema_named(self.default_database, schema) + if self.setup_alternate_db and self.alternative_database: + self._drop_schema_named(self.alternative_database, schema) + + @property + def project_config(self): + return { + 'config-version': 2, + } + + @property + def profile_config(self): + return {} + + def run_dbt(self, args=None, expect_pass=True, profiles_dir=True): + res, success = self.run_dbt_and_check(args=args, profiles_dir=profiles_dir) + self.assertEqual( + success, expect_pass, + "dbt exit state did not match expected") + + return res + + + def run_dbt_and_capture(self, *args, **kwargs): + try: + initial_stdout = log_manager.stdout + initial_stderr = log_manager.stderr + stringbuf = io.StringIO() + log_manager.set_output_stream(stringbuf) + + res = self.run_dbt(*args, **kwargs) + stdout = stringbuf.getvalue() + + finally: + log_manager.set_output_stream(initial_stdout, initial_stderr) + + return res, stdout + + def run_dbt_and_check(self, args=None, profiles_dir=True): + log_manager.reset_handlers() + if args is None: + args = ["run"] + + final_args = [] + + if os.getenv('DBT_TEST_SINGLE_THREADED') in ('y', 'Y', '1'): + final_args.append('--single-threaded') + + final_args.extend(args) + + if profiles_dir: + final_args.extend(['--profiles-dir', self.test_root_dir]) + final_args.append('--log-cache-events') + + logger.info("Invoking dbt with {}".format(final_args)) + return dbt.handle_and_check(final_args) + + def run_sql_file(self, path, kwargs=None): + with open(path, 'r') as f: + statements = f.read().split(";") + for statement in statements: + self.run_sql(statement, kwargs=kwargs) + + def transform_sql(self, query, kwargs=None): + to_return = query + + base_kwargs = { + 'schema': self.unique_schema(), + 'database': self.adapter.quote(self.default_database), + } + if kwargs is None: + kwargs = {} + base_kwargs.update(kwargs) + + to_return = to_return.format(**base_kwargs) + + return to_return + + def run_sql_bigquery(self, sql, fetch): + """Run an SQL query on a bigquery adapter. No cursors, transactions, + etc. to worry about""" + + do_fetch = fetch != 'None' + _, res = self.adapter.execute(sql, fetch=do_fetch) + + # convert dataframe to matrix-ish repr + if fetch == 'one': + return res[0] + else: + return list(res) + + def run_sql(self, query, fetch='None', kwargs=None, connection_name=None): + if connection_name is None: + connection_name = '__test' + + if query.strip() == "": + return + + sql = self.transform_sql(query, kwargs=kwargs) + + with self.get_connection(connection_name) as conn: + logger.debug('test connection "{}" executing: {}'.format(conn.name, sql)) + return self.run_sql_bigquery(sql, fetch) + + def _ilike(self, target, value): + return "{} ilike '{}'".format(target, value) + + def get_many_table_columns_bigquery(self, tables, schema, database=None): + result = [] + for table in tables: + relation = self._make_relation(table, schema, database) + columns = self.adapter.get_columns_in_relation(relation) + for col in columns: + result.append((table, col.column, col.dtype, col.char_size)) + return result + + def get_many_table_columns(self, tables, schema, database=None): + result = self.get_many_table_columns_bigquery(tables, schema, database) + result.sort(key=lambda x: '{}.{}'.format(x[0], x[1])) + return result + + def filter_many_columns(self, column): + if len(column) == 3: + table_name, column_name, data_type = column + char_size = None + else: + table_name, column_name, data_type, char_size = column + return (table_name, column_name, data_type, char_size) + + @contextmanager + def get_connection(self, name=None): + """Create a test connection context where all executed macros, etc will + get self.adapter as the adapter. + + This allows tests to run normal adapter macros as if reset_adapters() + were not called by handle_and_check (for asserts, etc) + """ + if name is None: + name = '__test' + with patch.object(providers, 'get_adapter', return_value=self.adapter): + with self.adapter.connection_named(name): + conn = self.adapter.connections.get_thread_connection() + yield conn + + def get_relation_columns(self, relation): + with self.get_connection(): + columns = self.adapter.get_columns_in_relation(relation) + + return sorted(((c.name, c.dtype, c.char_size) for c in columns), + key=lambda x: x[0]) + + def get_table_columns(self, table, schema=None, database=None): + schema = self.unique_schema() if schema is None else schema + database = self.default_database if database is None else database + relation = self.adapter.Relation.create( + database=database, + schema=schema, + identifier=table, + type='table', + quote_policy=self.config.quoting + ) + return self.get_relation_columns(relation) + + def get_table_columns_as_dict(self, tables, schema=None): + col_matrix = self.get_many_table_columns(tables, schema) + res = {} + for row in col_matrix: + table_name = row[0] + col_def = row[1:] + if table_name not in res: + res[table_name] = [] + res[table_name].append(col_def) + return res + + def get_models_in_schema(self, schema=None): + schema = self.unique_schema() if schema is None else schema + sql = """ + select table_name, + case when table_type = 'BASE TABLE' then 'table' + when table_type = 'VIEW' then 'view' + else table_type + end as materialization + from information_schema.tables + where {} + order by table_name + """ + + sql = sql.format(self._ilike('table_schema', schema)) + result = self.run_sql(sql, fetch='all') + + return {model_name: materialization for (model_name, materialization) in result} + + def _assertTablesEqualSql(self, relation_a, relation_b, columns=None): + if columns is None: + columns = self.get_relation_columns(relation_a) + column_names = [c[0] for c in columns] + + sql = self.adapter.get_rows_different_sql( + relation_a, relation_b, column_names + ) + + return sql + + def assertTablesEqual(self, table_a, table_b, + table_a_schema=None, table_b_schema=None, + table_a_db=None, table_b_db=None): + if table_a_schema is None: + table_a_schema = self.unique_schema() + + if table_b_schema is None: + table_b_schema = self.unique_schema() + + if table_a_db is None: + table_a_db = self.default_database + + if table_b_db is None: + table_b_db = self.default_database + + relation_a = self._make_relation(table_a, table_a_schema, table_a_db) + relation_b = self._make_relation(table_b, table_b_schema, table_b_db) + + self._assertTableColumnsEqual(relation_a, relation_b) + + sql = self._assertTablesEqualSql(relation_a, relation_b) + result = self.run_sql(sql, fetch='one') + + self.assertEqual( + result[0], + 0, + 'row_count_difference nonzero: ' + sql + ) + self.assertEqual( + result[1], + 0, + 'num_mismatched nonzero: ' + sql + ) + + def _make_relation(self, identifier, schema=None, database=None): + if schema is None: + schema = self.unique_schema() + if database is None: + database = self.default_database + return self.adapter.Relation.create( + database=database, + schema=schema, + identifier=identifier, + quote_policy=self.config.quoting + ) + + def get_many_relation_columns(self, relations): + """Returns a dict of (datbase, schema) -> (dict of (table_name -> list of columns)) + """ + schema_fqns = {} + for rel in relations: + this_schema = schema_fqns.setdefault((rel.database, rel.schema), []) + this_schema.append(rel.identifier) + + column_specs = {} + for key, tables in schema_fqns.items(): + database, schema = key + columns = self.get_many_table_columns(tables, schema, database=database) + table_columns = {} + for col in columns: + table_columns.setdefault(col[0], []).append(col[1:]) + for rel_name, columns in table_columns.items(): + key = (database, schema, rel_name) + column_specs[key] = columns + + return column_specs + + def assertManyRelationsEqual(self, relations, default_schema=None, default_database=None): + if default_schema is None: + default_schema = self.unique_schema() + if default_database is None: + default_database = self.default_database + + specs = [] + for relation in relations: + if not isinstance(relation, (tuple, list)): + relation = [relation] + + assert len(relation) <= 3 + + if len(relation) == 3: + relation = self._make_relation(*relation) + elif len(relation) == 2: + relation = self._make_relation(relation[0], relation[1], default_database) + elif len(relation) == 1: + relation = self._make_relation(relation[0], default_schema, default_database) + else: + raise ValueError('relation must be a sequence of 1, 2, or 3 values') + + specs.append(relation) + + with self.get_connection(): + column_specs = self.get_many_relation_columns(specs) + + # make sure everyone has equal column definitions + first_columns = None + for relation in specs: + key = (relation.database, relation.schema, relation.identifier) + # get a good error here instead of a hard-to-diagnose KeyError + self.assertIn(key, column_specs, f'No columns found for {key}') + columns = column_specs[key] + if first_columns is None: + first_columns = columns + else: + self.assertEqual( + first_columns, columns, + '{} did not match {}'.format(str(specs[0]), str(relation)) + ) + + # make sure everyone has the same data. if we got here, everyone had + # the same column specs! + first_relation = None + for relation in specs: + if first_relation is None: + first_relation = relation + else: + sql = self._assertTablesEqualSql(first_relation, relation, + columns=first_columns) + result = self.run_sql(sql, fetch='one') + + self.assertEqual( + result[0], + 0, + 'row_count_difference nonzero: ' + sql + ) + self.assertEqual( + result[1], + 0, + 'num_mismatched nonzero: ' + sql + ) + + def assertManyTablesEqual(self, *args): + schema = self.unique_schema() + + all_tables = [] + for table_equivalencies in args: + all_tables += list(table_equivalencies) + + all_cols = self.get_table_columns_as_dict(all_tables, schema) + + for table_equivalencies in args: + first_table = table_equivalencies[0] + first_relation = self._make_relation(first_table) + + # assert that all tables have the same columns + base_result = all_cols[first_table] + self.assertTrue(len(base_result) > 0) + + for other_table in table_equivalencies[1:]: + other_result = all_cols[other_table] + self.assertTrue(len(other_result) > 0) + self.assertEqual(base_result, other_result) + + other_relation = self._make_relation(other_table) + sql = self._assertTablesEqualSql(first_relation, + other_relation, + columns=base_result) + result = self.run_sql(sql, fetch='one') + + self.assertEqual( + result[0], + 0, + 'row_count_difference nonzero: ' + sql + ) + self.assertEqual( + result[1], + 0, + 'num_mismatched nonzero: ' + sql + ) + + + def _assertTableRowCountsEqual(self, relation_a, relation_b): + cmp_query = """ + with table_a as ( + + select count(*) as num_rows from {} + + ), table_b as ( + + select count(*) as num_rows from {} + + ) + + select table_a.num_rows - table_b.num_rows as difference + from table_a, table_b + + """.format(str(relation_a), str(relation_b)) + + res = self.run_sql(cmp_query, fetch='one') + + self.assertEqual(int(res[0]), 0, "Row count of table {} doesn't match row count of table {}. ({} rows different)".format( + relation_a.identifier, + relation_b.identifier, + res[0] + ) + ) + + def assertTableDoesNotExist(self, table, schema=None, database=None): + columns = self.get_table_columns(table, schema, database) + + self.assertEqual( + len(columns), + 0 + ) + + def assertTableDoesExist(self, table, schema=None, database=None): + columns = self.get_table_columns(table, schema, database) + + self.assertGreater( + len(columns), + 0 + ) + + def _assertTableColumnsEqual(self, relation_a, relation_b): + table_a_result = self.get_relation_columns(relation_a) + table_b_result = self.get_relation_columns(relation_b) + + text_types = {'text', 'character varying', 'character', 'varchar'} + + self.assertEqual(len(table_a_result), len(table_b_result)) + for a_column, b_column in zip(table_a_result, table_b_result): + a_name, a_type, a_size = a_column + b_name, b_type, b_size = b_column + self.assertEqual(a_name, b_name, + '{} vs {}: column "{}" != "{}"'.format( + relation_a, relation_b, a_name, b_name + )) + + self.assertEqual(a_type, b_type, + '{} vs {}: column "{}" has type "{}" != "{}"'.format( + relation_a, relation_b, a_name, a_type, b_type + )) + + self.assertEqual(a_size, b_size, + '{} vs {}: column "{}" has size "{}" != "{}"'.format( + relation_a, relation_b, a_name, a_size, b_size + )) + + def assertEquals(self, *args, **kwargs): + # assertEquals is deprecated. This makes the warnings less chatty + self.assertEqual(*args, **kwargs) + + def assertBetween(self, timestr, start, end=None): + datefmt = '%Y-%m-%dT%H:%M:%S.%fZ' + if end is None: + end = datetime.utcnow() + + parsed = datetime.strptime(timestr, datefmt) + + self.assertLessEqual(start, parsed, + 'parsed date {} happened before {}'.format( + parsed, + start.strftime(datefmt)) + ) + self.assertGreaterEqual(end, parsed, + 'parsed date {} happened after {}'.format( + parsed, + end.strftime(datefmt)) + ) + + +def use_profile(profile_name): + """A decorator to declare a test method as using a particular profile. + Handles both setting the nose attr and calling self.use_profile. + + Use like this: + + class TestSomething(DBIntegrationTest): + @use_profile('postgres') + def test_postgres_thing(self): + self.assertEqual(self.adapter_type, 'postgres') + + @use_profile('snowflake') + def test_snowflake_thing(self): + self.assertEqual(self.adapter_type, 'snowflake') + """ + def outer(wrapped): + @getattr(pytest.mark, 'profile_'+profile_name) + @wraps(wrapped) + def func(self, *args, **kwargs): + return wrapped(self, *args, **kwargs) + # sanity check at import time + assert _profile_from_test_name(wrapped.__name__) == profile_name + return func + return outer + + +class AnyFloat: + """Any float. Use this in assertEqual() calls to assert that it is a float. + """ + def __eq__(self, other): + return isinstance(other, float) + + +class AnyString: + """Any string. Use this in assertEqual() calls to assert that it is a string. + """ + def __eq__(self, other): + return isinstance(other, str) + + +class AnyStringWith: + def __init__(self, contains=None): + self.contains = contains + + def __eq__(self, other): + if not isinstance(other, str): + return False + + if self.contains is None: + return True + + return self.contains in other + + def __repr__(self): + return 'AnyStringWith<{!r}>'.format(self.contains) + + +def bigquery_rate_limiter(err, *args): + msg = str(err) + if 'too many table update operations for this table' in msg: + time.sleep(1) + return True + return False + + +def get_manifest(): + path = './target/partial_parse.msgpack' + if os.path.exists(path): + with open(path, 'rb') as fp: + manifest_mp = fp.read() + manifest: Manifest = Manifest.from_msgpack(manifest_mp) + return manifest + else: + return None diff --git a/tests/integration/bigquery_test/adapter-models/schema.yml b/tests/integration/bigquery_test/adapter-models/schema.yml new file mode 100644 index 000000000..97a54b3c7 --- /dev/null +++ b/tests/integration/bigquery_test/adapter-models/schema.yml @@ -0,0 +1,39 @@ +version: 2 +models: +- name: test_get_columns_in_table + columns: + - name: field_1 + tests: + - not_null + - name: field_2 + tests: + - not_null + - name: field_3 + tests: + - not_null + - name: nested_field + tests: + - not_null + - name: repeated_column + tests: + - not_null +- name: test_flattened_get_columns_in_table + columns: + - name: field_1 + tests: + - not_null + - name: field_2 + tests: + - not_null + - name: field_3 + tests: + - not_null + - name: field_4 + tests: + - not_null + - name: field_5 + tests: + - not_null + - name: repeated_column + tests: + - not_null diff --git a/tests/integration/bigquery_test/adapter-models/source.sql b/tests/integration/bigquery_test/adapter-models/source.sql new file mode 100644 index 000000000..2c419f25d --- /dev/null +++ b/tests/integration/bigquery_test/adapter-models/source.sql @@ -0,0 +1,41 @@ + +with nested_base as ( + select + struct( + 'a' as field_a, + 'b' as field_b + ) as repeated_nested + + union all + + select + struct( + 'a' as field_a, + 'b' as field_b + ) as repeated_nested +), + +nested as ( + + select + array_agg(repeated_nested) as repeated_column + + from nested_base + +), + +base as ( + + select + 1 as field_1, + 2 as field_2, + 3 as field_3, + + struct( + 4 as field_4, + 5 as field_5 + ) as nested_field +) + +select * +from base, nested diff --git a/tests/integration/bigquery_test/adapter-models/test_flattened_get_columns_in_table.sql b/tests/integration/bigquery_test/adapter-models/test_flattened_get_columns_in_table.sql new file mode 100644 index 000000000..3572ad6dc --- /dev/null +++ b/tests/integration/bigquery_test/adapter-models/test_flattened_get_columns_in_table.sql @@ -0,0 +1,22 @@ +{% set source = ref('source') %} +{% set cols = adapter.get_columns_in_relation(source) %} + +{{ log('source') }} +{{ log(source) }} + +{% set flattened = [] %} +{% for col in cols %} + {% if col.mode == 'REPEATED' %} + {% set _ = flattened.append(col) %} + {% else %} + {% set _ = flattened.extend(col.flatten()) %} + {% endif %} +{% endfor %} + +select + {% for col in flattened %} + {{ col.name }} + {% if not loop.last %}, {% endif %} + {% endfor %} + +from {{ source }} diff --git a/tests/integration/bigquery_test/adapter-models/test_get_columns_in_table.sql b/tests/integration/bigquery_test/adapter-models/test_get_columns_in_table.sql new file mode 100644 index 000000000..8135dbe9a --- /dev/null +++ b/tests/integration/bigquery_test/adapter-models/test_get_columns_in_table.sql @@ -0,0 +1,10 @@ +{% set source = ref('source') %} +{% set cols = adapter.get_columns_in_relation(source) %} + +select + {% for col in cols %} + {{ col.name }} + {% if not loop.last %}, {% endif %} + {% endfor %} + +from {{ source }} diff --git a/tests/integration/bigquery_test/adapter-specific-models/expiring_table.sql b/tests/integration/bigquery_test/adapter-specific-models/expiring_table.sql new file mode 100644 index 000000000..43258a714 --- /dev/null +++ b/tests/integration/bigquery_test/adapter-specific-models/expiring_table.sql @@ -0,0 +1 @@ +select 1 as id diff --git a/tests/integration/bigquery_test/case-sensitive-models/fUnKyCaSe.sql b/tests/integration/bigquery_test/case-sensitive-models/fUnKyCaSe.sql new file mode 100644 index 000000000..1934d4cfc --- /dev/null +++ b/tests/integration/bigquery_test/case-sensitive-models/fUnKyCaSe.sql @@ -0,0 +1,5 @@ +{{ config(materialized='incremental') }} +select 1 as id +{% if is_incremental() %} +this is a syntax error! +{% endif %} diff --git a/tests/integration/bigquery_test/case-sensitive-schemas/model.sql b/tests/integration/bigquery_test/case-sensitive-schemas/model.sql new file mode 100644 index 000000000..1934d4cfc --- /dev/null +++ b/tests/integration/bigquery_test/case-sensitive-schemas/model.sql @@ -0,0 +1,5 @@ +{{ config(materialized='incremental') }} +select 1 as id +{% if is_incremental() %} +this is a syntax error! +{% endif %} diff --git a/tests/integration/bigquery_test/copy-failing-models/copy_bad_materialization.sql b/tests/integration/bigquery_test/copy-failing-models/copy_bad_materialization.sql new file mode 100644 index 000000000..b6093645d --- /dev/null +++ b/tests/integration/bigquery_test/copy-failing-models/copy_bad_materialization.sql @@ -0,0 +1,2 @@ +{{ config(copy_materialization='view') }} +{{ ref('original') }} \ No newline at end of file diff --git a/tests/integration/bigquery_test/copy-failing-models/original.sql b/tests/integration/bigquery_test/copy-failing-models/original.sql new file mode 100644 index 000000000..26d9cae7b --- /dev/null +++ b/tests/integration/bigquery_test/copy-failing-models/original.sql @@ -0,0 +1 @@ +select 1 as id \ No newline at end of file diff --git a/tests/integration/bigquery_test/copy-models/additional.sql b/tests/integration/bigquery_test/copy-models/additional.sql new file mode 100644 index 000000000..33560d6c0 --- /dev/null +++ b/tests/integration/bigquery_test/copy-models/additional.sql @@ -0,0 +1 @@ +select 2 as id diff --git a/tests/integration/bigquery_test/copy-models/copy_as_incremental.sql b/tests/integration/bigquery_test/copy-models/copy_as_incremental.sql new file mode 100644 index 000000000..bbe8e5acd --- /dev/null +++ b/tests/integration/bigquery_test/copy-models/copy_as_incremental.sql @@ -0,0 +1,2 @@ +{{ config(copy_materialization='incremental') }} +{{ ref('original') }} \ No newline at end of file diff --git a/tests/integration/bigquery_test/copy-models/copy_as_several_tables.sql b/tests/integration/bigquery_test/copy-models/copy_as_several_tables.sql new file mode 100644 index 000000000..99b04e1b3 --- /dev/null +++ b/tests/integration/bigquery_test/copy-models/copy_as_several_tables.sql @@ -0,0 +1 @@ +select * from {{ ref('original') }}, {{ source('test_copy_several_tables', 'additional') }} diff --git a/tests/integration/bigquery_test/copy-models/copy_as_table.sql b/tests/integration/bigquery_test/copy-models/copy_as_table.sql new file mode 100644 index 000000000..3f2fe2550 --- /dev/null +++ b/tests/integration/bigquery_test/copy-models/copy_as_table.sql @@ -0,0 +1 @@ +{{ ref('original') }} diff --git a/tests/integration/bigquery_test/copy-models/original.sql b/tests/integration/bigquery_test/copy-models/original.sql new file mode 100644 index 000000000..26d9cae7b --- /dev/null +++ b/tests/integration/bigquery_test/copy-models/original.sql @@ -0,0 +1 @@ +select 1 as id \ No newline at end of file diff --git a/tests/integration/bigquery_test/copy-models/schema.yml b/tests/integration/bigquery_test/copy-models/schema.yml new file mode 100644 index 000000000..defbd47b5 --- /dev/null +++ b/tests/integration/bigquery_test/copy-models/schema.yml @@ -0,0 +1,6 @@ +version: 2 +sources: + - name: test_copy_several_tables + schema: "{{ target.schema }}" + tables: + - name: additional diff --git a/tests/integration/bigquery_test/data/data_seed.csv b/tests/integration/bigquery_test/data/data_seed.csv new file mode 100644 index 000000000..afd0a31ef --- /dev/null +++ b/tests/integration/bigquery_test/data/data_seed.csv @@ -0,0 +1,5 @@ +id,dupe +1,a +2,a +3,a +4,a diff --git a/tests/integration/bigquery_test/data/incremental_overwrite_date_expected.csv b/tests/integration/bigquery_test/data/incremental_overwrite_date_expected.csv new file mode 100644 index 000000000..7454b880b --- /dev/null +++ b/tests/integration/bigquery_test/data/incremental_overwrite_date_expected.csv @@ -0,0 +1,5 @@ +id,date_day +10,2020-01-01 +20,2020-01-01 +30,2020-01-02 +40,2020-01-02 diff --git a/tests/integration/bigquery_test/data/incremental_overwrite_day_expected.csv b/tests/integration/bigquery_test/data/incremental_overwrite_day_expected.csv new file mode 100644 index 000000000..e9db40a62 --- /dev/null +++ b/tests/integration/bigquery_test/data/incremental_overwrite_day_expected.csv @@ -0,0 +1,5 @@ +id,date_time +10,2020-01-01 00:00:00 +20,2020-01-01 00:00:00 +30,2020-01-02 00:00:00 +40,2020-01-02 00:00:00 diff --git a/tests/integration/bigquery_test/data/incremental_overwrite_range_expected.csv b/tests/integration/bigquery_test/data/incremental_overwrite_range_expected.csv new file mode 100644 index 000000000..203a4a9d0 --- /dev/null +++ b/tests/integration/bigquery_test/data/incremental_overwrite_range_expected.csv @@ -0,0 +1,5 @@ +id,date_int +10,20200101 +20,20200101 +30,20200102 +40,20200102 diff --git a/tests/integration/bigquery_test/data/incremental_overwrite_time_expected.csv b/tests/integration/bigquery_test/data/incremental_overwrite_time_expected.csv new file mode 100644 index 000000000..50559dd51 --- /dev/null +++ b/tests/integration/bigquery_test/data/incremental_overwrite_time_expected.csv @@ -0,0 +1,5 @@ +id,date_hour +10,2020-01-01 01:00:00 +20,2020-01-01 01:00:00 +30,2020-01-01 02:00:00 +40,2020-01-01 02:00:00 diff --git a/tests/integration/bigquery_test/data/merge_expected.csv b/tests/integration/bigquery_test/data/merge_expected.csv new file mode 100644 index 000000000..0aa05500c --- /dev/null +++ b/tests/integration/bigquery_test/data/merge_expected.csv @@ -0,0 +1,7 @@ +id,date_time +1,2020-01-01 00:00:00 +2,2020-01-01 00:00:00 +3,2020-01-01 00:00:00 +4,2020-01-02 00:00:00 +5,2020-01-02 00:00:00 +6,2020-01-02 00:00:00 \ No newline at end of file diff --git a/tests/integration/bigquery_test/dp-models/confirmation.sql b/tests/integration/bigquery_test/dp-models/confirmation.sql new file mode 100644 index 000000000..858c0e2e3 --- /dev/null +++ b/tests/integration/bigquery_test/dp-models/confirmation.sql @@ -0,0 +1,20 @@ + +-- This model checks to confirm that each date partition was created correctly. +-- Columns day_1, day_2, and day_3 should have a value of 1, and count_days should be 3 + +with base as ( + + select + case when _PARTITIONTIME = '2018-01-01' then 1 else 0 end as day_1, + case when _PARTITIONTIME = '2018-01-02' then 1 else 0 end as day_2, + case when _PARTITIONTIME = '2018-01-03' then 1 else 0 end as day_3 + from {{ ref('partitioned') }} + +) + +select distinct + sum(day_1) over () as day_1, + sum(day_2) over () as day_2, + sum(day_3) over () as day_3, + count(*) over () as count_days +from base diff --git a/tests/integration/bigquery_test/dp-models/confirmation_noconfig.sql b/tests/integration/bigquery_test/dp-models/confirmation_noconfig.sql new file mode 100644 index 000000000..7c061f4a5 --- /dev/null +++ b/tests/integration/bigquery_test/dp-models/confirmation_noconfig.sql @@ -0,0 +1,20 @@ + +-- This model checks to confirm that each date partition was created correctly. +-- Columns day_1, day_2, and day_3 should have a value of 1, and count_days should be 3 + +with base as ( + + select + case when _PARTITIONTIME = '2018-01-01' then 1 else 0 end as day_1, + case when _PARTITIONTIME = '2018-01-02' then 1 else 0 end as day_2, + case when _PARTITIONTIME = '2018-01-03' then 1 else 0 end as day_3 + from {{ ref('partitioned_noconfig') }} + +) + +select distinct + sum(day_1) over () as day_1, + sum(day_2) over () as day_2, + sum(day_3) over () as day_3, + count(*) over () as count_days +from base diff --git a/tests/integration/bigquery_test/dp-models/events_20180101.sql b/tests/integration/bigquery_test/dp-models/events_20180101.sql new file mode 100644 index 000000000..9a8f54d5b --- /dev/null +++ b/tests/integration/bigquery_test/dp-models/events_20180101.sql @@ -0,0 +1,4 @@ + +{{ config(materialized='table') }} + +select 1 as id diff --git a/tests/integration/bigquery_test/dp-models/events_20180102.sql b/tests/integration/bigquery_test/dp-models/events_20180102.sql new file mode 100644 index 000000000..63bfcdc13 --- /dev/null +++ b/tests/integration/bigquery_test/dp-models/events_20180102.sql @@ -0,0 +1,4 @@ + +{{ config(materialized='table') }} + +select 2 as id diff --git a/tests/integration/bigquery_test/dp-models/events_20180103.sql b/tests/integration/bigquery_test/dp-models/events_20180103.sql new file mode 100644 index 000000000..09a9f02c7 --- /dev/null +++ b/tests/integration/bigquery_test/dp-models/events_20180103.sql @@ -0,0 +1,4 @@ + +{{ config(materialized='table') }} + +select 3 as id diff --git a/tests/integration/bigquery_test/dp-models/partitioned.sql b/tests/integration/bigquery_test/dp-models/partitioned.sql new file mode 100644 index 000000000..5d77021d3 --- /dev/null +++ b/tests/integration/bigquery_test/dp-models/partitioned.sql @@ -0,0 +1,16 @@ + +{{ + config( + materialized='table', + partitions=['20180101', '20180102', '20180103'], + verbose=True + ) +}} + +-- Hack to make sure our events models run first. +-- In practice, these would be source data +-- {{ ref('events_20180101') }} +-- {{ ref('events_20180102') }} +-- {{ ref('events_20180103') }} + +select * from `{{ this.schema }}`.`{{ date_sharded_table('events_') }}` diff --git a/tests/integration/bigquery_test/dp-models/partitioned_noconfig.sql b/tests/integration/bigquery_test/dp-models/partitioned_noconfig.sql new file mode 100644 index 000000000..c199d37a9 --- /dev/null +++ b/tests/integration/bigquery_test/dp-models/partitioned_noconfig.sql @@ -0,0 +1,7 @@ +-- Hack to make sure our events models run first. +-- In practice, these would be source data +-- {{ ref('events_20180101') }} +-- {{ ref('events_20180102') }} +-- {{ ref('events_20180103') }} + +select * from `{{ this.schema }}`.`{{ date_sharded_table('events_') }}` diff --git a/tests/integration/bigquery_test/dp-models/partitioned_simple.sql b/tests/integration/bigquery_test/dp-models/partitioned_simple.sql new file mode 100644 index 000000000..af65072be --- /dev/null +++ b/tests/integration/bigquery_test/dp-models/partitioned_simple.sql @@ -0,0 +1,14 @@ + +{{ + config( + materialized='table', + partition_date='20180101', + verbose=True + ) +}} + +-- Hack to make sure our events models run first. +-- In practice, these would be source data +-- {{ ref('events_20180101') }} + +select * from `{{ this.schema }}`.`events_20180101` diff --git a/tests/integration/bigquery_test/dp-models/schema.yml b/tests/integration/bigquery_test/dp-models/schema.yml new file mode 100644 index 000000000..3daf468dc --- /dev/null +++ b/tests/integration/bigquery_test/dp-models/schema.yml @@ -0,0 +1,33 @@ +version: 2 +models: +- name: partitioned_simple + columns: + - name: id + tests: + - not_null + - unique +- name: confirmation + columns: &conf_columns + - name: cast(count_days as string) + tests: + - accepted_values: + values: + - 3 + - name: cast(day_1 as string) + tests: + - accepted_values: + values: + - 1 + - name: cast(day_2 as string) + tests: + - accepted_values: + values: + - 1 + - name: cast(day_3 as string) + tests: + - accepted_values: + values: + - 1 + +- name: confirmation_noconfig + columns: *conf_columns diff --git a/tests/integration/bigquery_test/execution-project-models/model.sql b/tests/integration/bigquery_test/execution-project-models/model.sql new file mode 100644 index 000000000..43258a714 --- /dev/null +++ b/tests/integration/bigquery_test/execution-project-models/model.sql @@ -0,0 +1 @@ +select 1 as id diff --git a/tests/integration/bigquery_test/execution-project-models/schema.yml b/tests/integration/bigquery_test/execution-project-models/schema.yml new file mode 100644 index 000000000..390165a86 --- /dev/null +++ b/tests/integration/bigquery_test/execution-project-models/schema.yml @@ -0,0 +1,10 @@ +version: 2 +models: +- name: model + description: | + I'm testing the profile execution_project + tests: + - project_for_job_id: + region: region-us + project_id: "{{ project_id}}" + unique_schema_id: "{{ unique_schema_id }}" \ No newline at end of file diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_range.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_range.sql new file mode 100644 index 000000000..581007c01 --- /dev/null +++ b/tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_range.sql @@ -0,0 +1,46 @@ + +{{ + config( + materialized="incremental", + unique_key="id", + cluster_by="id", + partition_by={ + "field": "id", + "data_type": "int64", + "range": { + "start": 1, + "end": 10, + "interval": 1 + } + } + ) +}} + + +with data as ( + + {% if not is_incremental() %} + + select 1 as id, cast('2020-01-01' as datetime) as date_time union all + select 2 as id, cast('2020-01-01' as datetime) as date_time union all + select 3 as id, cast('2020-01-01' as datetime) as date_time union all + select 4 as id, cast('2020-01-01' as datetime) as date_time + + {% else %} + + select 1 as id, cast('2020-01-01' as datetime) as date_time union all + select 2 as id, cast('2020-01-01' as datetime) as date_time union all + select 3 as id, cast('2020-01-01' as datetime) as date_time union all + select 4 as id, cast('2020-01-02' as datetime) as date_time union all + select 5 as id, cast('2020-01-02' as datetime) as date_time union all + select 6 as id, cast('2020-01-02' as datetime) as date_time + + {% endif %} + +) + +select * from data + +{% if is_incremental() %} +where id >= (select max(id) from {{ this }}) +{% endif %} diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_time.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_time.sql new file mode 100644 index 000000000..7629a9904 --- /dev/null +++ b/tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_time.sql @@ -0,0 +1,42 @@ + +{{ + config( + materialized="incremental", + unique_key="id", + cluster_by="id", + partition_by={ + "field": "date_time", + "data_type": "datetime" + } + ) +}} + + + +with data as ( + + {% if not is_incremental() %} + + select 1 as id, cast('2020-01-01' as datetime) as date_time union all + select 2 as id, cast('2020-01-01' as datetime) as date_time union all + select 3 as id, cast('2020-01-01' as datetime) as date_time union all + select 4 as id, cast('2020-01-01' as datetime) as date_time + + {% else %} + + select 1 as id, cast('2020-01-01' as datetime) as date_time union all + select 2 as id, cast('2020-01-01' as datetime) as date_time union all + select 3 as id, cast('2020-01-01' as datetime) as date_time union all + select 4 as id, cast('2020-01-02' as datetime) as date_time union all + select 5 as id, cast('2020-01-02' as datetime) as date_time union all + select 6 as id, cast('2020-01-02' as datetime) as date_time + + {% endif %} + +) + +select * from data + +{% if is_incremental() %} +where date_time > (select max(date_time) from {{ this }}) +{% endif %} diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_date.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_date.sql new file mode 100644 index 000000000..df67599ef --- /dev/null +++ b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_date.sql @@ -0,0 +1,41 @@ + +{{ + config( + materialized="incremental", + incremental_strategy='insert_overwrite', + cluster_by="id", + partition_by={ + "field": "date_day", + "data_type": "date" + } + ) +}} + + +with data as ( + + {% if not is_incremental() %} + + select 1 as id, cast('2020-01-01' as date) as date_day union all + select 2 as id, cast('2020-01-01' as date) as date_day union all + select 3 as id, cast('2020-01-01' as date) as date_day union all + select 4 as id, cast('2020-01-01' as date) as date_day + + {% else %} + + -- we want to overwrite the 4 records in the 2020-01-01 partition + -- with the 2 records below, but add two more in the 2020-01-02 partition + select 10 as id, cast('2020-01-01' as date) as date_day union all + select 20 as id, cast('2020-01-01' as date) as date_day union all + select 30 as id, cast('2020-01-02' as date) as date_day union all + select 40 as id, cast('2020-01-02' as date) as date_day + + {% endif %} + +) + +select * from data + +{% if is_incremental() %} +where date_day >= _dbt_max_partition +{% endif %} diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day.sql new file mode 100644 index 000000000..3434a5b47 --- /dev/null +++ b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day.sql @@ -0,0 +1,41 @@ + +{{ + config( + materialized="incremental", + incremental_strategy='insert_overwrite', + cluster_by="id", + partition_by={ + "field": "date_time", + "data_type": "datetime" + } + ) +}} + + +with data as ( + + {% if not is_incremental() %} + + select 1 as id, cast('2020-01-01' as datetime) as date_time union all + select 2 as id, cast('2020-01-01' as datetime) as date_time union all + select 3 as id, cast('2020-01-01' as datetime) as date_time union all + select 4 as id, cast('2020-01-01' as datetime) as date_time + + {% else %} + + -- we want to overwrite the 4 records in the 2020-01-01 partition + -- with the 2 records below, but add two more in the 2020-01-02 partition + select 10 as id, cast('2020-01-01' as datetime) as date_time union all + select 20 as id, cast('2020-01-01' as datetime) as date_time union all + select 30 as id, cast('2020-01-02' as datetime) as date_time union all + select 40 as id, cast('2020-01-02' as datetime) as date_time + + {% endif %} + +) + +select * from data + +{% if is_incremental() %} +where date_time >= _dbt_max_partition +{% endif %} diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_partitions.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_partitions.sql new file mode 100644 index 000000000..ef6cf8883 --- /dev/null +++ b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_partitions.sql @@ -0,0 +1,42 @@ + +{{ + config( + materialized="incremental", + incremental_strategy='insert_overwrite', + cluster_by="id", + partitions=["'2020-01-01'","'2020-01-02'"], + partition_by={ + "field": "date_day", + "data_type": "date" + } + ) +}} + + +with data as ( + + {% if not is_incremental() %} + + select 1 as id, cast('2020-01-01' as date) as date_day union all + select 2 as id, cast('2020-01-01' as date) as date_day union all + select 3 as id, cast('2020-01-01' as date) as date_day union all + select 4 as id, cast('2020-01-01' as date) as date_day + + {% else %} + + -- we want to overwrite the 4 records in the 2020-01-01 partition + -- with the 2 records below, but add two more in the 2020-01-02 partition + select 10 as id, cast('2020-01-01' as date) as date_day union all + select 20 as id, cast('2020-01-01' as date) as date_day union all + select 30 as id, cast('2020-01-02' as date) as date_day union all + select 40 as id, cast('2020-01-02' as date) as date_day + + {% endif %} + +) + +select * from data + +{% if is_incremental() %} +where date_day in ({{ config.get("partitions") | join(",") }}) +{% endif %} diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_range.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_range.sql new file mode 100644 index 000000000..44d3ffc86 --- /dev/null +++ b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_range.sql @@ -0,0 +1,46 @@ + +{{ + config( + materialized="incremental", + incremental_strategy='insert_overwrite', + cluster_by="id", + partition_by={ + "field": "date_int", + "data_type": "int64", + "range": { + "start": 20200101, + "end": 20200110, + "interval": 1 + } + } + ) +}} + + +with data as ( + + {% if not is_incremental() %} + + select 1 as id, 20200101 as date_int union all + select 2 as id, 20200101 as date_int union all + select 3 as id, 20200101 as date_int union all + select 4 as id, 20200101 as date_int + + {% else %} + + -- we want to overwrite the 4 records in the 20200101 partition + -- with the 2 records below, but add two more in the 20200102 partition + select 10 as id, 20200101 as date_int union all + select 20 as id, 20200101 as date_int union all + select 30 as id, 20200102 as date_int union all + select 40 as id, 20200102 as date_int + + {% endif %} + +) + +select * from data + +{% if is_incremental() %} +where date_int >= _dbt_max_partition +{% endif %} diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_time.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_time.sql new file mode 100644 index 000000000..77f901894 --- /dev/null +++ b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_time.sql @@ -0,0 +1,42 @@ + +{{ + config( + materialized="incremental", + incremental_strategy='insert_overwrite', + cluster_by="id", + partition_by={ + "field": "date_hour", + "data_type": "datetime", + "granularity": "hour" + } + ) +}} + + +with data as ( + + {% if not is_incremental() %} + + select 1 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all + select 2 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all + select 3 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all + select 4 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour + + {% else %} + + -- we want to overwrite the 4 records in the 2020-01-01 01:00:00 partition + -- with the 2 records below, but add two more in the 2020-01-00 02:00:00 partition + select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all + select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all + select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour union all + select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour + + {% endif %} + +) + +select * from data + +{% if is_incremental() %} +where date_hour >= _dbt_max_partition +{% endif %} diff --git a/tests/integration/bigquery_test/location-models/model.sql b/tests/integration/bigquery_test/location-models/model.sql new file mode 100644 index 000000000..43258a714 --- /dev/null +++ b/tests/integration/bigquery_test/location-models/model.sql @@ -0,0 +1 @@ +select 1 as id diff --git a/tests/integration/bigquery_test/macros/partition_metadata.sql b/tests/integration/bigquery_test/macros/partition_metadata.sql new file mode 100644 index 000000000..4a4131c9e --- /dev/null +++ b/tests/integration/bigquery_test/macros/partition_metadata.sql @@ -0,0 +1,19 @@ + +{% test number_partitions(model, expected) %} + + {%- set result = get_partitions_metadata(model) %} + + {% if result %} + {% set partitions = result.columns['partition_id'].values() %} + {% else %} + {% set partitions = () %} + {% endif %} + + {% set actual = partitions | length %} + {% set success = 1 if model and actual == expected else 0 %} + + select 'Expected {{ expected }}, but got {{ actual }}' as validation_error + from (select true) + where {{ success }} = 0 + +{% endtest %} diff --git a/tests/integration/bigquery_test/macros/test_creation.sql b/tests/integration/bigquery_test/macros/test_creation.sql new file mode 100644 index 000000000..f0f238919 --- /dev/null +++ b/tests/integration/bigquery_test/macros/test_creation.sql @@ -0,0 +1,18 @@ + +{% test was_materialized(model, name, type) %} + + {#-- don't run this query in the parsing step #} + {%- if model -%} + {%- set table = adapter.get_relation(database=model.database, schema=model.schema, + identifier=model.name) -%} + {%- else -%} + {%- set table = {} -%} + {%- endif -%} + + {% if table %} + select '{{ table.type }} does not match expected value {{ type }}' + from (select true) + where '{{ table.type }}' != '{{ type }}' + {% endif %} + +{% endtest %} diff --git a/tests/integration/bigquery_test/macros/test_int_inference.sql b/tests/integration/bigquery_test/macros/test_int_inference.sql new file mode 100644 index 000000000..a1ab1c8a9 --- /dev/null +++ b/tests/integration/bigquery_test/macros/test_int_inference.sql @@ -0,0 +1,36 @@ + +{% macro assert_eq(value, expected, msg) %} + {% if value != expected %} + {% do exceptions.raise_compiler_error(msg ~ value) %} + {% endif %} +{% endmacro %} + + +{% macro test_int_inference() %} + + {% set sql %} + select + 0 as int_0, + 1 as int_1, + 2 as int_2 + {% endset %} + + {% set result = run_query(sql) %} + {% do assert_eq((result | length), 1, 'expected 1 result, got ') %} + {% set actual_0 = result[0]['int_0'] %} + {% set actual_1 = result[0]['int_1'] %} + {% set actual_2 = result[0]['int_2'] %} + + {% do assert_eq(actual_0, 0, 'expected expected actual_0 to be 0, it was ') %} + {% do assert_eq((actual_0 | string), '0', 'expected string form of actual_0 to be 0, it was ') %} + {% do assert_eq((actual_0 * 2), 0, 'expected actual_0 * 2 to be 0, it was ') %} {# not 00 #} + + {% do assert_eq(actual_1, 1, 'expected actual_1 to be 1, it was ') %} + {% do assert_eq((actual_1 | string), '1', 'expected string form of actual_1 to be 1, it was ') %} + {% do assert_eq((actual_1 * 2), 2, 'expected actual_1 * 2 to be 2, it was ') %} {# not 11 #} + + {% do assert_eq(actual_2, 2, 'expected actual_2 to be 2, it was ') %} + {% do assert_eq((actual_2 | string), '2', 'expected string form of actual_2 to be 2, it was ') %} + {% do assert_eq((actual_2 * 2), 4, 'expected actual_2 * 2 to be 4, it was ') %} {# not 22 #} + +{% endmacro %} diff --git a/tests/integration/bigquery_test/macros/test_project_for_job_id.sql b/tests/integration/bigquery_test/macros/test_project_for_job_id.sql new file mode 100644 index 000000000..327f4e2e2 --- /dev/null +++ b/tests/integration/bigquery_test/macros/test_project_for_job_id.sql @@ -0,0 +1,7 @@ +{% test project_for_job_id(model, region, unique_schema_id, project_id) %} +select 1 +from `region-us`.INFORMATION_SCHEMA.JOBS_BY_PROJECT +where date(creation_time) = current_date + and job_project = {{project_id}} + and destination_table.dataset_id = {{unique_schema_id}} +{% endtest %} diff --git a/tests/integration/bigquery_test/macros/wrapped_macros.sql b/tests/integration/bigquery_test/macros/wrapped_macros.sql new file mode 100644 index 000000000..fc66f5a95 --- /dev/null +++ b/tests/integration/bigquery_test/macros/wrapped_macros.sql @@ -0,0 +1,43 @@ +{% macro my_create_schema(db_name, schema_name) %} + {% if not execute %} + {% do return(None) %} + {% endif %} + {% set relation = api.Relation.create(database=db_name, schema=schema_name).without_identifier() %} + {% do create_schema(relation) %} +{% endmacro %} + +{% macro my_drop_schema(db_name, schema_name) %} + {% if not execute %} + {% do return(None) %} + {% endif %} + {% set relation = api.Relation.create(database=db_name, schema=schema_name).without_identifier() %} + {% do drop_schema(relation) %} +{% endmacro %} + + +{% macro my_create_table_as(db_name, schema_name, table_name) %} + {% if not execute %} + {% do return(None) %} + {% endif %} + {% set relation = api.Relation.create(database=db_name, schema=schema_name, identifier=table_name) %} + {% do run_query(create_table_as(false, relation, 'select 1 as id')) %} +{% endmacro %} + + +{% macro ensure_one_relation_in(db_name, schema_name) %} + {% if not execute %} + {% do return(None) %} + {% endif %} + {% set relation = api.Relation.create(database=db_name, schema=schema_name).without_identifier() %} + {% set results = list_relations_without_caching(relation) %} + {% set rlen = (results | length) %} + {% if rlen != 1 %} + {% do exceptions.raise_compiler_error('Incorect number of results (expected 1): ' ~ rlen) %} + {% endif %} + {% set result = results[0] %} + {% set columns = get_columns_in_relation(result) %} + {% set clen = (columns | length) %} + {% if clen != 1 %} + {% do exceptions.raise_compiler_error('Incorrect number of columns (expected 1): ' ~ clen) %} + {% endif %} +{% endmacro %} diff --git a/tests/integration/bigquery_test/models/clustered_model.sql b/tests/integration/bigquery_test/models/clustered_model.sql new file mode 100644 index 000000000..1e0987ccc --- /dev/null +++ b/tests/integration/bigquery_test/models/clustered_model.sql @@ -0,0 +1,10 @@ + +{{ + config( + materialized = "table", + partition_by = {"field": "updated_at", "data_type": "date"}, + cluster_by = "dupe", + ) +}} + +select * from {{ ref('view_model') }} diff --git a/tests/integration/bigquery_test/models/fUnKyCaSe.sql b/tests/integration/bigquery_test/models/fUnKyCaSe.sql new file mode 100644 index 000000000..43258a714 --- /dev/null +++ b/tests/integration/bigquery_test/models/fUnKyCaSe.sql @@ -0,0 +1 @@ +select 1 as id diff --git a/tests/integration/bigquery_test/models/labeled_model.sql b/tests/integration/bigquery_test/models/labeled_model.sql new file mode 100644 index 000000000..b8847e819 --- /dev/null +++ b/tests/integration/bigquery_test/models/labeled_model.sql @@ -0,0 +1,8 @@ +{{ + config( + materialized = "table", + labels = {'town': 'fish', 'analytics': 'yes'} + ) +}} + +select * from {{ ref('view_model') }} diff --git a/tests/integration/bigquery_test/models/multi_clustered_model.sql b/tests/integration/bigquery_test/models/multi_clustered_model.sql new file mode 100644 index 000000000..c2093d6d6 --- /dev/null +++ b/tests/integration/bigquery_test/models/multi_clustered_model.sql @@ -0,0 +1,10 @@ + +{{ + config( + materialized = "table", + partition_by = {"field": "updated_at", "data_type": "date"}, + cluster_by = ["dupe","id"], + ) +}} + +select * from {{ ref('view_model') }} diff --git a/tests/integration/bigquery_test/models/partitioned_model.sql b/tests/integration/bigquery_test/models/partitioned_model.sql new file mode 100644 index 000000000..0f30a2185 --- /dev/null +++ b/tests/integration/bigquery_test/models/partitioned_model.sql @@ -0,0 +1,9 @@ + +{{ + config( + materialized = "table", + partition_by = {'field': 'updated_at', 'data_type': 'date'}, + ) +}} + +select * from {{ ref('view_model') }} diff --git a/tests/integration/bigquery_test/models/schema.yml b/tests/integration/bigquery_test/models/schema.yml new file mode 100644 index 000000000..68625ffc6 --- /dev/null +++ b/tests/integration/bigquery_test/models/schema.yml @@ -0,0 +1,60 @@ +version: 2 +models: +- name: view_model + description: | + View model description "with double quotes" + and with 'single quotes' as welll as other; + '''abc123''' + reserved -- characters + -- + /* comment */ + columns: + - name: dupe + tests: + - unique + - name: id + tests: + - not_null + - unique + - name: updated_at + tests: + - not_null + tests: + - was_materialized: + name: view_model + type: view +- name: table_model + description: | + View model description "with double quotes" + and with 'single quotes' as welll as other; + '''abc123''' + reserved -- characters + -- + /* comment */ + columns: + - name: id + tests: + - not_null + tests: + - was_materialized: + name: table_model + type: table +- name: fUnKyCaSe + columns: + - name: id + tests: + - not_null + - unique + tests: + - was_materialized: + name: fUnKyCaSe + type: view + + +sources: + - name: raw + project: "{{ target.database }}" + dataset: "{{ target.schema }}" + tables: + - name: seed + identifier: data_seed diff --git a/tests/integration/bigquery_test/models/sql_header_model.sql b/tests/integration/bigquery_test/models/sql_header_model.sql new file mode 100644 index 000000000..e49d82c4b --- /dev/null +++ b/tests/integration/bigquery_test/models/sql_header_model.sql @@ -0,0 +1,14 @@ +{{ config(materialized="table") }} + +{# This will fail if it is not extracted correctly #} +{% call set_sql_header(config) %} + CREATE TEMPORARY FUNCTION a_to_b(str STRING) + RETURNS STRING AS ( + CASE + WHEN LOWER(str) = 'a' THEN 'b' + ELSE str + END + ); +{% endcall %} + +select a_to_b(dupe) as dupe from {{ ref('view_model') }} diff --git a/tests/integration/bigquery_test/models/sql_header_model_incr.sql b/tests/integration/bigquery_test/models/sql_header_model_incr.sql new file mode 100644 index 000000000..2b9597686 --- /dev/null +++ b/tests/integration/bigquery_test/models/sql_header_model_incr.sql @@ -0,0 +1,17 @@ + +{{ config(materialized="incremental") }} + +{# This will fail if it is not extracted correctly #} +{% call set_sql_header(config) %} + DECLARE int_var INT64 DEFAULT 42; + + CREATE TEMPORARY FUNCTION a_to_b(str STRING) + RETURNS STRING AS ( + CASE + WHEN LOWER(str) = 'a' THEN 'b' + ELSE str + END + ); +{% endcall %} + +select a_to_b(dupe) as dupe from {{ ref('view_model') }} diff --git a/tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite.sql b/tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite.sql new file mode 100644 index 000000000..0cafb8489 --- /dev/null +++ b/tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite.sql @@ -0,0 +1,33 @@ + +{# + Ensure that the insert overwrite incremental strategy + works correctly when a UDF is used in a sql_header. The + failure mode here is that dbt might inject the UDF header + twice: once for the `create table` and then again for the + merge statement. +#} + +{{ config( + materialized="incremental", + incremental_strategy='insert_overwrite', + partition_by={"field": "dt", "data_type": "date"} +) }} + +{# This will fail if it is not extracted correctly #} +{% call set_sql_header(config) %} + DECLARE int_var INT64 DEFAULT 42; + + CREATE TEMPORARY FUNCTION a_to_b(str STRING) + RETURNS STRING AS ( + CASE + WHEN LOWER(str) = 'a' THEN 'b' + ELSE str + END + ); +{% endcall %} + +select + current_date() as dt, + a_to_b(dupe) as dupe + +from {{ ref('view_model') }} diff --git a/tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite_static.sql b/tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite_static.sql new file mode 100644 index 000000000..4d760a0fd --- /dev/null +++ b/tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite_static.sql @@ -0,0 +1,32 @@ + +{# + Ensure that the insert overwrite incremental strategy + works correctly when a UDF is used in a sql_header. The + failure mode here is that dbt might inject the UDF header + twice: once for the `create table` and then again for the + merge statement. +#} + +{{ config( + materialized="incremental", + incremental_strategy='insert_overwrite', + partition_by={"field": "dt", "data_type": "date"}, + partitions=["'2020-01-1'"] +) }} + +{# This will fail if it is not extracted correctly #} +{% call set_sql_header(config) %} + CREATE TEMPORARY FUNCTION a_to_b(str STRING) + RETURNS STRING AS ( + CASE + WHEN LOWER(str) = 'a' THEN 'b' + ELSE str + END + ); +{% endcall %} + +select + cast('2020-01-01' as date) as dt, + a_to_b(dupe) as dupe + +from {{ ref('view_model') }} diff --git a/tests/integration/bigquery_test/models/table_model.sql b/tests/integration/bigquery_test/models/table_model.sql new file mode 100644 index 000000000..c748eb687 --- /dev/null +++ b/tests/integration/bigquery_test/models/table_model.sql @@ -0,0 +1,8 @@ +{{ + config( + materialized = "table", + persist_docs={ "relation": true, "columns": true, "schema": true } + ) +}} + +select * from {{ ref('view_model') }} diff --git a/tests/integration/bigquery_test/models/view_model.sql b/tests/integration/bigquery_test/models/view_model.sql new file mode 100644 index 000000000..bacdaf874 --- /dev/null +++ b/tests/integration/bigquery_test/models/view_model.sql @@ -0,0 +1,14 @@ +{{ + config( + materialized = "view", + persist_docs={ "relation": true, "columns": true, "schema": true } + ) +}} + + +select + id, + current_date as updated_at, + dupe + +from {{ source('raw', 'seed') }} diff --git a/tests/integration/bigquery_test/partition-models/my_model.sql b/tests/integration/bigquery_test/partition-models/my_model.sql new file mode 100644 index 000000000..554db0932 --- /dev/null +++ b/tests/integration/bigquery_test/partition-models/my_model.sql @@ -0,0 +1,15 @@ + + +{{ + config( + materialized="table", + partition_by=var('partition_by'), + cluster_by=var('cluster_by'), + partition_expiration_days=var('partition_expiration_days'), + require_partition_filter=var('require_partition_filter') + ) +}} + +select 1 as id, 'dr. bigquery' as name, current_timestamp() as cur_time, current_date() as cur_date +union all +select 2 as id, 'prof. bigquery' as name, current_timestamp() as cur_time, current_date() as cur_date diff --git a/tests/integration/bigquery_test/partition-models/schema.yml b/tests/integration/bigquery_test/partition-models/schema.yml new file mode 100644 index 000000000..208701eaa --- /dev/null +++ b/tests/integration/bigquery_test/partition-models/schema.yml @@ -0,0 +1,6 @@ +version: 2 +models: +- name: my_model + tests: + - number_partitions: + expected: "{{ var('expected', 1) }}" diff --git a/tests/integration/bigquery_test/test_bigquery_adapter_functions.py b/tests/integration/bigquery_test/test_bigquery_adapter_functions.py new file mode 100644 index 000000000..06e230f30 --- /dev/null +++ b/tests/integration/bigquery_test/test_bigquery_adapter_functions.py @@ -0,0 +1,77 @@ +from tests.integration.base import DBTIntegrationTest, FakeArgs, use_profile +import yaml + + +class TestBigqueryAdapterFunctions(DBTIntegrationTest): + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "adapter-models" + + @property + def profile_config(self): + return self.bigquery_profile() + + @use_profile('bigquery') + def test__bigquery_adapter_functions(self): + results = self.run_dbt() + self.assertEqual(len(results), 3) + + test_results = self.run_dbt(['test']) + + self.assertTrue(len(test_results) > 0) + for result in test_results: + self.assertEqual(result.status, 'pass') + self.assertFalse(result.skipped) + self.assertEqual(result.failures, 0) + + +class TestBigqueryAdapterMacros(DBTIntegrationTest): + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "models" + + def _create_schema_named(self, database, schema): + # do not create the initial schema. We'll do this ourselves! + pass + + @use_profile('bigquery') + def test__bigquery_run_create_drop_schema(self): + schema_args = yaml.safe_dump({ + 'db_name': self.default_database, + 'schema_name': self.unique_schema(), + }) + self.run_dbt( + ['run-operation', 'my_create_schema', '--args', schema_args]) + relation_args = yaml.safe_dump({ + 'db_name': self.default_database, + 'schema_name': self.unique_schema(), + 'table_name': 'some_table', + }) + self.run_dbt(['run-operation', 'my_create_table_as', + '--args', relation_args]) + # exercise list_relations_without_caching and get_columns_in_relation + self.run_dbt( + ['run-operation', 'ensure_one_relation_in', '--args', schema_args]) + # now to drop the schema + schema_relation = self.adapter.Relation.create( + database=self.default_database, schema=self.unique_schema()).without_identifier() + with self.adapter.connection_named('test'): + results = self.adapter.list_relations_without_caching( + schema_relation) + assert len(results) == 1 + + self.run_dbt( + ['run-operation', 'my_drop_schema', '--args', schema_args]) + with self.adapter.connection_named('test'): + results = self.adapter.list_relations_without_caching( + schema_relation) + assert len(results) == 0 diff --git a/tests/integration/bigquery_test/test_bigquery_adapter_specific.py b/tests/integration/bigquery_test/test_bigquery_adapter_specific.py new file mode 100644 index 000000000..03170378b --- /dev/null +++ b/tests/integration/bigquery_test/test_bigquery_adapter_specific.py @@ -0,0 +1,40 @@ +""""Test adapter specific config options.""" +from pprint import pprint + +from tests.integration.base import DBTIntegrationTest, use_profile +import textwrap +import yaml + + +class TestBigqueryAdapterSpecific(DBTIntegrationTest): + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "adapter-specific-models" + + @property + def profile_config(self): + return self.bigquery_profile() + + @property + def project_config(self): + return yaml.safe_load(textwrap.dedent('''\ + config-version: 2 + models: + test: + materialized: table + expiring_table: + hours_to_expiration: 4 + ''')) + + @use_profile('bigquery') + def test_bigquery_hours_to_expiration(self): + _, stdout = self.run_dbt_and_capture(['--debug', 'run']) + + self.assertIn( + 'expiration_timestamp=TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL ' + '4 hour)', stdout) diff --git a/tests/integration/bigquery_test/test_bigquery_case_sensitive.py b/tests/integration/bigquery_test/test_bigquery_case_sensitive.py new file mode 100644 index 000000000..60700eada --- /dev/null +++ b/tests/integration/bigquery_test/test_bigquery_case_sensitive.py @@ -0,0 +1,34 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + + +class TestCaseSensitiveModelBigQueryRun(DBTIntegrationTest): + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "case-sensitive-models" + + @use_profile('bigquery') + def test__bigquery_double_run_fails(self): + results = self.run_dbt() + self.assertEqual(len(results), 1) + self.run_dbt(expect_pass=False) + + +class TestCaseSensitiveSchemaBigQueryRun(TestCaseSensitiveModelBigQueryRun): + # same test, but the schema is funky instead of the model name + @property + def schema(self): + return "bigquery_test" + + def unique_schema(self): + schema = self.schema + + to_return = "{}_{}".format(self.prefix, schema) + return to_return + + @property + def models(self): + return "case-sensitive-schemas" diff --git a/tests/integration/bigquery_test/test_bigquery_changing_partitions.py b/tests/integration/bigquery_test/test_bigquery_changing_partitions.py new file mode 100644 index 000000000..7d543397e --- /dev/null +++ b/tests/integration/bigquery_test/test_bigquery_changing_partitions.py @@ -0,0 +1,183 @@ +from tests.integration.base import DBTIntegrationTest, FakeArgs, use_profile +import json + + +class TestChangingPartitions(DBTIntegrationTest): + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "partition-models" + + def run_changes(self, before, after): + results = self.run_dbt(['run', '--vars', json.dumps(before)]) + self.assertEqual(len(results), 1) + + results = self.run_dbt(['run', '--vars', json.dumps(after)]) + self.assertEqual(len(results), 1) + + def test_partitions(self, expected): + test_results = self.run_dbt(['test', '--vars', json.dumps(expected)]) + + for result in test_results: + self.assertEqual(result.status, 'pass') + self.assertFalse(result.skipped) + self.assertEqual(result.failures, 0) + + @use_profile('bigquery') + def test_bigquery_add_partition(self): + before = {"partition_by": None, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + after = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'}, + "cluster_by": None, + 'partition_expiration_days': 7, + 'require_partition_filter': True} + self.run_changes(before, after) + self.test_partitions({"expected": 1}) + + @use_profile('bigquery') + def test_bigquery_add_partition_year(self): + before = {"partition_by": None, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + after = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp', 'granularity': 'year'}, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + self.run_changes(before, after) + self.test_partitions({"expected": 1}) + + @use_profile('bigquery') + def test_bigquery_add_partition_month(self): + before = {"partition_by": None, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + after = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp', 'granularity': 'month'}, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + self.run_changes(before, after) + self.test_partitions({"expected": 1}) + + @use_profile('bigquery') + def test_bigquery_add_partition_hour(self): + before = {"partition_by": None, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + after = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp', 'granularity': 'hour'}, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + self.run_changes(before, after) + self.test_partitions({"expected": 1}) + + @use_profile('bigquery') + def test_bigquery_add_partition_hour(self): + before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp', 'granularity': 'day'}, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + after = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp', 'granularity': 'hour'}, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + self.run_changes(before, after) + self.test_partitions({"expected": 1}) + + @use_profile('bigquery') + def test_bigquery_remove_partition(self): + before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'}, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + after = {"partition_by": None, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + self.run_changes(before, after) + + @use_profile('bigquery') + def test_bigquery_change_partitions(self): + before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'}, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + after = {"partition_by": {'field': "cur_date"}, + "cluster_by": None, + 'partition_expiration_days': 7, + 'require_partition_filter': True} + self.run_changes(before, after) + self.test_partitions({"expected": 1}) + self.run_changes(after, before) + self.test_partitions({"expected": 1}) + + @use_profile('bigquery') + def test_bigquery_change_partitions_from_int(self): + before = {"partition_by": {"field": "id", "data_type": "int64", "range": {"start": 0, "end": 10, "interval": 1}}, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + after = {"partition_by": {"field": "cur_date", "data_type": "date"}, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + self.run_changes(before, after) + self.test_partitions({"expected": 1}) + self.run_changes(after, before) + self.test_partitions({"expected": 2}) + + @use_profile('bigquery') + def test_bigquery_add_clustering(self): + before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'}, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + after = {"partition_by": {'field': "cur_date"}, + "cluster_by": "id", + 'partition_expiration_days': None, + 'require_partition_filter': None} + self.run_changes(before, after) + + @use_profile('bigquery') + def test_bigquery_remove_clustering(self): + before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'}, + "cluster_by": "id", + 'partition_expiration_days': None, + 'require_partition_filter': None} + after = {"partition_by": {'field': "cur_date"}, + "cluster_by": None, + 'partition_expiration_days': None, + 'require_partition_filter': None} + self.run_changes(before, after) + + @use_profile('bigquery') + def test_bigquery_change_clustering(self): + before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'}, + "cluster_by": "id", + 'partition_expiration_days': None, + 'require_partition_filter': None} + after = {"partition_by": {'field': "cur_date"}, + "cluster_by": "name", + 'partition_expiration_days': None, + 'require_partition_filter': None} + self.run_changes(before, after) + + @use_profile('bigquery') + def test_bigquery_change_clustering_strict(self): + before = {'partition_by': {'field': 'cur_time', 'data_type': 'timestamp'}, + 'cluster_by': 'id', + 'partition_expiration_days': None, + 'require_partition_filter': None} + after = {'partition_by': {'field': 'cur_date', 'data_type': 'date'}, + 'cluster_by': 'name', + 'partition_expiration_days': None, + 'require_partition_filter': None} + self.run_changes(before, after) diff --git a/tests/integration/bigquery_test/test_bigquery_copy_failing_models.py b/tests/integration/bigquery_test/test_bigquery_copy_failing_models.py new file mode 100644 index 000000000..659525663 --- /dev/null +++ b/tests/integration/bigquery_test/test_bigquery_copy_failing_models.py @@ -0,0 +1,36 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import textwrap +import yaml + + +class TestBigqueryCopyTableFails(DBTIntegrationTest): + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "copy-failing-models" + + @property + def profile_config(self): + return self.bigquery_profile() + + @property + def project_config(self): + return yaml.safe_load(textwrap.dedent('''\ + config-version: 2 + models: + test: + original: + materialized: table + copy_bad_materialization: + materialized: copy + ''')) + + @use_profile('bigquery') + def test__bigquery_copy_table_fails(self): + results = self.run_dbt(expect_pass=False) + self.assertEqual(len(results), 2) + self.assertEqual(results[1].status, 'error') diff --git a/tests/integration/bigquery_test/test_bigquery_copy_models.py b/tests/integration/bigquery_test/test_bigquery_copy_models.py new file mode 100644 index 000000000..f4362659b --- /dev/null +++ b/tests/integration/bigquery_test/test_bigquery_copy_models.py @@ -0,0 +1,41 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import textwrap +import yaml + + +class TestBigqueryCopyTable(DBTIntegrationTest): + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "copy-models" + + @property + def profile_config(self): + return self.bigquery_profile() + + @property + def project_config(self): + return yaml.safe_load(textwrap.dedent('''\ + config-version: 2 + models: + test: + original: + materialized: table + additional: + materialized: table + copy_as_table: + materialized: copy + copy_as_several_tables: + materialized: copy + copy_as_incremental: + materialized: copy + ''')) + + @use_profile('bigquery') + def test__bigquery_copy_table(self): + results = self.run_dbt() + self.assertEqual(len(results), 5) diff --git a/tests/integration/bigquery_test/test_bigquery_date_partitioning.py b/tests/integration/bigquery_test/test_bigquery_date_partitioning.py new file mode 100644 index 000000000..fad0b50a0 --- /dev/null +++ b/tests/integration/bigquery_test/test_bigquery_date_partitioning.py @@ -0,0 +1,46 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import textwrap +import yaml + + +class TestBigqueryDatePartitioning(DBTIntegrationTest): + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "dp-models" + + @property + def profile_config(self): + return self.bigquery_profile() + + @property + def project_config(self): + return yaml.safe_load(textwrap.dedent('''\ + config-version: 2 + models: + test: + partitioned_noconfig: + materialized: table + partitions: + - 20180101 + - 20180102 + - 20180103 + verbose: true + ''')) + + @use_profile('bigquery') + def test__bigquery_date_partitioning(self): + results = self.run_dbt() + self.assertEqual(len(results), 8) + + test_results = self.run_dbt(['test']) + + self.assertTrue(len(test_results) > 0) + for result in test_results: + self.assertEqual(result.status, 'pass') + self.assertFalse(result.skipped) + self.assertEqual(result.failures, 0) diff --git a/tests/integration/bigquery_test/test_bigquery_execution_project.py b/tests/integration/bigquery_test/test_bigquery_execution_project.py new file mode 100644 index 000000000..50c4c92f5 --- /dev/null +++ b/tests/integration/bigquery_test/test_bigquery_execution_project.py @@ -0,0 +1,23 @@ +import os +from tests.integration.base import DBTIntegrationTest, use_profile + + +class TestAlternateExecutionProjectBigQueryRun(DBTIntegrationTest): + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "execution-project-models" + + @use_profile('bigquery') + def test__bigquery_execute_project(self): + results = self.run_dbt(['run', '--models', 'model']) + self.assertEqual(len(results), 1) + execution_project = os.environ['BIGQUERY_TEST_ALT_DATABASE'] + self.run_dbt(['test', + '--target', 'alternate', + '--vars', '{ project_id: %s, unique_schema_id: %s }' + % (execution_project, self.unique_schema())], + expect_pass=False) diff --git a/tests/integration/bigquery_test/test_bigquery_location_change.py b/tests/integration/bigquery_test/test_bigquery_location_change.py new file mode 100644 index 000000000..02911f7d3 --- /dev/null +++ b/tests/integration/bigquery_test/test_bigquery_location_change.py @@ -0,0 +1,31 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import os + + +class TestBigqueryErrorHandling(DBTIntegrationTest): + def setUp(self): + self.valid_location = os.getenv('DBT_TEST_BIGQUERY_INITIAL_LOCATION', 'US') + self.invalid_location = os.getenv('DBT_TEST_BIGQUERY_BAD_LOCATION', 'northamerica-northeast1') + self.location = self.valid_location + super().setUp() + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "location-models" + + def bigquery_profile(self): + result = super().bigquery_profile() + result['test']['outputs']['default2']['location'] = self.location + return result + + @use_profile('bigquery') + def test_bigquery_location_invalid(self): + self.run_dbt() + self.location = self.invalid_location + self.use_profile('bigquery') + _, stdout = self.run_dbt_and_capture(expect_pass=False) + assert 'Query Job SQL Follows' not in stdout diff --git a/tests/integration/bigquery_test/test_bigquery_query_results.py b/tests/integration/bigquery_test/test_bigquery_query_results.py new file mode 100644 index 000000000..6cd32f383 --- /dev/null +++ b/tests/integration/bigquery_test/test_bigquery_query_results.py @@ -0,0 +1,24 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + + +class TestBaseBigQueryResults(DBTIntegrationTest): + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "models" + + @property + def project_config(self): + return { + 'config-version': 2, + 'macro-paths': ['macros'], + } + + @use_profile('bigquery') + def test__bigquery_type_inference(self): + result = self.run_dbt(['run-operation', 'test_int_inference']) + self.assertTrue(result.success) diff --git a/tests/integration/bigquery_test/test_bigquery_repeated_records.py b/tests/integration/bigquery_test/test_bigquery_repeated_records.py new file mode 100644 index 000000000..16ee69cda --- /dev/null +++ b/tests/integration/bigquery_test/test_bigquery_repeated_records.py @@ -0,0 +1,68 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import json + +class TestBaseBigQueryRun(DBTIntegrationTest): + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "models" + + @property + def project_config(self): + return { + 'config-version': 2, + 'macro-paths': ['macros'], + } + + @use_profile('bigquery') + def test__bigquery_fetch_nested_records(self): + sql = """ + select + struct( + cast('Michael' as string) as fname, + cast('Stonebreaker' as string) as lname + ) as user, + [ + struct(1 as val_1, cast(2.12 as numeric) as val_2), + struct(3 as val_1, cast(4.83 as numeric) as val_2) + ] as val + + union all + + select + struct( + cast('Johnny' as string) as fname, + cast('Brickmaker' as string) as lname + ) as user, + [ + struct(7 as val_1, cast(8 as numeric) as val_2), + struct(9 as val_1, cast(null as numeric) as val_2) + ] as val + """ + + + status, res = self.adapter.execute(sql, fetch=True) + + self.assertEqual(len(res), 2, "incorrect row count") + + expected = { + "user": [ + '{"fname": "Michael", "lname": "Stonebreaker"}', + '{"fname": "Johnny", "lname": "Brickmaker"}' + ], + "val": [ + '[{"val_1": 1, "val_2": 2.12}, {"val_1": 3, "val_2": 4.83}]', + '[{"val_1": 7, "val_2": 8}, {"val_1": 9, "val_2": null}]' + ] + } + + for i, key in enumerate(expected): + line = "row {} for key {} ({} vs {})".format(i, key, expected[key][i], res[i][key]) + # py2 serializes these in an unordered way - deserialize to compare + v1 = expected[key][i] + v2 = res[i][key] + self.assertEqual(json.loads(v1), json.loads(v2), line) diff --git a/tests/integration/bigquery_test/test_bigquery_update_columns.py b/tests/integration/bigquery_test/test_bigquery_update_columns.py new file mode 100644 index 000000000..ed3c93556 --- /dev/null +++ b/tests/integration/bigquery_test/test_bigquery_update_columns.py @@ -0,0 +1,85 @@ +import os + +from tests.integration.base import DBTIntegrationTest, use_profile + + +class TestBigqueryUpdateColumnPolicyTag(DBTIntegrationTest): + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "update-column-policy-tag" + + @property + def project_config(self): + return { + 'config-version': 2, + 'vars': { + 'policy_tag': self.policy_tag + } + } + + @property + def policy_tag(self): + return os.environ.get('BIGQUERY_POLICY_TAG') + + @use_profile('bigquery') + def test__bigquery_update_column_policy_tag(self): + if self.policy_tag: + results = self.run_dbt(['run', '--models', 'policy_tag_table']) + self.assertEqual(len(results), 1) + + with self.get_connection() as conn: + client = conn.handle + + table = client.get_table( + self.adapter.connections.get_bq_table( + self.default_database, self.unique_schema(), 'policy_tag_table') + ) + + for schema_field in table.schema: + self.assertEquals(schema_field.policy_tags.names, + (self.policy_tag,)) + + +class TestBigqueryUpdateColumnDescription(DBTIntegrationTest): + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "update-column-description" + + @property + def project_config(self): + return { + 'config-version': 2, + 'vars': { + 'field_description': self.field_description + } + } + + @property + def field_description(self): + return 'this is a field' + + @use_profile('bigquery') + def test__bigquery_update_column_description(self): + results = self.run_dbt(['run', '--models', 'description_table']) + self.assertEqual(len(results), 1) + + with self.get_connection() as conn: + client = conn.handle + + table = client.get_table( + self.adapter.connections.get_bq_table( + self.default_database, self.unique_schema(), 'description_table') + ) + + for schema_field in table.schema: + self.assertEquals(schema_field.description, self.field_description) diff --git a/tests/integration/bigquery_test/test_incremental_strategies.py b/tests/integration/bigquery_test/test_incremental_strategies.py new file mode 100644 index 000000000..32c151b20 --- /dev/null +++ b/tests/integration/bigquery_test/test_incremental_strategies.py @@ -0,0 +1,29 @@ +from tests.integration.base import DBTIntegrationTest, FakeArgs, use_profile + +class TestBigQueryScripting(DBTIntegrationTest): + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "incremental-strategy-models" + + @use_profile('bigquery') + def test__bigquery_assert_incrementals(self): + results = self.run_dbt() + self.assertEqual(len(results), 7) + + results = self.run_dbt() + self.assertEqual(len(results), 7) + + results = self.run_dbt(['seed']) + + self.assertTablesEqual('incremental_merge_range', 'merge_expected') + self.assertTablesEqual('incremental_merge_time', 'merge_expected') + self.assertTablesEqual('incremental_overwrite_time', 'incremental_overwrite_time_expected') + self.assertTablesEqual('incremental_overwrite_date', 'incremental_overwrite_date_expected') + self.assertTablesEqual('incremental_overwrite_partitions', 'incremental_overwrite_date_expected') + self.assertTablesEqual('incremental_overwrite_day', 'incremental_overwrite_day_expected') + self.assertTablesEqual('incremental_overwrite_range', 'incremental_overwrite_range_expected') diff --git a/tests/integration/bigquery_test/test_simple_bigquery_view.py b/tests/integration/bigquery_test/test_simple_bigquery_view.py new file mode 100644 index 000000000..44df6c099 --- /dev/null +++ b/tests/integration/bigquery_test/test_simple_bigquery_view.py @@ -0,0 +1,71 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import random +import time + + +class TestBaseBigQueryRun(DBTIntegrationTest): + + @property + def schema(self): + return "bigquery_test" + + @property + def models(self): + return "models" + + @property + def project_config(self): + return { + 'config-version': 2, + 'data-paths': ['data'], + 'macro-paths': ['macros'], + 'seeds': { + 'quote_columns': False, + }, + } + + @property + def profile_config(self): + return self.bigquery_profile() + + def assert_nondupes_pass(self): + # The 'dupe' model should fail, but all others should pass + test_results = self.run_dbt(['test'], expect_pass=False) + + for result in test_results: + if 'dupe' in result.node.name: + self.assertEqual(result.status, 'fail') + self.assertFalse(result.skipped) + self.assertTrue(result.failures > 0) + + # assert that actual tests pass + else: + self.assertEqual(result.status, 'pass') + self.assertFalse(result.skipped) + self.assertEqual(result.failures, 0) + + +class TestSimpleBigQueryRun(TestBaseBigQueryRun): + + @use_profile('bigquery') + def test__bigquery_simple_run(self): + # make sure seed works twice. Full-refresh is a no-op + self.run_dbt(['seed']) + self.run_dbt(['seed', '--full-refresh']) + results = self.run_dbt() + # Bump expected number of results when adding new model + self.assertEqual(len(results), 11) + self.assert_nondupes_pass() + + +class TestUnderscoreBigQueryRun(TestBaseBigQueryRun): + prefix = "_test{}{:04}".format(int(time.time()), random.randint(0, 9999)) + + @use_profile('bigquery') + def test_bigquery_run_twice(self): + self.run_dbt(['seed']) + results = self.run_dbt() + self.assertEqual(len(results), 11) + results = self.run_dbt() + self.assertEqual(len(results), 11) + self.assert_nondupes_pass() diff --git a/tests/integration/bigquery_test/update-column-description/description_table.sql b/tests/integration/bigquery_test/update-column-description/description_table.sql new file mode 100644 index 000000000..7110ac8b0 --- /dev/null +++ b/tests/integration/bigquery_test/update-column-description/description_table.sql @@ -0,0 +1,9 @@ +{{ + config( + materialized='table', + persist_docs={ 'columns': true } + ) +}} + +select + 1 field diff --git a/tests/integration/bigquery_test/update-column-description/description_table.yml b/tests/integration/bigquery_test/update-column-description/description_table.yml new file mode 100644 index 000000000..28d68f007 --- /dev/null +++ b/tests/integration/bigquery_test/update-column-description/description_table.yml @@ -0,0 +1,7 @@ +version: 2 + +models: +- name: description_table + columns: + - name: field + description: '{{ var("field_description") }}' diff --git a/tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.sql b/tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.sql new file mode 100644 index 000000000..7110ac8b0 --- /dev/null +++ b/tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.sql @@ -0,0 +1,9 @@ +{{ + config( + materialized='table', + persist_docs={ 'columns': true } + ) +}} + +select + 1 field diff --git a/tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.yml b/tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.yml new file mode 100644 index 000000000..01a76c50d --- /dev/null +++ b/tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.yml @@ -0,0 +1,8 @@ +version: 2 + +models: +- name: policy_tag_table + columns: + - name: field + policy_tags: + - '{{ var("policy_tag") }}' diff --git a/tests/integration/changing_relation_type_test/models/model.sql b/tests/integration/changing_relation_type_test/models/model.sql new file mode 100644 index 000000000..c88e66ee1 --- /dev/null +++ b/tests/integration/changing_relation_type_test/models/model.sql @@ -0,0 +1,8 @@ + +{{ config(materialized=var('materialized')) }} + +select '{{ var("materialized") }}' as materialization + +{% if var('materialized') == 'incremental' and is_incremental() %} + where 'abc' != (select max(materialization) from {{ this }}) +{% endif %} diff --git a/tests/integration/changing_relation_type_test/test_changing_relation_type.py b/tests/integration/changing_relation_type_test/test_changing_relation_type.py new file mode 100644 index 000000000..0c7830f1e --- /dev/null +++ b/tests/integration/changing_relation_type_test/test_changing_relation_type.py @@ -0,0 +1,45 @@ +from pytest import mark +from tests.integration.base import DBTIntegrationTest, use_profile, bigquery_rate_limiter + + +class TestChangingRelationType(DBTIntegrationTest): + + @property + def schema(self): + return "changing_relation_type" + + @staticmethod + def dir(path): + return path.lstrip("/") + + @property + def models(self): + return self.dir("models") + + @mark.flaky(rerun_filter=bigquery_rate_limiter, max_runs=3) + @use_profile("bigquery") + def test__bigquery__switch_materialization(self): + # BQ has a weird check that prevents the dropping of tables in the view materialization + # if --full-refresh is not provided. This is to prevent the clobbering of a date-sharded + # table with a view if a model config is accidently changed. We should probably remove that check + # and then remove these bq-specific tests + + results = self.run_dbt(['run', '--vars', 'materialized: view']) + self.assertEqual(results[0].node.config.materialized, 'view') + self.assertEqual(len(results), 1) + + results = self.run_dbt(['run', '--vars', 'materialized: table']) + self.assertEqual(results[0].node.config.materialized, 'table') + self.assertEqual(len(results), 1) + + results = self.run_dbt(['run', '--vars', 'materialized: view', "--full-refresh"]) + self.assertEqual(results[0].node.config.materialized, 'view') + self.assertEqual(len(results), 1) + + results = self.run_dbt(['run', '--vars', 'materialized: incremental']) + self.assertEqual(results[0].node.config.materialized, 'incremental') + self.assertEqual(len(results), 1) + + results = self.run_dbt(['run', '--vars', 'materialized: view', "--full-refresh"]) + self.assertEqual(results[0].node.config.materialized, 'view') + self.assertEqual(len(results), 1) diff --git a/tests/integration/column_quoting/data/seed.csv b/tests/integration/column_quoting/data/seed.csv new file mode 100644 index 000000000..d4a1e26ee --- /dev/null +++ b/tests/integration/column_quoting/data/seed.csv @@ -0,0 +1,4 @@ +col_A,col_B +1,2 +3,4 +5,6 diff --git a/tests/integration/column_quoting/models-unquoted/model.sql b/tests/integration/column_quoting/models-unquoted/model.sql new file mode 100644 index 000000000..a7fefd127 --- /dev/null +++ b/tests/integration/column_quoting/models-unquoted/model.sql @@ -0,0 +1,12 @@ +{% set col_a = '`col_a`' %} +{% set col_b = '`col_b`' %} + +{{config( + materialized = 'incremental', + unique_key = col_a, + incremental_strategy = var('strategy') + )}} + +select +{{ col_a }}, {{ col_b }} +from {{ref('seed')}} diff --git a/tests/integration/column_quoting/models/model.sql b/tests/integration/column_quoting/models/model.sql new file mode 100644 index 000000000..6db19b06d --- /dev/null +++ b/tests/integration/column_quoting/models/model.sql @@ -0,0 +1,12 @@ +{% set col_a = '`col_A`' %} +{% set col_b = '`col_B`' %} + +{{config( + materialized = 'incremental', + unique_key = col_a, + incremental_strategy = var('strategy') + )}} + +select +{{ col_a }}, {{ col_b }} +from {{ref('seed')}} diff --git a/tests/integration/column_quoting/test_column_quotes.py b/tests/integration/column_quoting/test_column_quotes.py new file mode 100644 index 000000000..c484317ac --- /dev/null +++ b/tests/integration/column_quoting/test_column_quotes.py @@ -0,0 +1,78 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import os + + +class BaseColumnQuotingTest(DBTIntegrationTest): + def column_quoting(self): + raise NotImplementedError('column_quoting not implemented') + + @property + def schema(self): + return 'dbt_column_quoting' + + @staticmethod + def dir(value): + return os.path.normpath(value) + + def _run_columnn_quotes(self, strategy='delete+insert'): + strategy_vars = '{{"strategy": "{}"}}'.format(strategy) + self.run_dbt(['seed', '--vars', strategy_vars]) + self.run_dbt(['run', '--vars', strategy_vars]) + self.run_dbt(['run', '--vars', strategy_vars]) + + +class TestColumnQuotingDefault(BaseColumnQuotingTest): + @property + def project_config(self): + return { + 'config-version': 2 + } + + @property + def models(self): + return self.dir('models') + + def run_dbt(self, *args, **kwargs): + return super().run_dbt(*args, **kwargs) + + @use_profile('bigquery') + def test_bigquery_column_quotes(self): + self._run_columnn_quotes(strategy='merge') + + +class TestColumnQuotingDisabled(BaseColumnQuotingTest): + @property + def models(self): + return self.dir('models-unquoted') + + @property + def project_config(self): + return { + 'config-version': 2, + 'seeds': { + 'quote_columns': False, + }, + } + + @use_profile('bigquery') + def test_bigquery_column_quotes_merged(self): + self._run_columnn_quotes(strategy='merge') + + +class TestColumnQuotingEnabled(BaseColumnQuotingTest): + @property + def models(self): + return self.dir('models') + + @property + def project_config(self): + return { + 'config-version': 2, + 'seeds': { + 'quote_columns': True, + }, + } + + @use_profile('bigquery') + def test_bigquery_column_quotes_merged(self): + self._run_columnn_quotes(strategy='merge') diff --git a/tests/integration/column_type_tests/bq_models/model.sql b/tests/integration/column_type_tests/bq_models/model.sql new file mode 100644 index 000000000..94e4fba18 --- /dev/null +++ b/tests/integration/column_type_tests/bq_models/model.sql @@ -0,0 +1,5 @@ +select + CAST(1 as int64) as int64_col, + CAST(2.0 as float64) as float64_col, + CAST(3.0 as numeric) as numeric_col, + CAST('3' as string) as string_col, diff --git a/tests/integration/column_type_tests/bq_models/schema.yml b/tests/integration/column_type_tests/bq_models/schema.yml new file mode 100644 index 000000000..8eb8a9ae2 --- /dev/null +++ b/tests/integration/column_type_tests/bq_models/schema.yml @@ -0,0 +1,10 @@ +version: 2 +models: + - name: model + tests: + - is_type: + column_map: + int64_col: ['integer', 'number'] + float64_col: ['float', 'number'] + numeric_col: ['numeric', 'number'] + string_col: ['string', 'not number'] diff --git a/tests/integration/column_type_tests/bq_models_alter_type/altered_schema.yml b/tests/integration/column_type_tests/bq_models_alter_type/altered_schema.yml new file mode 100644 index 000000000..3a78da404 --- /dev/null +++ b/tests/integration/column_type_tests/bq_models_alter_type/altered_schema.yml @@ -0,0 +1,10 @@ +version: 2 +models: + - name: model + tests: + - is_type: + column_map: + int64_col: ['string', 'not number'] + float64_col: ['float', 'number'] + numeric_col: ['numeric', 'number'] + string_col: ['string', 'not number'] diff --git a/tests/integration/column_type_tests/bq_models_alter_type/model.sql b/tests/integration/column_type_tests/bq_models_alter_type/model.sql new file mode 100644 index 000000000..066252ae5 --- /dev/null +++ b/tests/integration/column_type_tests/bq_models_alter_type/model.sql @@ -0,0 +1,6 @@ +{{ config(materialized='table') }} +select + CAST(1 as int64) as int64_col, + CAST(2.0 as float64) as float64_col, + CAST(3.0 as numeric) as numeric_col, + CAST('3' as string) as string_col, diff --git a/tests/integration/column_type_tests/macros/test_alter_column_type.sql b/tests/integration/column_type_tests/macros/test_alter_column_type.sql new file mode 100644 index 000000000..133d59fad --- /dev/null +++ b/tests/integration/column_type_tests/macros/test_alter_column_type.sql @@ -0,0 +1,5 @@ +-- Macro to alter a column type +{% macro test_alter_column_type(model_name, column_name, new_column_type) %} + {% set relation = ref(model_name) %} + {{ alter_column_type(relation, column_name, new_column_type) }} +{% endmacro %} diff --git a/tests/integration/column_type_tests/macros/test_is_type.sql b/tests/integration/column_type_tests/macros/test_is_type.sql new file mode 100644 index 000000000..2f1ffde2b --- /dev/null +++ b/tests/integration/column_type_tests/macros/test_is_type.sql @@ -0,0 +1,72 @@ + +{% macro simple_type_check_column(column, check) %} + {% if check == 'string' %} + {{ return(column.is_string()) }} + {% elif check == 'float' %} + {{ return(column.is_float()) }} + {% elif check == 'number' %} + {{ return(column.is_number()) }} + {% elif check == 'numeric' %} + {{ return(column.is_numeric()) }} + {% elif check == 'integer' %} + {{ return(column.is_integer()) }} + {% else %} + {% do exceptions.raise_compiler_error('invalid type check value: ' ~ check) %} + {% endif %} +{% endmacro %} + +{% macro type_check_column(column, type_checks) %} + {% set failures = [] %} + {% for type_check in type_checks %} + {% if type_check.startswith('not ') %} + {% if simple_type_check_column(column, type_check[4:]) %} + {% do log('simple_type_check_column got ', True) %} + {% do failures.append(type_check) %} + {% endif %} + {% else %} + {% if not simple_type_check_column(column, type_check) %} + {% do failures.append(type_check) %} + {% endif %} + {% endif %} + {% endfor %} + {% if (failures | length) > 0 %} + {% do log('column ' ~ column.name ~ ' had failures: ' ~ failures, info=True) %} + {% endif %} + {% do return((failures | length) == 0) %} +{% endmacro %} + +{% test is_type(model, column_map) %} + {% if not execute %} + {{ return(None) }} + {% endif %} + {% if not column_map %} + {% do exceptions.raise_compiler_error('test_is_type must have a column name') %} + {% endif %} + {% set columns = adapter.get_columns_in_relation(model) %} + {% if (column_map | length) != (columns | length) %} + {% set column_map_keys = (column_map | list | string) %} + {% set column_names = (columns | map(attribute='name') | list | string) %} + {% do exceptions.raise_compiler_error('did not get all the columns/all columns not specified:\n' ~ column_map_keys ~ '\nvs\n' ~ column_names) %} + {% endif %} + {% set bad_columns = [] %} + {% for column in columns %} + {% set column_key = (column.name | lower) %} + {% if column_key in column_map %} + {% set type_checks = column_map[column_key] %} + {% if not type_checks %} + {% do exceptions.raise_compiler_error('no type checks?') %} + {% endif %} + {% if not type_check_column(column, type_checks) %} + {% do bad_columns.append(column.name) %} + {% endif %} + {% else %} + {% do exceptions.raise_compiler_error('column key ' ~ column_key ~ ' not found in ' ~ (column_map | list | string)) %} + {% endif %} + {% endfor %} + {% do log('bad columns: ' ~ bad_columns, info=True) %} + {% for bad_column in bad_columns %} + select '{{ bad_column }}' as bad_column + {{ 'union all' if not loop.last }} + {% endfor %} + select * from (select 1 limit 0) as nothing +{% endtest %} diff --git a/tests/integration/column_type_tests/test_alter_column_types.py b/tests/integration/column_type_tests/test_alter_column_types.py new file mode 100644 index 000000000..60bda6df1 --- /dev/null +++ b/tests/integration/column_type_tests/test_alter_column_types.py @@ -0,0 +1,28 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import yaml + + +class TestAlterColumnTypes(DBTIntegrationTest): + @property + def schema(self): + return 'alter_column_types' + + def run_and_alter_and_test(self, alter_column_type_args): + self.assertEqual(len(self.run_dbt(['run'])), 1) + self.run_dbt(['run-operation', 'test_alter_column_type', '--args', alter_column_type_args]) + self.assertEqual(len(self.run_dbt(['test'])), 1) + + +class TestBigQueryAlterColumnTypes(TestAlterColumnTypes): + @property + def models(self): + return 'bq_models_alter_type' + + @use_profile('bigquery') + def test_bigquery_column_types(self): + alter_column_type_args = yaml.safe_dump({ + 'model_name': 'model', + 'column_name': 'int64_col', + 'new_column_type': 'string' + }) + self.run_and_alter_and_test(alter_column_type_args) diff --git a/tests/integration/column_type_tests/test_column_types.py b/tests/integration/column_type_tests/test_column_types.py new file mode 100644 index 000000000..ab04cc8f3 --- /dev/null +++ b/tests/integration/column_type_tests/test_column_types.py @@ -0,0 +1,21 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + + +class TestColumnTypes(DBTIntegrationTest): + @property + def schema(self): + return 'column_types' + + def run_and_test(self): + self.assertEqual(len(self.run_dbt(['run'])), 1) + self.assertEqual(len(self.run_dbt(['test'])), 1) + + +class TestBigQueryColumnTypes(TestColumnTypes): + @property + def models(self): + return 'bq_models' + + @use_profile('bigquery') + def test_bigquery_column_types(self): + self.run_and_test() diff --git a/tests/integration/dbt_debug_test/models/model.sql b/tests/integration/dbt_debug_test/models/model.sql new file mode 100644 index 000000000..2c2d9c8de --- /dev/null +++ b/tests/integration/dbt_debug_test/models/model.sql @@ -0,0 +1 @@ +seled 1 as id diff --git a/tests/integration/dbt_debug_test/test_debug.py b/tests/integration/dbt_debug_test/test_debug.py new file mode 100644 index 000000000..7e346f56e --- /dev/null +++ b/tests/integration/dbt_debug_test/test_debug.py @@ -0,0 +1,33 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import os +import re +import yaml + +import pytest + + +class TestDebug(DBTIntegrationTest): + @property + def schema(self): + return 'dbt_debug' + + @staticmethod + def dir(value): + return os.path.normpath(value) + + @property + def models(self): + return self.dir('models') + + @use_profile('bigquery') + def test_bigquery_ok(self): + self.run_dbt(['debug']) + self.assertNotIn('ERROR', self.capsys.readouterr().out) + +class TestDebugProfileVariable(TestDebug): + @property + def project_config(self): + return { + 'config-version': 2, + 'profile': '{{ "te" ~ "st" }}' + } diff --git a/tests/integration/defer_state_test/data/seed.csv b/tests/integration/defer_state_test/data/seed.csv new file mode 100644 index 000000000..1a728c8ab --- /dev/null +++ b/tests/integration/defer_state_test/data/seed.csv @@ -0,0 +1,3 @@ +id,name +1,Alice +2,Bob diff --git a/tests/integration/defer_state_test/macros/infinite_macros.sql b/tests/integration/defer_state_test/macros/infinite_macros.sql new file mode 100644 index 000000000..81d2083d3 --- /dev/null +++ b/tests/integration/defer_state_test/macros/infinite_macros.sql @@ -0,0 +1,13 @@ +{# trigger infinite recursion if not handled #} + +{% macro my_infinitely_recursive_macro() %} + {{ return(adapter.dispatch('my_infinitely_recursive_macro')()) }} +{% endmacro %} + +{% macro default__my_infinitely_recursive_macro() %} + {% if unmet_condition %} + {{ my_infinitely_recursive_macro() }} + {% else %} + {{ return('') }} + {% endif %} +{% endmacro %} diff --git a/tests/integration/defer_state_test/macros/macros.sql b/tests/integration/defer_state_test/macros/macros.sql new file mode 100644 index 000000000..79519c1b6 --- /dev/null +++ b/tests/integration/defer_state_test/macros/macros.sql @@ -0,0 +1,3 @@ +{% macro my_macro() %} + {% do log('in a macro' ) %} +{% endmacro %} diff --git a/tests/integration/defer_state_test/models/ephemeral_model.sql b/tests/integration/defer_state_test/models/ephemeral_model.sql new file mode 100644 index 000000000..2f976e3a9 --- /dev/null +++ b/tests/integration/defer_state_test/models/ephemeral_model.sql @@ -0,0 +1,2 @@ +{{ config(materialized='ephemeral') }} +select * from {{ ref('view_model') }} diff --git a/tests/integration/defer_state_test/models/exposures.yml b/tests/integration/defer_state_test/models/exposures.yml new file mode 100644 index 000000000..489dec3c3 --- /dev/null +++ b/tests/integration/defer_state_test/models/exposures.yml @@ -0,0 +1,8 @@ +version: 2 +exposures: + - name: my_exposure + type: application + depends_on: + - ref('view_model') + owner: + email: test@example.com diff --git a/tests/integration/defer_state_test/models/schema.yml b/tests/integration/defer_state_test/models/schema.yml new file mode 100644 index 000000000..1ec506d3d --- /dev/null +++ b/tests/integration/defer_state_test/models/schema.yml @@ -0,0 +1,9 @@ +version: 2 +models: + - name: view_model + columns: + - name: id + tests: + - unique + - not_null + - name: name diff --git a/tests/integration/defer_state_test/models/table_model.sql b/tests/integration/defer_state_test/models/table_model.sql new file mode 100644 index 000000000..65909318b --- /dev/null +++ b/tests/integration/defer_state_test/models/table_model.sql @@ -0,0 +1,5 @@ +{{ config(materialized='table') }} +select * from {{ ref('ephemeral_model') }} + +-- establish a macro dependency to trigger state:modified.macros +-- depends on: {{ my_macro() }} \ No newline at end of file diff --git a/tests/integration/defer_state_test/models/view_model.sql b/tests/integration/defer_state_test/models/view_model.sql new file mode 100644 index 000000000..72cb07a5e --- /dev/null +++ b/tests/integration/defer_state_test/models/view_model.sql @@ -0,0 +1,4 @@ +select * from {{ ref('seed') }} + +-- establish a macro dependency that trips infinite recursion if not handled +-- depends on: {{ my_infinitely_recursive_macro() }} \ No newline at end of file diff --git a/tests/integration/defer_state_test/test_defer_state.py b/tests/integration/defer_state_test/test_defer_state.py new file mode 100644 index 000000000..16e0df5f8 --- /dev/null +++ b/tests/integration/defer_state_test/test_defer_state.py @@ -0,0 +1,85 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import copy +import json +import os +import shutil + +import pytest + + +class TestDeferState(DBTIntegrationTest): + @property + def schema(self): + return "defer_state" + + @property + def models(self): + return "models" + + def setUp(self): + self.other_schema = None + super().setUp() + self._created_schemas.add(self.other_schema) + + @property + def project_config(self): + return { + 'config-version': 2, + 'seeds': { + 'test': { + 'quote_columns': False, + } + } + } + + def get_profile(self, adapter_type): + if self.other_schema is None: + self.other_schema = self.unique_schema() + '_other' + profile = super().get_profile(adapter_type) + default_name = profile['test']['target'] + profile['test']['outputs']['otherschema'] = copy.deepcopy(profile['test']['outputs'][default_name]) + profile['test']['outputs']['otherschema']['schema'] = self.other_schema + return profile + + def copy_state(self): + assert not os.path.exists('state') + os.makedirs('state') + shutil.copyfile('target/manifest.json', 'state/manifest.json') + + def run_and_defer(self): + results = self.run_dbt(['seed']) + assert len(results) == 1 + assert not any(r.node.deferred for r in results) + results = self.run_dbt(['run']) + assert len(results) == 2 + assert not any(r.node.deferred for r in results) + results = self.run_dbt(['test']) + assert len(results) == 2 + + # copy files over from the happy times when we had a good target + self.copy_state() + + # test tests first, because run will change things + # no state, wrong schema, failure. + self.run_dbt(['test', '--target', 'otherschema'], expect_pass=False) + + # no state, run also fails + self.run_dbt(['run', '--target', 'otherschema'], expect_pass=False) + + # defer test, it succeeds + results = self.run_dbt(['test', '-m', 'view_model+', '--state', 'state', '--defer', '--target', 'otherschema']) + + # with state it should work though + results = self.run_dbt(['run', '-m', 'view_model', '--state', 'state', '--defer', '--target', 'otherschema']) + assert self.other_schema not in results[0].node.compiled_sql + assert self.unique_schema() in results[0].node.compiled_sql + + with open('target/manifest.json') as fp: + data = json.load(fp) + assert data['nodes']['seed.test.seed']['deferred'] + + assert len(results) == 1 + + @use_profile('bigquery') + def test_bigquery_state_changetarget(self): + self.run_and_defer() diff --git a/tests/integration/deprecation_tests/adapter-macro-macros/macros.sql b/tests/integration/deprecation_tests/adapter-macro-macros/macros.sql new file mode 100644 index 000000000..b03ca28f1 --- /dev/null +++ b/tests/integration/deprecation_tests/adapter-macro-macros/macros.sql @@ -0,0 +1,17 @@ +{% macro some_macro(arg1, arg2) -%} + {{ adapter_macro('some_macro', arg1, arg2) }} +{%- endmacro %} + + +{% macro default__some_macro(arg1, arg2) %} + {% do exceptions.raise_compiler_error('not allowed') %} +{% endmacro %} + +{% macro postgres__some_macro(arg1, arg2) -%} + {{ arg1 }}{{ arg2 }} +{%- endmacro %} + + +{% macro some_other_macro(arg1, arg2) -%} + {{ adapter_macro('test.some_macro', arg1, arg2) }} +{%- endmacro %} diff --git a/tests/integration/deprecation_tests/adapter-macro-models-package/model.sql b/tests/integration/deprecation_tests/adapter-macro-models-package/model.sql new file mode 100644 index 000000000..7b86017b4 --- /dev/null +++ b/tests/integration/deprecation_tests/adapter-macro-models-package/model.sql @@ -0,0 +1,4 @@ +{% if some_other_macro('foo', 'bar') != 'foobar' %} + {% do exceptions.raise_compiler_error('invalid foobar') %} +{% endif %} +select 1 as id diff --git a/tests/integration/deprecation_tests/adapter-macro-models/model.sql b/tests/integration/deprecation_tests/adapter-macro-models/model.sql new file mode 100644 index 000000000..37621d3d2 --- /dev/null +++ b/tests/integration/deprecation_tests/adapter-macro-models/model.sql @@ -0,0 +1,4 @@ +{% if some_macro('foo', 'bar') != 'foobar' %} + {% do exceptions.raise_compiler_error('invalid foobar') %} +{% endif %} +select 1 as id diff --git a/tests/integration/deprecation_tests/test_deprecations.py b/tests/integration/deprecation_tests/test_deprecations.py new file mode 100644 index 000000000..cf394a34a --- /dev/null +++ b/tests/integration/deprecation_tests/test_deprecations.py @@ -0,0 +1,62 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + +from dbt import deprecations +import dbt.exceptions + + +class BaseTestDeprecations(DBTIntegrationTest): + def setUp(self): + super().setUp() + deprecations.reset_deprecations() + + @property + def schema(self): + return "deprecation_test" + + @staticmethod + def dir(path): + return path.lstrip("/") + + +class TestAdapterMacroDeprecation(BaseTestDeprecations): + @property + def models(self): + return self.dir('adapter-macro-models') + + @property + def project_config(self): + return { + 'config-version': 2, + 'macro-paths': [self.dir('adapter-macro-macros')] + } + + @use_profile('bigquery') + def test_bigquery_adapter_macro(self): + self.assertEqual(deprecations.active_deprecations, set()) + # picked up the default -> error + with self.assertRaises(dbt.exceptions.CompilationException) as exc: + self.run_dbt(expect_pass=False) + exc_str = ' '.join(str(exc.exception).split()) # flatten all whitespace + assert 'not allowed' in exc_str # we saw the default macro + + +class TestAdapterMacroDeprecationPackages(BaseTestDeprecations): + @property + def models(self): + return self.dir('adapter-macro-models-package') + + @property + def project_config(self): + return { + 'config-version': 2, + 'macro-paths': [self.dir('adapter-macro-macros')] + } + + @use_profile('bigquery') + def test_bigquery_adapter_macro_pkg(self): + self.assertEqual(deprecations.active_deprecations, set()) + # picked up the default -> error + with self.assertRaises(dbt.exceptions.CompilationException) as exc: + self.run_dbt(expect_pass=False) + exc_str = ' '.join(str(exc.exception).split()) # flatten all whitespace + assert 'not allowed' in exc_str # we saw the default macro diff --git a/tests/integration/docs_generate_tests/assets/lorem-ipsum.txt b/tests/integration/docs_generate_tests/assets/lorem-ipsum.txt new file mode 100644 index 000000000..cee7a927c --- /dev/null +++ b/tests/integration/docs_generate_tests/assets/lorem-ipsum.txt @@ -0,0 +1 @@ +Lorem ipsum dolor sit amet, consectetur adipisicing elit, sed do eiusmod tempor incididunt ut labore et dolore magna aliqua. Ut enim ad minim veniam, quis nostrud exercitation ullamco laboris nisi ut aliquip ex ea commodo consequat. Duis aute irure dolor in reprehenderit in voluptate velit esse cillum dolore eu fugiat nulla pariatur. Excepteur sint occaecat cupidatat non proident, sunt in culpa qui officia deserunt mollit anim id est laborum. diff --git a/tests/integration/docs_generate_tests/bq_models/clustered.sql b/tests/integration/docs_generate_tests/bq_models/clustered.sql new file mode 100644 index 000000000..744d2ecb2 --- /dev/null +++ b/tests/integration/docs_generate_tests/bq_models/clustered.sql @@ -0,0 +1,9 @@ +{{ + config( + materialized='table', + partition_by={'field': 'updated_at', 'data_type': 'date'}, + cluster_by=['first_name'] + ) +}} + +select id,first_name,email,ip_address,DATE(updated_at) as updated_at from {{ ref('seed') }} diff --git a/tests/integration/docs_generate_tests/bq_models/multi_clustered.sql b/tests/integration/docs_generate_tests/bq_models/multi_clustered.sql new file mode 100644 index 000000000..e47df02f9 --- /dev/null +++ b/tests/integration/docs_generate_tests/bq_models/multi_clustered.sql @@ -0,0 +1,9 @@ +{{ + config( + materialized='table', + partition_by={'field': 'updated_at', 'data_type': 'date'}, + cluster_by=['first_name','email'] + ) +}} + +select id,first_name,email,ip_address,DATE(updated_at) as updated_at from {{ ref('seed') }} diff --git a/tests/integration/docs_generate_tests/bq_models/nested_table.sql b/tests/integration/docs_generate_tests/bq_models/nested_table.sql new file mode 100644 index 000000000..22f9048d6 --- /dev/null +++ b/tests/integration/docs_generate_tests/bq_models/nested_table.sql @@ -0,0 +1,15 @@ +{{ + config( + materialized='table' + ) +}} + +select + 1 as field_1, + 2 as field_2, + 3 as field_3, + + struct( + 5 as field_5, + 6 as field_6 + ) as nested_field diff --git a/tests/integration/docs_generate_tests/bq_models/nested_view.sql b/tests/integration/docs_generate_tests/bq_models/nested_view.sql new file mode 100644 index 000000000..29e7b4a2d --- /dev/null +++ b/tests/integration/docs_generate_tests/bq_models/nested_view.sql @@ -0,0 +1,7 @@ +{{ + config( + materialized='view' + ) +}} + +select * from {{ ref('nested_table') }} diff --git a/tests/integration/docs_generate_tests/bq_models/schema.yml b/tests/integration/docs_generate_tests/bq_models/schema.yml new file mode 100644 index 000000000..87ebd8c71 --- /dev/null +++ b/tests/integration/docs_generate_tests/bq_models/schema.yml @@ -0,0 +1,44 @@ +version: 2 + +models: + - name: nested_view + description: "The test model" + columns: + - name: field_1 + description: The first field + - name: field_2 + description: The second field + - name: field_3 + description: The third field + - name: nested_field.field_4 + description: The first nested field + - name: nested_field.field_5 + description: The second nested field + - name: clustered + description: "A clustered and partitioned copy of the test model" + columns: + - name: id + description: The user id + - name: first_name + description: The user's name + - name: email + description: The user's email + - name: ip_address + description: The user's IP address + - name: updated_at + description: When the user was updated + - name: multi_clustered + description: "A clustered and partitioned copy of the test model, clustered on multiple columns" + columns: + - name: id + description: The user id + - name: first_name + description: The user's name + - name: email + description: The user's email + - name: ip_address + description: The user's IP address + - name: updated_at + description: When the user was updated + + diff --git a/tests/integration/docs_generate_tests/bq_models_noschema/disabled.sql b/tests/integration/docs_generate_tests/bq_models_noschema/disabled.sql new file mode 100644 index 000000000..e4368a859 --- /dev/null +++ b/tests/integration/docs_generate_tests/bq_models_noschema/disabled.sql @@ -0,0 +1,2 @@ +{{ config(disabled=true, schema='notrealnotreal') }} +select 1 as id diff --git a/tests/integration/docs_generate_tests/bq_models_noschema/model.sql b/tests/integration/docs_generate_tests/bq_models_noschema/model.sql new file mode 100644 index 000000000..2fb872e84 --- /dev/null +++ b/tests/integration/docs_generate_tests/bq_models_noschema/model.sql @@ -0,0 +1,2 @@ +{{ config(schema=var('extra_schema')) }} +select 1 as id diff --git a/tests/integration/docs_generate_tests/cross_db_models/model.sql b/tests/integration/docs_generate_tests/cross_db_models/model.sql new file mode 100644 index 000000000..0759a4572 --- /dev/null +++ b/tests/integration/docs_generate_tests/cross_db_models/model.sql @@ -0,0 +1,8 @@ +{{ + config( + materialized='view', + database=var('alternate_db') + ) +}} + +select * from {{ ref('seed') }} diff --git a/tests/integration/docs_generate_tests/cross_db_models/schema.yml b/tests/integration/docs_generate_tests/cross_db_models/schema.yml new file mode 100644 index 000000000..6f3fc6619 --- /dev/null +++ b/tests/integration/docs_generate_tests/cross_db_models/schema.yml @@ -0,0 +1,21 @@ +version: 2 + +models: + - name: model + description: "The test model" + columns: + - name: id + description: The user ID number + tests: + - unique + - not_null + - name: first_name + description: The user's first name + - name: email + description: The user's email + - name: ip_address + description: The user's IP address + - name: updated_at + description: The last time this user's email was updated + tests: + - test.nothing diff --git a/tests/integration/docs_generate_tests/fail_macros/failure.sql b/tests/integration/docs_generate_tests/fail_macros/failure.sql new file mode 100644 index 000000000..f0519ed24 --- /dev/null +++ b/tests/integration/docs_generate_tests/fail_macros/failure.sql @@ -0,0 +1,3 @@ +{% macro get_catalog(information_schema, schemas) %} + {% do exceptions.raise_compiler_error('rejected: no catalogs for you') %} +{% endmacro %} diff --git a/tests/integration/docs_generate_tests/macros/dummy_test.sql b/tests/integration/docs_generate_tests/macros/dummy_test.sql new file mode 100644 index 000000000..adf4d7d93 --- /dev/null +++ b/tests/integration/docs_generate_tests/macros/dummy_test.sql @@ -0,0 +1,9 @@ + +{% test nothing(model) %} + +-- a silly test to make sure that table-level tests show up in the manifest +-- without a column_name field +select 0 + +{% endtest %} + diff --git a/tests/integration/docs_generate_tests/macros/macro.md b/tests/integration/docs_generate_tests/macros/macro.md new file mode 100644 index 000000000..6a48ffc1b --- /dev/null +++ b/tests/integration/docs_generate_tests/macros/macro.md @@ -0,0 +1,8 @@ + +{% docs macro_info %} +My custom test that I wrote that does nothing +{% enddocs %} + +{% docs macro_arg_info %} +The model for my custom test +{% enddocs %} diff --git a/tests/integration/docs_generate_tests/macros/schema.yml b/tests/integration/docs_generate_tests/macros/schema.yml new file mode 100644 index 000000000..6b33f4730 --- /dev/null +++ b/tests/integration/docs_generate_tests/macros/schema.yml @@ -0,0 +1,10 @@ +version: 2 +macros: + - name: test_nothing + description: "{{ doc('macro_info') }}" + meta: + some_key: 100 + arguments: + - name: model + type: Relation + description: "{{ doc('macro_arg_info') }}" diff --git a/tests/integration/docs_generate_tests/models/model.sql b/tests/integration/docs_generate_tests/models/model.sql new file mode 100644 index 000000000..0759a4572 --- /dev/null +++ b/tests/integration/docs_generate_tests/models/model.sql @@ -0,0 +1,8 @@ +{{ + config( + materialized='view', + database=var('alternate_db') + ) +}} + +select * from {{ ref('seed') }} diff --git a/tests/integration/docs_generate_tests/models/readme.md b/tests/integration/docs_generate_tests/models/readme.md new file mode 100644 index 000000000..d59a7f447 --- /dev/null +++ b/tests/integration/docs_generate_tests/models/readme.md @@ -0,0 +1 @@ +This is a readme.md file with {{ invalid-ish jinja }} in it diff --git a/tests/integration/docs_generate_tests/models/schema.yml b/tests/integration/docs_generate_tests/models/schema.yml new file mode 100644 index 000000000..600be50ea --- /dev/null +++ b/tests/integration/docs_generate_tests/models/schema.yml @@ -0,0 +1,82 @@ +version: 2 + +models: + - name: model + description: "The test model" + docs: + show: false + columns: + - name: id + description: The user ID number + tests: + - unique + - not_null + - name: first_name + description: The user's first name + - name: email + description: The user's email + - name: ip_address + description: The user's IP address + - name: updated_at + description: The last time this user's email was updated + tests: + - test.nothing + + - name: second_model + description: "The second test model" + docs: + show: false + columns: + - name: id + description: The user ID number + - name: first_name + description: The user's first name + - name: email + description: The user's email + - name: ip_address + description: The user's IP address + - name: updated_at + description: The last time this user's email was updated + + +sources: + - name: my_source + description: "My source" + loader: a_loader + schema: "{{ var('test_schema') }}" + tables: + - name: my_table + description: "My table" + identifier: seed + quoting: + identifier: True + columns: + - name: id + description: "An ID field" + + +exposures: + - name: simple_exposure + type: dashboard + depends_on: + - ref('model') + - source('my_source', 'my_table') + owner: + email: something@example.com + - name: notebook_exposure + type: notebook + depends_on: + - ref('model') + - ref('second_model') + owner: + email: something@example.com + name: Some name + description: > + A description of the complex exposure + maturity: medium + meta: + tool: 'my_tool' + languages: + - python + tags: ['my_department'] + url: http://example.com/notebook/1 diff --git a/tests/integration/docs_generate_tests/models/second_model.sql b/tests/integration/docs_generate_tests/models/second_model.sql new file mode 100644 index 000000000..37f83155c --- /dev/null +++ b/tests/integration/docs_generate_tests/models/second_model.sql @@ -0,0 +1,8 @@ +{{ + config( + materialized='view', + schema='test', + ) +}} + +select * from {{ ref('seed') }} diff --git a/tests/integration/docs_generate_tests/ref_models/docs.md b/tests/integration/docs_generate_tests/ref_models/docs.md new file mode 100644 index 000000000..c5ad96862 --- /dev/null +++ b/tests/integration/docs_generate_tests/ref_models/docs.md @@ -0,0 +1,31 @@ +{% docs ephemeral_summary %} +A summmary table of the ephemeral copy of the seed data +{% enddocs %} + +{% docs summary_first_name %} +The first name being summarized +{% enddocs %} + +{% docs summary_count %} +The number of instances of the first name +{% enddocs %} + +{% docs view_summary %} +A view of the summary of the ephemeral copy of the seed data +{% enddocs %} + +{% docs source_info %} +My source +{% enddocs %} + +{% docs table_info %} +My table +{% enddocs %} + +{% docs column_info %} +An ID field +{% enddocs %} + +{% docs notebook_info %} +A description of the complex exposure +{% enddocs %} diff --git a/tests/integration/docs_generate_tests/ref_models/ephemeral_copy.sql b/tests/integration/docs_generate_tests/ref_models/ephemeral_copy.sql new file mode 100644 index 000000000..3f7e698ce --- /dev/null +++ b/tests/integration/docs_generate_tests/ref_models/ephemeral_copy.sql @@ -0,0 +1,7 @@ +{{ + config( + materialized = "ephemeral" + ) +}} + +select * from {{ source("my_source", "my_table") }} diff --git a/tests/integration/docs_generate_tests/ref_models/ephemeral_summary.sql b/tests/integration/docs_generate_tests/ref_models/ephemeral_summary.sql new file mode 100644 index 000000000..4dba8a564 --- /dev/null +++ b/tests/integration/docs_generate_tests/ref_models/ephemeral_summary.sql @@ -0,0 +1,9 @@ +{{ + config( + materialized = "table" + ) +}} + +select first_name, count(*) as ct from {{ref('ephemeral_copy')}} +group by first_name +order by first_name asc diff --git a/tests/integration/docs_generate_tests/ref_models/schema.yml b/tests/integration/docs_generate_tests/ref_models/schema.yml new file mode 100644 index 000000000..6d7dcea05 --- /dev/null +++ b/tests/integration/docs_generate_tests/ref_models/schema.yml @@ -0,0 +1,48 @@ +version: 2 + +models: + - name: ephemeral_summary + description: "{{ doc('ephemeral_summary') }}" + columns: &summary_columns + - name: first_name + description: "{{ doc('summary_first_name') }}" + - name: ct + description: "{{ doc('summary_count') }}" + - name: view_summary + description: "{{ doc('view_summary') }}" + columns: *summary_columns + +sources: + - name: my_source + description: "{{ doc('source_info') }}" + loader: a_loader + schema: "{{ var('test_schema') }}" + quoting: + database: False + identifier: False + tables: + - name: my_table + description: "{{ doc('table_info') }}" + identifier: seed + quoting: + identifier: True + columns: + - name: id + description: "{{ doc('column_info') }}" + +exposures: + - name: notebook_exposure + type: notebook + depends_on: + - ref('view_summary') + owner: + email: something@example.com + name: Some name + description: "{{ doc('notebook_info') }}" + maturity: medium + url: http://example.com/notebook/1 + meta: + tool: 'my_tool' + languages: + - python + tags: ['my_department'] diff --git a/tests/integration/docs_generate_tests/ref_models/view_summary.sql b/tests/integration/docs_generate_tests/ref_models/view_summary.sql new file mode 100644 index 000000000..6ad6c3dd0 --- /dev/null +++ b/tests/integration/docs_generate_tests/ref_models/view_summary.sql @@ -0,0 +1,8 @@ +{{ + config( + materialized = "view" + ) +}} + +select first_name, ct from {{ref('ephemeral_summary')}} +order by ct asc diff --git a/tests/integration/docs_generate_tests/rs_models/model.sql b/tests/integration/docs_generate_tests/rs_models/model.sql new file mode 100644 index 000000000..8ca73c7a0 --- /dev/null +++ b/tests/integration/docs_generate_tests/rs_models/model.sql @@ -0,0 +1,7 @@ +{{ + config( + materialized='view', bind=False + ) +}} + +select * from {{ ref('seed') }} diff --git a/tests/integration/docs_generate_tests/rs_models/schema.yml b/tests/integration/docs_generate_tests/rs_models/schema.yml new file mode 100644 index 000000000..37dd397c8 --- /dev/null +++ b/tests/integration/docs_generate_tests/rs_models/schema.yml @@ -0,0 +1,16 @@ +version: 2 + +models: + - name: model + description: "The test model" + columns: + - name: id + description: The user ID number + - name: first_name + description: The user's first name + - name: email + description: The user's email + - name: ip_address + description: The user's IP address + - name: updated_at + description: The last time this user's email was updated diff --git a/tests/integration/docs_generate_tests/seed/schema.yml b/tests/integration/docs_generate_tests/seed/schema.yml new file mode 100644 index 000000000..ef5e7dc9e --- /dev/null +++ b/tests/integration/docs_generate_tests/seed/schema.yml @@ -0,0 +1,15 @@ +version: 2 +seeds: + - name: seed + description: "The test seed" + columns: + - name: id + description: The user ID number + - name: first_name + description: The user's first name + - name: email + description: The user's email + - name: ip_address + description: The user's IP address + - name: updated_at + description: The last time this user's email was updated diff --git a/tests/integration/docs_generate_tests/seed/seed.csv b/tests/integration/docs_generate_tests/seed/seed.csv new file mode 100644 index 000000000..ef154f552 --- /dev/null +++ b/tests/integration/docs_generate_tests/seed/seed.csv @@ -0,0 +1,2 @@ +id,first_name,email,ip_address,updated_at +1,Larry,lking0@miitbeian.gov.cn,69.135.206.194,2008-09-12 19:08:31 diff --git a/tests/integration/docs_generate_tests/snapshot/snapshot_seed.sql b/tests/integration/docs_generate_tests/snapshot/snapshot_seed.sql new file mode 100644 index 000000000..83bc54fe7 --- /dev/null +++ b/tests/integration/docs_generate_tests/snapshot/snapshot_seed.sql @@ -0,0 +1,11 @@ +{% snapshot snapshot_seed %} +{{ + config( + unique_key='id', + strategy='check', + check_cols='all', + target_schema=var('alternate_schema') + ) +}} +select * from {{ ref('seed') }} +{% endsnapshot %} \ No newline at end of file diff --git a/tests/integration/docs_generate_tests/test_docs_generate.py b/tests/integration/docs_generate_tests/test_docs_generate.py new file mode 100644 index 000000000..a791897ad --- /dev/null +++ b/tests/integration/docs_generate_tests/test_docs_generate.py @@ -0,0 +1,1981 @@ +import hashlib +import json +import os +import random +import shutil +import tempfile +import time +from datetime import datetime +from unittest.mock import ANY, patch + +from pytest import mark +from tests.integration.base import DBTIntegrationTest, use_profile, AnyFloat, \ + AnyString, AnyStringWith, normalize, Normalized + +import dbt.tracking +import dbt.version +from dbt.exceptions import CompilationException + + +def _read_file(path): + with open(path, 'r') as fp: + return fp.read().replace('\r', '').replace('\\r', '') + + +class LineIndifferent: + def __init__(self, expected): + self.expected = expected.replace('\r', '') + + def __eq__(self, other): + return self.expected == other.replace('\r', '') + + def __repr__(self): + return 'LineIndifferent({!r})'.format(self.expected) + + def __str__(self): + return self.__repr__() + + +class OneOf: + def __init__(self, *options): + self.options = options + + def __eq__(self, other): + return any(o == other for o in self.options) + + def __repr__(self): + return 'OneOf(*{!r})'.format(self.options) + + def __str__(self): + return self.__repr__() + + +def _read_json(path): + # read json generated by dbt. + with open(path) as fp: + return json.load(fp) + + +def walk_files(path): + for root, dirs, files in os.walk(path): + for basename in files: + yield os.path.join(root, basename) + + +class TestDocsGenerateEscapes(DBTIntegrationTest): + prefix = "pgtest{}{:04}".format(int(time.time()), random.randint(0, 9999)) + + @property + def schema(self): + return 'docs_generate_029' + + @staticmethod + def dir(path): + return normalize(path) + + @property + def models(self): + return self.dir("trivial_models") + + def run_and_generate(self): + self.assertEqual(len(self.run_dbt(['run'])), 1) + os.remove(normalize('target/manifest.json')) + os.remove(normalize('target/run_results.json')) + self.run_dbt(['docs', 'generate']) + + +class TestDocsGenerate(DBTIntegrationTest): + setup_alternate_db = True + + def adapter_case(self, value): + return value.lower() + + def setUp(self): + super().setUp() + self.maxDiff = None + self.alternate_schema = self.unique_schema() + '_test' + self.alternate_schema = self.alternate_schema.upper() + + self._created_schemas.add(self.alternate_schema) + os.environ['DBT_ENV_CUSTOM_ENV_env_key'] = 'env_value' + + def tearDown(self): + super().tearDown() + del os.environ['DBT_ENV_CUSTOM_ENV_env_key'] + + @property + def schema(self): + return 'docs_generate_029' + + @staticmethod + def dir(path): + return normalize(path) + + @property + def models(self): + return self.dir("models") + + @property + def project_config(self): + return { + 'config-version': 2, + 'quoting': { + 'identifier': False + } + } + + def run_and_generate(self, extra=None, seed_count=1, model_count=2, alternate_db=None, args=None): + if alternate_db is None: + alternate_db = self.alternative_database + project = { + "data-paths": [self.dir("seed")], + 'macro-paths': [self.dir('macros')], + 'snapshot-paths': [self.dir('snapshot')], + 'vars': { + 'alternate_db': alternate_db, + 'alternate_schema': self.alternate_schema, + }, + 'seeds': { + 'quote_columns': True, + }, + } + if extra: + project.update(extra) + self.use_default_project(project) + + vars_arg = '--vars={{test_schema: {}}}'.format(self.unique_schema()) + + self.assertEqual(len(self.run_dbt(["seed", vars_arg])), seed_count) + self.assertEqual(len(self.run_dbt(['run', vars_arg])), model_count) + os.remove(normalize('target/manifest.json')) + os.remove(normalize('target/run_results.json')) + self.generate_start_time = datetime.utcnow() + base_args = ['docs', 'generate', vars_arg] + if args: + base_args.extend(args) + self.run_dbt(base_args) + + def _no_stats(self): + return { + 'has_stats': { + 'id': 'has_stats', + 'label': 'Has Stats?', + 'value': False, + 'description': 'Indicates whether there are statistics for this table', + 'include': False, + }, + } + + def _bigquery_stats(self, is_table, partition=None, cluster=None): + stats = {} + + if is_table: + stats.update({ + 'num_bytes': { + 'id': 'num_bytes', + 'label': AnyString(), + 'value': AnyFloat(), + 'description': AnyString(), + 'include': True, + }, + 'num_rows': { + 'id': 'num_rows', + 'label': AnyString(), + 'value': AnyFloat(), + 'description': AnyString(), + 'include': True, + } + }) + + if partition is not None: + stats.update({ + 'partitioning_type': { + 'id': 'partitioning_type', + 'label': AnyString(), + 'value': partition, + 'description': AnyString(), + 'include': True + } + }) + + if cluster is not None: + stats.update({ + 'clustering_fields': { + 'id': 'clustering_fields', + 'label': AnyString(), + 'value': cluster, + 'description': AnyString(), + 'include': True + } + }) + + has_stats = { + 'id': 'has_stats', + 'label': 'Has Stats?', + 'value': bool(stats), + 'description': 'Indicates whether there are statistics for this table', + 'include': False, + } + stats['has_stats'] = has_stats + + return stats + + def _expected_catalog(self, id_type, text_type, time_type, view_type, + table_type, model_stats, seed_stats=None, case=None, + case_columns=False, model_database=None): + if case is None: + def case(x): return x + col_case = case if case_columns else lambda x: x + + if seed_stats is None: + seed_stats = model_stats + + if model_database is None: + model_database = self.default_database + my_schema_name = self.unique_schema() + role = self.get_role() + expected_cols = { + col_case('id'): { + 'name': col_case('id'), + 'index': 1, + 'type': id_type, + 'comment': None, + }, + col_case('first_name'): { + 'name': col_case('first_name'), + 'index': 2, + 'type': text_type, + 'comment': None, + }, + col_case('email'): { + 'name': col_case('email'), + 'index': 3, + 'type': text_type, + 'comment': None, + }, + col_case('ip_address'): { + 'name': col_case('ip_address'), + 'index': 4, + 'type': text_type, + 'comment': None, + }, + col_case('updated_at'): { + 'name': col_case('updated_at'), + 'index': 5, + 'type': time_type, + 'comment': None, + }, + } + return { + 'nodes': { + 'model.test.model': { + 'unique_id': 'model.test.model', + 'metadata': { + 'schema': my_schema_name, + 'database': model_database, + 'name': case('model'), + 'type': view_type, + 'comment': None, + 'owner': role, + }, + 'stats': model_stats, + 'columns': expected_cols, + }, + 'model.test.second_model': { + 'unique_id': 'model.test.second_model', + 'metadata': { + 'schema': self.alternate_schema, + 'database': self.default_database, + 'name': case('second_model'), + 'type': view_type, + 'comment': None, + 'owner': role, + }, + 'stats': model_stats, + 'columns': expected_cols, + }, + 'seed.test.seed': { + 'unique_id': 'seed.test.seed', + 'metadata': { + 'schema': my_schema_name, + 'database': self.default_database, + 'name': case('seed'), + 'type': table_type, + 'comment': None, + 'owner': role, + }, + 'stats': seed_stats, + 'columns': expected_cols, + }, + }, + 'sources': { + 'source.test.my_source.my_table': { + 'unique_id': 'source.test.my_source.my_table', + 'metadata': { + 'schema': my_schema_name, + 'database': self.default_database, + 'name': case('seed'), + 'type': table_type, + 'comment': None, + 'owner': role, + }, + 'stats': seed_stats, + 'columns': expected_cols, + }, + }, + } + + def get_role(self): + return None + + def expected_bigquery_catalog(self): + return self._expected_catalog( + id_type='INT64', + text_type='STRING', + time_type='DATETIME', + view_type='view', + table_type='table', + model_stats=self._bigquery_stats(False), + seed_stats=self._bigquery_stats(True), + model_database=self.alternative_database, + ) + + @staticmethod + def _clustered_bigquery_columns(update_type): + return { + 'id': { + 'comment': None, + 'index': 1, + 'name': 'id', + 'type': 'INT64', + }, + 'first_name': { + 'comment': None, + 'index': 2, + 'name': 'first_name', + 'type': 'STRING', + }, + 'email': { + 'comment': None, + 'index': 3, + 'name': 'email', + 'type': 'STRING', + }, + 'ip_address': { + 'comment': None, + 'index': 4, + 'name': 'ip_address', + 'type': 'STRING', + }, + 'updated_at': { + 'comment': None, + 'index': 5, + 'name': 'updated_at', + 'type': update_type, + }, + } + + def expected_bigquery_complex_catalog(self): + my_schema_name = self.unique_schema() + role = self.get_role() + table_stats = self._bigquery_stats(True) + clustering_stats = self._bigquery_stats(True, partition='updated_at', + cluster='first_name') + multi_clustering_stats = self._bigquery_stats(True, partition='updated_at', + cluster='first_name, email') + nesting_columns = { + 'field_1': { + 'name': 'field_1', + 'index': 1, + 'type': 'INT64', + 'comment': None + }, + 'field_2': { + 'name': 'field_2', + 'index': 2, + 'type': 'INT64', + 'comment': None + }, + 'field_3': { + 'name': 'field_3', + 'index': 3, + 'type': 'INT64', + 'comment': None + }, + 'nested_field': { + 'name': 'nested_field', + 'index': 4, + 'type': 'STRUCT', + 'comment': None + }, + 'nested_field.field_5': { + 'name': 'nested_field.field_5', + 'index': 5, + 'type': 'INT64', + 'comment': None + }, + 'nested_field.field_6': { + 'name': 'nested_field.field_6', + 'index': 6, + 'type': 'INT64', + 'comment': None + } + } + + return { + 'nodes': { + 'model.test.clustered': { + 'unique_id': 'model.test.clustered', + 'metadata': { + 'comment': None, + 'name': 'clustered', + 'owner': None, + 'schema': my_schema_name, + 'database': self.default_database, + 'type': 'table' + }, + 'stats': clustering_stats, + 'columns': self._clustered_bigquery_columns('DATE'), + }, + 'model.test.multi_clustered': { + 'unique_id': 'model.test.multi_clustered', + 'metadata': { + 'comment': None, + 'name': 'multi_clustered', + 'owner': None, + 'schema': my_schema_name, + 'database': self.default_database, + 'type': 'table' + }, + 'stats': multi_clustering_stats, + 'columns': self._clustered_bigquery_columns('DATE'), + }, + 'seed.test.seed': { + 'unique_id': 'seed.test.seed', + 'metadata': { + 'comment': None, + 'name': 'seed', + 'owner': None, + 'schema': my_schema_name, + 'database': self.default_database, + 'type': 'table', + }, + 'stats': table_stats, + 'columns': self._clustered_bigquery_columns('DATETIME'), + }, + 'model.test.nested_view': { + 'unique_id': 'model.test.nested_view', + 'metadata': { + 'schema': my_schema_name, + 'database': self.default_database, + 'name': 'nested_view', + 'type': 'view', + 'owner': role, + 'comment': None + }, + 'stats': self._bigquery_stats(False), + 'columns': nesting_columns, + }, + 'model.test.nested_table': { + 'unique_id': 'model.test.nested_table', + 'metadata': { + 'schema': my_schema_name, + 'database': self.default_database, + 'name': 'nested_table', + 'type': 'table', + 'owner': role, + 'comment': None + }, + 'stats': table_stats, + 'columns': nesting_columns, + }, + }, + 'sources': {}, + } + + def verify_catalog(self, expected): + self.assertTrue(os.path.exists('./target/catalog.json')) + + catalog = _read_json('./target/catalog.json') + + assert set(catalog) == {'errors', 'metadata', 'nodes', 'sources'} + + self.verify_metadata( + catalog['metadata'], 'https://schemas.getdbt.com/dbt/catalog/v1.json') + assert not catalog['errors'] + + for key in 'nodes', 'sources': + assert catalog[key] == expected[key] + + def verify_manifest_macros(self, manifest, expected=None): + self.assertIn('macros', manifest) + if expected is None: + self._verify_generic_macro_structure(manifest) + return + for unique_id, expected_macro in expected.items(): + self.assertIn(unique_id, manifest['macros']) + actual_macro = manifest['macros'][unique_id] + self.assertEqual(expected_macro, actual_macro) + + def rendered_model_config(self, **updates): + result = { + 'database': None, + 'schema': None, + 'alias': None, + 'enabled': True, + 'materialized': 'view', + 'pre-hook': [], + 'post-hook': [], + 'column_types': {}, + 'quoting': {}, + 'tags': [], + 'persist_docs': {}, + 'full_refresh': None, + 'on_schema_change': 'ignore', + 'meta': {}, + } + result.update(updates) + return result + + def unrendered_model_config(self, **updates): + return updates + + def rendered_seed_config(self, **updates): + result = { + 'enabled': True, + 'materialized': 'seed', + 'persist_docs': {}, + 'pre-hook': [], + 'post-hook': [], + 'column_types': {}, + 'quoting': {}, + 'tags': [], + 'quote_columns': True, + 'full_refresh': None, + 'on_schema_change': 'ignore', + 'database': None, + 'schema': None, + 'alias': None, + 'meta': {}, + } + result.update(updates) + return result + + def unrendered_seed_config(self, **updates): + result = {'quote_columns': True} + result.update(updates) + return result + + def rendered_snapshot_config(self, **updates): + result = { + 'database': None, + 'schema': None, + 'alias': None, + 'enabled': True, + 'materialized': 'snapshot', + 'pre-hook': [], + 'post-hook': [], + 'column_types': {}, + 'quoting': {}, + 'tags': [], + 'persist_docs': {}, + 'full_refresh': None, + 'on_schema_change': 'ignore', + 'strategy': 'check', + 'check_cols': 'all', + 'unique_key': 'id', + 'target_schema': None, + 'meta': {}, + } + result.update(updates) + return result + + def unrendered_snapshot_config(self, **updates): + result = { + 'check_cols': 'all', + 'strategy': 'check', + 'target_schema': None, + 'unique_key': 'id' + } + result.update(updates) + return result + + def rendered_tst_config(self, **updates): + result = { + 'enabled': True, + 'materialized': 'test', + 'tags': [], + 'severity': 'ERROR', + 'store_failures': None, + 'warn_if': '!= 0', + 'error_if': '!= 0', + 'fail_calc': 'count(*)', + 'where': None, + 'limit': None, + 'database': None, + 'schema': 'dbt_test__audit', + 'alias': None, + 'meta': {}, + } + result.update(updates) + return result + + def unrendered_tst_config(self, **updates): + result = {} + result.update(updates) + return result + + def _verify_generic_macro_structure(self, manifest): + # just test a known global macro to avoid having to update this every + # time they change. + self.assertIn('macro.dbt.column_list', manifest['macros']) + macro = manifest['macros']['macro.dbt.column_list'] + self.assertEqual( + set(macro), + { + 'path', 'original_file_path', 'package_name', + 'root_path', 'name', 'unique_id', 'tags', 'resource_type', + 'depends_on', 'meta', 'description', 'patch_path', 'arguments', + 'macro_sql', 'docs', 'created_at', + } + ) + # Don't compare the sql, just make sure it exists + self.assertTrue(len(macro['macro_sql']) > 10) + without_sql = { + k: v for k, v in macro.items() + if k not in {'macro_sql'} + } + # Windows means we can't hard-code these. + helpers_path = Normalized('macros/materializations/helpers.sql') + root_path = Normalized(os.path.join( + self.dbt_core_install_root, 'include', 'global_project' + )) + self.assertEqual( + { + 'path': helpers_path, + 'original_file_path': helpers_path, + 'package_name': 'dbt', + 'root_path': root_path, + 'name': 'column_list', + 'unique_id': 'macro.dbt.column_list', + 'created_at': ANY, + 'tags': [], + 'resource_type': 'macro', + 'depends_on': {'macros': []}, + 'description': '', + 'docs': {'show': True}, + 'patch_path': None, + 'meta': {}, + 'arguments': [], + }, + without_sql, + ) + + def expected_seeded_manifest(self, model_database=None, quote_model=False): + models_path = self.dir('models') + model_sql_path = os.path.join(models_path, 'model.sql') + second_model_sql_path = os.path.join(models_path, 'second_model.sql') + model_schema_yml_path = os.path.join(models_path, 'schema.yml') + seed_schema_yml_path = os.path.join(self.dir('seed'), 'schema.yml') + seed_path = self.dir(os.path.join('seed', 'seed.csv')) + snapshot_path = self.dir(os.path.join('snapshot', 'snapshot_seed.sql')) + + my_schema_name = self.unique_schema() + test_audit_schema = my_schema_name + '_dbt_test__audit' + + if model_database is None: + model_database = self.alternative_database + + model_config = self.rendered_model_config(database=model_database) + second_config = self.rendered_model_config( + schema=self.alternate_schema[-4:]) + + unrendered_model_config = self.unrendered_model_config( + database=model_database, materialized='view') + unrendered_second_config = self.unrendered_model_config( + schema=self.alternate_schema[-4:], materialized='view') + + seed_config = self.rendered_seed_config() + unrendered_seed_config = self.unrendered_seed_config() + + test_config = self.rendered_tst_config() + unrendered_test_config = self.unrendered_tst_config() + + snapshot_config = self.rendered_snapshot_config( + target_schema=self.alternate_schema) + unrendered_snapshot_config = self.unrendered_snapshot_config( + target_schema=self.alternate_schema + ) + + quote_database = quote_schema = True + relation_name_node_format = self._relation_name_format( + quote_database, quote_schema, quote_model + ) + relation_name_source_format = self._relation_name_format( + quote_database, quote_schema, quote_identifier=True + ) + + return { + 'dbt_schema_version': 'https://schemas.getdbt.com/dbt/manifest/v3.json', + 'dbt_version': dbt.version.__version__, + 'nodes': { + 'model.test.model': { + 'compiled_path': Normalized('target/compiled/test/models/model.sql'), + 'build_path': None, + 'created_at': ANY, + 'name': 'model', + 'root_path': self.test_root_realpath, + 'relation_name': relation_name_node_format.format( + model_database, my_schema_name, 'model' + ), + 'resource_type': 'model', + 'path': 'model.sql', + 'original_file_path': model_sql_path, + 'package_name': 'test', + 'raw_sql': LineIndifferent(_read_file(model_sql_path).rstrip('\r\n')), + 'refs': [['seed']], + 'sources': [], + 'depends_on': {'nodes': ['seed.test.seed'], 'macros': []}, + 'unique_id': 'model.test.model', + 'fqn': ['test', 'model'], + 'tags': [], + 'meta': {}, + 'config': model_config, + 'schema': my_schema_name, + 'database': model_database, + 'deferred': False, + 'alias': 'model', + 'description': 'The test model', + 'columns': { + 'id': { + 'name': 'id', + 'description': 'The user ID number', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'first_name': { + 'name': 'first_name', + 'description': "The user's first name", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'email': { + 'name': 'email', + 'description': "The user's email", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'ip_address': { + 'name': 'ip_address', + 'description': "The user's IP address", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'updated_at': { + 'name': 'updated_at', + 'description': "The last time this user's email was updated", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + }, + 'patch_path': 'test://' + model_schema_yml_path, + 'docs': {'show': False}, + 'compiled': True, + 'compiled_sql': ANY, + 'extra_ctes_injected': True, + 'extra_ctes': [], + 'checksum': self._checksum_file(model_sql_path), + 'unrendered_config': unrendered_model_config, + }, + 'model.test.second_model': { + 'compiled_path': Normalized('target/compiled/test/models/second_model.sql'), + 'build_path': None, + 'created_at': ANY, + 'name': 'second_model', + 'root_path': self.test_root_realpath, + 'relation_name': relation_name_node_format.format( + self.default_database, self.alternate_schema, + 'second_model' + ), + 'resource_type': 'model', + 'path': 'second_model.sql', + 'original_file_path': second_model_sql_path, + 'package_name': 'test', + 'raw_sql': LineIndifferent(_read_file(second_model_sql_path).rstrip('\r\n')), + 'refs': [['seed']], + 'sources': [], + 'depends_on': {'nodes': ['seed.test.seed'], 'macros': []}, + 'unique_id': 'model.test.second_model', + 'fqn': ['test', 'second_model'], + 'tags': [], + 'meta': {}, + 'config': second_config, + 'schema': self.alternate_schema, + 'database': self.default_database, + 'deferred': False, + 'alias': 'second_model', + 'description': 'The second test model', + 'columns': { + 'id': { + 'name': 'id', + 'description': 'The user ID number', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'first_name': { + 'name': 'first_name', + 'description': "The user's first name", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'email': { + 'name': 'email', + 'description': "The user's email", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'ip_address': { + 'name': 'ip_address', + 'description': "The user's IP address", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'updated_at': { + 'name': 'updated_at', + 'description': "The last time this user's email was updated", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + }, + 'patch_path': 'test://' + model_schema_yml_path, + 'docs': {'show': False}, + 'compiled': True, + 'compiled_sql': ANY, + 'extra_ctes_injected': True, + 'extra_ctes': [], + 'checksum': self._checksum_file(second_model_sql_path), + 'unrendered_config': unrendered_second_config + }, + 'seed.test.seed': { + 'compiled_path': None, + 'build_path': None, + 'created_at': ANY, + 'compiled': True, + 'compiled_sql': '', + 'config': seed_config, + 'patch_path': 'test://' + seed_schema_yml_path, + 'path': 'seed.csv', + 'name': 'seed', + 'root_path': self.test_root_realpath, + 'relation_name': relation_name_node_format.format( + self.default_database, my_schema_name, 'seed' + ), + 'resource_type': 'seed', + 'raw_sql': '', + 'package_name': 'test', + 'original_file_path': seed_path, + 'refs': [], + 'sources': [], + 'depends_on': {'nodes': [], 'macros': []}, + 'unique_id': 'seed.test.seed', + 'fqn': ['test', 'seed'], + 'tags': [], + 'meta': {}, + 'schema': my_schema_name, + 'database': self.default_database, + 'alias': 'seed', + 'deferred': False, + 'description': 'The test seed', + 'columns': { + 'id': { + 'name': 'id', + 'description': 'The user ID number', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'first_name': { + 'name': 'first_name', + 'description': "The user's first name", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'email': { + 'name': 'email', + 'description': "The user's email", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'ip_address': { + 'name': 'ip_address', + 'description': "The user's IP address", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'updated_at': { + 'name': 'updated_at', + 'description': "The last time this user's email was updated", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + }, + 'docs': {'show': True}, + 'compiled': True, + 'compiled_sql': '', + 'extra_ctes_injected': True, + 'extra_ctes': [], + 'checksum': self._checksum_file(seed_path), + 'unrendered_config': unrendered_seed_config, + }, + 'test.test.not_null_model_id.d01cc630e6': { + 'alias': 'not_null_model_id', + 'compiled_path': Normalized('target/compiled/test/models/schema.yml/schema_test/not_null_model_id.sql'), + 'build_path': None, + 'created_at': ANY, + 'column_name': 'id', + 'columns': {}, + 'config': test_config, + 'sources': [], + 'depends_on': { + 'macros': ['macro.dbt.test_not_null', 'macro.dbt.get_where_subquery'], + 'nodes': ['model.test.model'], + }, + 'deferred': False, + 'description': '', + 'fqn': ['test', 'schema_test', 'not_null_model_id'], + 'name': 'not_null_model_id', + 'original_file_path': model_schema_yml_path, + 'package_name': 'test', + 'patch_path': None, + 'path': Normalized('schema_test/not_null_model_id.sql'), + 'raw_sql': "{{ test_not_null(**_dbt_schema_test_kwargs) }}", + 'refs': [['model']], + 'relation_name': None, + 'resource_type': 'test', + 'root_path': self.test_root_realpath, + 'schema': test_audit_schema, + 'database': self.default_database, + 'tags': ['schema'], + 'meta': {}, + 'unique_id': 'test.test.not_null_model_id.d01cc630e6', + 'docs': {'show': True}, + 'compiled': True, + 'compiled_sql': AnyStringWith('where id is null'), + 'extra_ctes_injected': True, + 'extra_ctes': [], + 'test_metadata': { + 'namespace': None, + 'name': 'not_null', + 'kwargs': { + 'column_name': 'id', + 'model': "{{ get_where_subquery(ref('model')) }}", + }, + }, + 'checksum': {'name': 'none', 'checksum': ''}, + 'unrendered_config': unrendered_test_config, + }, + 'snapshot.test.snapshot_seed': { + 'alias': 'snapshot_seed', + 'compiled_path': None, + 'build_path': None, + 'created_at': ANY, + 'checksum': self._checksum_file(snapshot_path), + 'columns': {}, + 'compiled': True, + 'compiled_sql': ANY, + 'config': snapshot_config, + 'database': self.default_database, + 'deferred': False, + 'depends_on': { + 'macros': [], + 'nodes': ['seed.test.seed'], + }, + 'description': '', + 'docs': {'show': True}, + 'extra_ctes': [], + 'extra_ctes_injected': True, + 'fqn': ['test', 'snapshot_seed', 'snapshot_seed'], + 'meta': {}, + 'name': 'snapshot_seed', + 'original_file_path': snapshot_path, + 'package_name': 'test', + 'patch_path': None, + 'path': normalize('snapshot_seed.sql'), + 'raw_sql': LineIndifferent( + _read_file(snapshot_path) + .replace('{% snapshot snapshot_seed %}', '') + .replace('{% endsnapshot %}', '')), + 'refs': [['seed']], + 'relation_name': relation_name_node_format.format( + self.default_database, self.alternate_schema, + 'snapshot_seed' + ), + 'resource_type': 'snapshot', + 'root_path': self.test_root_realpath, + 'schema': self.alternate_schema, + 'sources': [], + 'tags': [], + 'unique_id': 'snapshot.test.snapshot_seed', + 'unrendered_config': unrendered_snapshot_config, + }, + 'test.test.test_nothing_model_.5d38568946': { + 'alias': 'test_nothing_model_', + 'compiled_path': Normalized('target/compiled/test/models/schema.yml/schema_test/test_nothing_model_.sql'), + 'build_path': None, + 'created_at': ANY, + 'column_name': None, + 'columns': {}, + 'config': test_config, + 'sources': [], + 'depends_on': { + 'macros': ['macro.test.test_nothing', 'macro.dbt.get_where_subquery'], + 'nodes': ['model.test.model'], + }, + 'deferred': False, + 'description': '', + 'fqn': ['test', 'schema_test', 'test_nothing_model_'], + 'name': 'test_nothing_model_', + 'original_file_path': model_schema_yml_path, + 'package_name': 'test', + 'patch_path': None, + 'path': normalize('schema_test/test_nothing_model_.sql'), + 'raw_sql': "{{ test.test_nothing(**_dbt_schema_test_kwargs) }}", + 'refs': [['model']], + 'relation_name': None, + 'resource_type': 'test', + 'root_path': self.test_root_realpath, + 'schema': test_audit_schema, + 'database': self.default_database, + 'tags': ['schema'], + 'meta': {}, + 'unique_id': 'test.test.test_nothing_model_.5d38568946', + 'docs': {'show': True}, + 'compiled': True, + 'compiled_sql': AnyStringWith('select 0'), + 'extra_ctes_injected': True, + 'extra_ctes': [], + 'test_metadata': { + 'namespace': 'test', + 'name': 'nothing', + 'kwargs': { + 'model': "{{ get_where_subquery(ref('model')) }}", + }, + }, + 'checksum': {'name': 'none', 'checksum': ''}, + 'unrendered_config': unrendered_test_config, + }, + 'test.test.unique_model_id.67b76558ff': { + 'alias': 'unique_model_id', + 'compiled_path': Normalized('target/compiled/test/models/schema.yml/schema_test/unique_model_id.sql'), + 'build_path': None, + 'created_at': ANY, + 'column_name': 'id', + 'columns': {}, + 'config': test_config, + 'sources': [], + 'depends_on': { + 'macros': ['macro.dbt.test_unique', 'macro.dbt.get_where_subquery'], + 'nodes': ['model.test.model'], + }, + 'deferred': False, + 'description': '', + 'fqn': ['test', 'schema_test', 'unique_model_id'], + 'name': 'unique_model_id', + 'original_file_path': model_schema_yml_path, + 'package_name': 'test', + 'patch_path': None, + 'path': normalize('schema_test/unique_model_id.sql'), + 'raw_sql': "{{ test_unique(**_dbt_schema_test_kwargs) }}", + 'refs': [['model']], + 'relation_name': None, + 'resource_type': 'test', + 'root_path': self.test_root_realpath, + 'schema': test_audit_schema, + 'database': self.default_database, + 'tags': ['schema'], + 'meta': {}, + 'unique_id': 'test.test.unique_model_id.67b76558ff', + 'docs': {'show': True}, + 'compiled': True, + 'compiled_sql': AnyStringWith('count(*)'), + 'extra_ctes_injected': True, + 'extra_ctes': [], + 'test_metadata': { + 'namespace': None, + 'name': 'unique', + 'kwargs': { + 'column_name': 'id', + 'model': "{{ get_where_subquery(ref('model')) }}", + }, + }, + 'checksum': {'name': 'none', 'checksum': ''}, + 'unrendered_config': unrendered_test_config, + }, + }, + 'sources': { + 'source.test.my_source.my_table': { + 'created_at': ANY, + 'columns': { + 'id': { + 'description': 'An ID field', + 'name': 'id', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + } + }, + 'config': { + 'enabled': True, + }, + 'quoting': { + 'database': None, + 'schema': None, + 'identifier': True, + 'column': None, + }, + 'database': self.default_database, + 'description': 'My table', + 'external': None, + 'freshness': {'error_after': None, 'warn_after': None, 'filter': None}, + 'identifier': 'seed', + 'loaded_at_field': None, + 'loader': 'a_loader', + 'meta': {}, + 'name': 'my_table', + 'original_file_path': self.dir('models/schema.yml'), + 'package_name': 'test', + 'path': self.dir('models/schema.yml'), + 'patch_path': None, + 'relation_name': relation_name_source_format.format( + self.default_database, my_schema_name, 'seed' + ), + 'resource_type': 'source', + 'root_path': self.test_root_realpath, + 'schema': my_schema_name, + 'source_description': 'My source', + 'source_name': 'my_source', + 'source_meta': {}, + 'tags': [], + 'unique_id': 'source.test.my_source.my_table', + 'fqn': ['test', 'my_source', 'my_table'], + 'unrendered_config': {}, + }, + }, + 'exposures': { + 'exposure.test.notebook_exposure': { + 'created_at': ANY, + 'depends_on': { + 'macros': [], + 'nodes': ['model.test.model', 'model.test.second_model'] + }, + 'description': 'A description of the complex exposure\n', + 'fqn': ['test', 'notebook_exposure'], + 'maturity': 'medium', + 'meta': {'tool': 'my_tool', 'languages': ['python']}, + 'tags': ['my_department'], + 'name': 'notebook_exposure', + 'original_file_path': self.dir('models/schema.yml'), + 'owner': { + 'email': 'something@example.com', + 'name': 'Some name' + }, + 'package_name': 'test', + 'path': 'schema.yml', + 'refs': [['model'], ['second_model']], + 'resource_type': 'exposure', + 'root_path': self.test_root_realpath, + 'sources': [], + 'type': 'notebook', + 'unique_id': 'exposure.test.notebook_exposure', + 'url': 'http://example.com/notebook/1' + }, + 'exposure.test.simple_exposure': { + 'created_at': ANY, + 'depends_on': { + 'macros': [], + 'nodes': [ + 'source.test.my_source.my_table', + 'model.test.model' + ], + }, + 'description': '', + 'fqn': ['test', 'simple_exposure'], + 'name': 'simple_exposure', + 'original_file_path': self.dir('models/schema.yml'), + 'owner': { + 'email': 'something@example.com', + 'name': None, + }, + 'package_name': 'test', + 'path': 'schema.yml', + 'refs': [['model']], + 'resource_type': 'exposure', + 'root_path': self.test_root_realpath, + 'sources': [['my_source', 'my_table']], + 'type': 'dashboard', + 'unique_id': 'exposure.test.simple_exposure', + 'url': None, + 'maturity': None, + 'meta': {}, + 'tags': [] + } + }, + 'selectors': {}, + 'parent_map': { + 'model.test.model': ['seed.test.seed'], + 'model.test.second_model': ['seed.test.seed'], + 'exposure.test.notebook_exposure': ['model.test.model', 'model.test.second_model'], + 'exposure.test.simple_exposure': ['model.test.model', 'source.test.my_source.my_table'], + 'seed.test.seed': [], + 'snapshot.test.snapshot_seed': ['seed.test.seed'], + 'source.test.my_source.my_table': [], + 'test.test.not_null_model_id.d01cc630e6': ['model.test.model'], + 'test.test.test_nothing_model_.5d38568946': ['model.test.model'], + 'test.test.unique_model_id.67b76558ff': ['model.test.model'], + }, + 'child_map': { + 'model.test.model': [ + 'exposure.test.notebook_exposure', + 'exposure.test.simple_exposure', + 'test.test.not_null_model_id.d01cc630e6', + 'test.test.test_nothing_model_.5d38568946', + 'test.test.unique_model_id.67b76558ff', + ], + 'model.test.second_model': ['exposure.test.notebook_exposure'], + 'exposure.test.notebook_exposure': [], + 'exposure.test.simple_exposure': [], + 'seed.test.seed': ['model.test.model', + 'model.test.second_model', + 'snapshot.test.snapshot_seed'], + 'snapshot.test.snapshot_seed': [], + 'source.test.my_source.my_table': ['exposure.test.simple_exposure'], + 'test.test.not_null_model_id.d01cc630e6': [], + 'test.test.test_nothing_model_.5d38568946': [], + 'test.test.unique_model_id.67b76558ff': [], + }, + 'docs': { + 'dbt.__overview__': ANY, + 'test.macro_info': ANY, + 'test.macro_arg_info': ANY, + }, + 'disabled': [], + } + + def expected_bigquery_complex_manifest(self): + nested_view_sql_path = self.dir('bq_models/nested_view.sql') + nested_table_sql_path = self.dir('bq_models/nested_table.sql') + clustered_sql_path = self.dir('bq_models/clustered.sql') + multi_clustered_sql_path = self.dir('bq_models/multi_clustered.sql') + seed_path = self.dir('seed/seed.csv') + snapshot_path = self.dir('snapshot/snapshot_seed.sql') + my_schema_name = self.unique_schema() + + return { + 'dbt_schema_version': 'https://schemas.getdbt.com/dbt/manifest/v3.json', + 'dbt_version': dbt.version.__version__, + 'nodes': { + 'model.test.clustered': { + 'alias': 'clustered', + 'config': self.rendered_model_config( + cluster_by=['first_name'], + materialized='table', + partition_by={'field': 'updated_at', + 'data_type': 'date'}, + ), + 'sources': [], + 'depends_on': {'macros': [], 'nodes': ['seed.test.seed']}, + 'fqn': ['test', 'clustered'], + 'compiled_path': Normalized('target/compiled/test/bq_models/clustered.sql'), + 'build_path': None, + 'created_at': ANY, + 'name': 'clustered', + 'original_file_path': clustered_sql_path, + 'package_name': 'test', + 'path': 'clustered.sql', + 'raw_sql': LineIndifferent(_read_file(clustered_sql_path).rstrip('\r\n')), + 'refs': [['seed']], + 'relation_name': '`{0}`.`{1}`.clustered'.format( + self.default_database, my_schema_name + ), + 'resource_type': 'model', + 'root_path': self.test_root_realpath, + 'schema': my_schema_name, + 'database': self.default_database, + 'tags': [], + 'meta': {}, + 'unique_id': 'model.test.clustered', + 'columns': { + 'email': { + 'description': "The user's email", + 'name': 'email', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'first_name': { + 'description': "The user's name", + 'name': 'first_name', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'id': { + 'description': 'The user id', + 'name': 'id', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'ip_address': { + 'description': "The user's IP address", + 'name': 'ip_address', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'updated_at': { + 'description': 'When the user was updated', + 'name': 'updated_at', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + }, + 'deferred': False, + 'description': 'A clustered and partitioned copy of the test model', + 'patch_path': 'test://' + self.dir('bq_models/schema.yml'), + 'docs': {'show': True}, + 'compiled': True, + 'compiled_sql': ANY, + 'extra_ctes_injected': True, + 'extra_ctes': [], + 'checksum': self._checksum_file(clustered_sql_path), + 'unrendered_config': self.unrendered_model_config( + cluster_by=['first_name'], + materialized='table', + partition_by={'field': 'updated_at', + 'data_type': 'date'}, + ), + }, + 'model.test.multi_clustered': { + 'alias': 'multi_clustered', + 'compiled_path': Normalized('target/compiled/test/bq_models/multi_clustered.sql'), + 'build_path': None, + 'created_at': ANY, + 'config': self.rendered_model_config( + cluster_by=['first_name', 'email'], + materialized='table', + partition_by={'field': 'updated_at', + 'data_type': 'date'} + ), + 'sources': [], + 'depends_on': {'macros': [], 'nodes': ['seed.test.seed']}, + 'fqn': ['test', 'multi_clustered'], + 'name': 'multi_clustered', + 'original_file_path': multi_clustered_sql_path, + 'package_name': 'test', + 'path': 'multi_clustered.sql', + 'raw_sql': LineIndifferent(_read_file(multi_clustered_sql_path).rstrip('\r\n')), + 'refs': [['seed']], + 'relation_name': '`{0}`.`{1}`.multi_clustered'.format( + self.default_database, my_schema_name + ), + 'resource_type': 'model', + 'root_path': self.test_root_realpath, + 'schema': my_schema_name, + 'database': self.default_database, + 'tags': [], + 'meta': {}, + 'unique_id': 'model.test.multi_clustered', + 'columns': { + 'email': { + 'description': "The user's email", + 'name': 'email', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'first_name': { + 'description': "The user's name", + 'name': 'first_name', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'id': { + 'description': 'The user id', + 'name': 'id', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'ip_address': { + 'description': "The user's IP address", + 'name': 'ip_address', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'updated_at': { + 'description': 'When the user was updated', + 'name': 'updated_at', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + }, + 'deferred': False, + 'description': 'A clustered and partitioned copy of the test model, clustered on multiple columns', + 'patch_path': 'test://' + self.dir('bq_models/schema.yml'), + 'docs': {'show': True}, + 'compiled': True, + 'compiled_sql': ANY, + 'extra_ctes_injected': True, + 'extra_ctes': [], + 'checksum': self._checksum_file(multi_clustered_sql_path), + 'unrendered_config': self.unrendered_model_config( + cluster_by=['first_name', 'email'], + materialized='table', + partition_by={'field': 'updated_at', + 'data_type': 'date'} + ), + }, + 'model.test.nested_view': { + 'alias': 'nested_view', + 'compiled_path': Normalized('target/compiled/test/bq_models/nested_view.sql'), + 'build_path': None, + 'created_at': ANY, + 'config': self.rendered_model_config(), + 'sources': [], + 'depends_on': { + 'macros': [], + 'nodes': ['model.test.nested_table'] + }, + 'fqn': ['test', 'nested_view'], + 'name': 'nested_view', + 'original_file_path': nested_view_sql_path, + 'package_name': 'test', + 'path': 'nested_view.sql', + 'raw_sql': LineIndifferent(_read_file(nested_view_sql_path).rstrip('\r\n')), + 'refs': [['nested_table']], + 'relation_name': '`{0}`.`{1}`.nested_view'.format( + self.default_database, my_schema_name + ), + 'resource_type': 'model', + 'root_path': self.test_root_realpath, + 'schema': my_schema_name, + 'database': self.default_database, + 'tags': [], + 'meta': {}, + 'unique_id': 'model.test.nested_view', + 'columns': { + 'field_1': { + 'name': 'field_1', + 'description': 'The first field', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'field_2': { + 'name': 'field_2', + 'description': 'The second field', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'field_3': { + 'name': 'field_3', + 'description': 'The third field', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'nested_field.field_4': { + 'name': 'nested_field.field_4', + 'description': 'The first nested field', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'nested_field.field_5': { + 'name': 'nested_field.field_5', + 'description': 'The second nested field', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + }, + 'deferred': False, + 'description': 'The test model', + 'patch_path': 'test://' + self.dir('bq_models/schema.yml'), + 'docs': {'show': True}, + 'compiled': True, + 'compiled_sql': ANY, + 'extra_ctes_injected': True, + 'extra_ctes': [], + 'checksum': self._checksum_file(nested_view_sql_path), + 'unrendered_config': self.unrendered_model_config(materialized='view'), + }, + 'model.test.nested_table': { + 'alias': 'nested_table', + 'compiled_path': Normalized('target/compiled/test/bq_models/nested_table.sql'), + 'build_path': None, + 'created_at': ANY, + 'config': self.rendered_model_config(materialized='table'), + 'sources': [], + 'depends_on': { + 'macros': [], + 'nodes': [] + }, + 'fqn': ['test', 'nested_table'], + 'name': 'nested_table', + 'original_file_path': nested_table_sql_path, + 'package_name': 'test', + 'patch_path': None, + 'path': 'nested_table.sql', + 'raw_sql': LineIndifferent(_read_file(nested_table_sql_path).rstrip('\r\n')), + 'refs': [], + 'relation_name': '`{0}`.`{1}`.nested_table'.format( + self.default_database, my_schema_name + ), + 'resource_type': 'model', + 'root_path': self.test_root_realpath, + 'schema': my_schema_name, + 'database': self.default_database, + 'tags': [], + 'meta': {}, + 'unique_id': 'model.test.nested_table', + 'columns': {}, + 'deferred': False, + 'description': '', + 'docs': {'show': True}, + 'compiled': True, + 'compiled_sql': ANY, + 'extra_ctes_injected': True, + 'extra_ctes': [], + 'checksum': self._checksum_file(nested_table_sql_path), + 'unrendered_config': self.unrendered_model_config(materialized='table'), + }, + 'seed.test.seed': { + 'compiled_path': None, + 'build_path': None, + 'created_at': ANY, + 'patch_path': 'test://' + self.dir('seed/schema.yml'), + 'path': 'seed.csv', + 'name': 'seed', + 'root_path': self.test_root_realpath, + 'relation_name': '`{0}`.`{1}`.seed'.format( + self.default_database, my_schema_name + ), + 'resource_type': 'seed', + 'raw_sql': '', + 'package_name': 'test', + 'original_file_path': seed_path, + 'refs': [], + 'sources': [], + 'depends_on': { + 'nodes': [], + 'macros': [], + }, + 'unique_id': 'seed.test.seed', + 'fqn': ['test', 'seed'], + 'tags': [], + 'meta': {}, + 'config': self.rendered_seed_config(), + 'schema': my_schema_name, + 'database': self.default_database, + 'alias': 'seed', + 'columns': { + 'id': { + 'name': 'id', + 'description': 'The user ID number', + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'first_name': { + 'name': 'first_name', + 'description': "The user's first name", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'email': { + 'name': 'email', + 'description': "The user's email", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'ip_address': { + 'name': 'ip_address', + 'description': "The user's IP address", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + 'updated_at': { + 'name': 'updated_at', + 'description': "The last time this user's email was updated", + 'data_type': None, + 'meta': {}, + 'quote': None, + 'tags': [], + }, + }, + 'deferred': False, + 'description': 'The test seed', + 'docs': {'show': True}, + 'compiled': True, + 'compiled_sql': '', + 'extra_ctes_injected': True, + 'extra_ctes': [], + 'checksum': self._checksum_file(seed_path), + 'unrendered_config': self.unrendered_seed_config(), + }, + 'snapshot.test.snapshot_seed': { + 'alias': 'snapshot_seed', + 'compiled_path': None, + 'build_path': None, + 'created_at': ANY, + 'checksum': self._checksum_file(snapshot_path), + 'columns': {}, + 'compiled': True, + 'compiled_sql': ANY, + 'config': self.rendered_snapshot_config( + target_schema=self.alternate_schema + ), + 'database': self.default_database, + 'deferred': False, + 'depends_on': {'macros': [], + 'nodes': ['seed.test.seed']}, + 'description': '', + 'docs': {'show': True}, + 'extra_ctes': [], + 'extra_ctes_injected': True, + 'fqn': ['test', 'snapshot_seed', 'snapshot_seed'], + 'meta': {}, + 'name': 'snapshot_seed', + 'original_file_path': snapshot_path, + 'package_name': 'test', + 'patch_path': None, + 'path': 'snapshot_seed.sql', + 'raw_sql': ANY, + 'refs': [['seed']], + 'relation_name': '`{0}`.`{1}`.snapshot_seed'.format( + self.default_database, self.alternate_schema + ), + 'resource_type': 'snapshot', + 'root_path': self.test_root_realpath, + 'schema': self.alternate_schema, + 'sources': [], + 'tags': [], + 'unique_id': 'snapshot.test.snapshot_seed', + 'unrendered_config': self.unrendered_snapshot_config( + target_schema=self.alternate_schema + )} + }, + 'sources': {}, + 'exposures': {}, + 'selectors': {}, + 'child_map': { + 'model.test.clustered': [], + 'model.test.multi_clustered': [], + 'model.test.nested_table': ['model.test.nested_view'], + 'model.test.nested_view': [], + 'seed.test.seed': ['model.test.clustered', + 'model.test.multi_clustered', + 'snapshot.test.snapshot_seed'], + 'snapshot.test.snapshot_seed': [] + }, + 'parent_map': { + 'model.test.clustered': ['seed.test.seed'], + 'model.test.multi_clustered': ['seed.test.seed'], + 'seed.test.seed': [], + 'snapshot.test.snapshot_seed': ['seed.test.seed'], + 'model.test.nested_table': [], + 'model.test.nested_view': ['model.test.nested_table'], + }, + 'docs': { + 'dbt.__overview__': ANY, + 'test.macro_info': ANY, + 'test.macro_arg_info': ANY, + }, + 'disabled': [], + } + + def _checksum_file(self, path): + """windows has silly git behavior that adds newlines, and python does + silly things if we just open(..., 'r').encode('utf-8'). + """ + with open(self.dir(path), 'rb') as fp: + hashed = hashlib.sha256(fp.read()).hexdigest() + return { + 'name': 'sha256', + 'checksum': hashed, + } + + def _path_to(self, searched_path: str, relative_path: str): + return { + 'searched_path': normalize(searched_path), + 'relative_path': normalize(relative_path), + 'project_root': normalize(self.test_root_dir), + } + + def _absolute_path_to(self, searched_path: str, relative_path: str): + return os.path.join( + normalize(self.test_root_dir), + normalize(searched_path), + normalize(relative_path) + ) + + def _relation_name_format(self, quote_database: bool, quote_schema: bool, + quote_identifier: bool): + return ".".join(( + self._quote("{0}") if quote_database else '{0}', + self._quote("{1}") if quote_schema else '{1}', + self._quote("{2}") if quote_identifier else '{2}', + )) + + def verify_metadata(self, metadata, dbt_schema_version): + assert 'generated_at' in metadata + self.assertBetween(metadata['generated_at'], + start=self.generate_start_time) + assert 'dbt_version' in metadata + assert metadata['dbt_version'] == dbt.version.__version__ + assert 'dbt_schema_version' in metadata + assert metadata['dbt_schema_version'] == dbt_schema_version + assert metadata['invocation_id'] == dbt.tracking.active_user.invocation_id + key = 'env_key' + if os.name == 'nt': + key = key.upper() + assert metadata['env'] == { + key: 'env_value' + } + + def verify_manifest(self, expected_manifest): + self.assertTrue(os.path.exists('./target/manifest.json')) + + manifest = _read_json('./target/manifest.json') + + manifest_keys = frozenset({ + 'nodes', 'sources', 'macros', 'parent_map', 'child_map', + 'docs', 'metadata', 'docs', 'disabled', 'exposures', 'selectors', + }) + + self.assertEqual(frozenset(manifest), manifest_keys) + + for key in manifest_keys: + if key == 'macros': + self.verify_manifest_macros( + manifest, expected_manifest.get('macros')) + elif key == 'metadata': + metadata = manifest['metadata'] + self.verify_metadata( + metadata, 'https://schemas.getdbt.com/dbt/manifest/v3.json') + assert 'project_id' in metadata and metadata[ + 'project_id'] == '098f6bcd4621d373cade4e832627b4f6' + assert 'send_anonymous_usage_stats' in metadata and metadata[ + 'send_anonymous_usage_stats'] is False + assert 'adapter_type' in metadata and metadata['adapter_type'] == self.adapter_type + else: + self.assertIn(key, expected_manifest) # sanity check + self.assertEqual(manifest[key], expected_manifest[key]) + + def _quote(self, value): + quote_char = '`' + return '{0}{1}{0}'.format(quote_char, value) + + def expected_run_results(self): + """ + The expected results of this run. + """ + + return [ + { + 'status': 'success', + 'message': None, + 'execution_time': AnyFloat(), + 'unique_id': 'model.test.model', + 'adapter_response': ANY, + 'thread_id': ANY, + 'timing': [ANY, ANY], + 'failures': ANY, + }, + { + 'status': 'success', + 'message': None, + 'execution_time': AnyFloat(), + 'unique_id': 'model.test.second_model', + 'adapter_response': ANY, + 'thread_id': ANY, + 'timing': [ANY, ANY], + 'failures': ANY, + }, + { + 'status': 'success', + 'message': None, + 'execution_time': AnyFloat(), + 'unique_id': 'seed.test.seed', + 'adapter_response': ANY, + 'thread_id': ANY, + 'timing': [ANY, ANY], + 'failures': ANY, + }, + { + 'status': 'success', + 'message': None, + 'execution_time': AnyFloat(), + 'unique_id': 'snapshot.test.snapshot_seed', + 'adapter_response': ANY, + 'thread_id': ANY, + 'timing': [ANY, ANY], + 'failures': ANY, + }, + { + 'status': 'success', + 'message': None, + 'execution_time': AnyFloat(), + 'unique_id': 'test.test.not_null_model_id.d01cc630e6', + 'adapter_response': ANY, + 'thread_id': ANY, + 'timing': [ANY, ANY], + 'failures': ANY, + }, + { + 'status': 'success', + 'message': None, + 'execution_time': AnyFloat(), + 'unique_id': 'test.test.test_nothing_model_.5d38568946', + 'adapter_response': ANY, + 'thread_id': ANY, + 'timing': [ANY, ANY], + 'failures': ANY, + }, + { + 'status': 'success', + 'message': None, + 'execution_time': AnyFloat(), + 'unique_id': 'test.test.unique_model_id.67b76558ff', + 'adapter_response': ANY, + 'thread_id': ANY, + 'timing': [ANY, ANY], + 'failures': ANY, + }, + ] + + def verify_run_results(self, expected_run_results): + run_results = _read_json('./target/run_results.json') + assert 'metadata' in run_results + self.verify_metadata( + run_results['metadata'], 'https://schemas.getdbt.com/dbt/run-results/v3.json') + self.assertIn('elapsed_time', run_results) + self.assertGreater(run_results['elapsed_time'], 0) + self.assertTrue( + isinstance(run_results['elapsed_time'], float), + "run_results['elapsed_time'] is of type {}, expected float".format( + str(type(run_results['elapsed_time']))) + ) + + assert 'args' in run_results + # sort the results so we can make reasonable assertions + run_results['results'].sort(key=lambda r: r['unique_id']) + assert run_results['results'] == expected_run_results + set(run_results) == {'elapsed_time', 'results', 'metadata'} + + @use_profile('bigquery') + def test__bigquery__run_and_generate(self): + self.run_and_generate() + + self.verify_catalog(self.expected_bigquery_catalog()) + self.verify_manifest(self.expected_seeded_manifest()) + self.verify_run_results(self.expected_run_results()) + + @use_profile('bigquery') + def test__bigquery__complex_models(self): + self.run_and_generate( + extra={'source-paths': [self.dir('bq_models')]}, + model_count=4 + ) + + self.verify_catalog(self.expected_bigquery_complex_catalog()) + self.verify_manifest(self.expected_bigquery_complex_manifest()) + + +class TestDocsGenerateMissingSchema(DBTIntegrationTest): + @property + def schema(self): + return 'docs_generate' + + @staticmethod + def dir(path): + return normalize(path) + + @property + def models(self): + return self.dir("bq_models_noschema") + + def setUp(self): + super().setUp() + self.extra_schema = self.unique_schema() + '_bq_test' + + def tearDown(self): + with self.adapter.connection_named('__test'): + self._drop_schema_named(self.default_database, self.extra_schema) + super().tearDown() + + @use_profile('bigquery') + def test_bigquery_docs_generate_noschema(self): + self.run_dbt([ + 'docs', 'generate', + '--vars', "{{extra_schema: {}}}".format(self.extra_schema) + ]) diff --git a/tests/integration/docs_generate_tests/trivial_models/model.sql b/tests/integration/docs_generate_tests/trivial_models/model.sql new file mode 100644 index 000000000..43258a714 --- /dev/null +++ b/tests/integration/docs_generate_tests/trivial_models/model.sql @@ -0,0 +1 @@ +select 1 as id diff --git a/tests/integration/hook_tests/data/example_seed.csv b/tests/integration/hook_tests/data/example_seed.csv new file mode 100644 index 000000000..29e4b3171 --- /dev/null +++ b/tests/integration/hook_tests/data/example_seed.csv @@ -0,0 +1,4 @@ +a,b,c +1,2,3 +4,5,6 +7,8,9 diff --git a/tests/integration/hook_tests/macros/before-and-after-bq.sql b/tests/integration/hook_tests/macros/before-and-after-bq.sql new file mode 100644 index 000000000..030b869b7 --- /dev/null +++ b/tests/integration/hook_tests/macros/before-and-after-bq.sql @@ -0,0 +1,30 @@ + +{% macro custom_run_hook_bq(state, target, run_started_at, invocation_id) %} + + insert into {{ target.schema }}.on_run_hook ( + state, + target_dbname, + target_host, + target_name, + target_schema, + target_type, + target_user, + target_pass, + target_threads, + run_started_at, + invocation_id + ) VALUES ( + '{{ state }}', + '{{ target.database }}', + '', {# bigquery has no host value #} + '{{ target.name }}', + '{{ target.schema }}', + '{{ target.type }}', + '', {# bigquery has no user value #} + '{{ target.get("pass", "") }}', + {{ target.threads }}, + '{{ run_started_at }}', + '{{ invocation_id }}' + ) + +{% endmacro %} diff --git a/tests/integration/hook_tests/macros/hook.sql b/tests/integration/hook_tests/macros/hook.sql new file mode 100644 index 000000000..733278473 --- /dev/null +++ b/tests/integration/hook_tests/macros/hook.sql @@ -0,0 +1,3 @@ +{% macro hook() %} + select 1 +{% endmacro %} diff --git a/tests/integration/hook_tests/models/hooks.sql b/tests/integration/hook_tests/models/hooks.sql new file mode 100644 index 000000000..2cd691ea7 --- /dev/null +++ b/tests/integration/hook_tests/models/hooks.sql @@ -0,0 +1,2 @@ + +select 1 as id diff --git a/tests/integration/hook_tests/seed-models-bq/schema.yml b/tests/integration/hook_tests/seed-models-bq/schema.yml new file mode 100644 index 000000000..21bbf202f --- /dev/null +++ b/tests/integration/hook_tests/seed-models-bq/schema.yml @@ -0,0 +1,7 @@ +version: 2 +seeds: +- name: example_seed + columns: + - name: a + tests: + - not_null diff --git a/tests/integration/hook_tests/seed_model_bigquery.sql b/tests/integration/hook_tests/seed_model_bigquery.sql new file mode 100644 index 000000000..7093a47e8 --- /dev/null +++ b/tests/integration/hook_tests/seed_model_bigquery.sql @@ -0,0 +1,18 @@ + +drop table if exists {schema}.on_model_hook; + +create table {schema}.on_model_hook ( + state STRING, -- start|end + + target_dbname STRING, + target_host STRING, + target_name STRING, + target_schema STRING, + target_type STRING, + target_user STRING, + target_pass STRING, + target_threads INT64, + + run_started_at STRING, + invocation_id STRING +); diff --git a/tests/integration/hook_tests/seed_run_bigquery.sql b/tests/integration/hook_tests/seed_run_bigquery.sql new file mode 100644 index 000000000..d9d5212ef --- /dev/null +++ b/tests/integration/hook_tests/seed_run_bigquery.sql @@ -0,0 +1,18 @@ + +drop table if exists {schema}.on_run_hook; + +create table {schema}.on_run_hook ( + state STRING, -- start|end + + target_dbname STRING, + target_host STRING, + target_name STRING, + target_schema STRING, + target_type STRING, + target_user STRING, + target_pass STRING, + target_threads INT64, + + run_started_at STRING, + invocation_id STRING +); \ No newline at end of file diff --git a/tests/integration/hook_tests/test_model_hooks_bq.py b/tests/integration/hook_tests/test_model_hooks_bq.py new file mode 100644 index 000000000..1cd3326cd --- /dev/null +++ b/tests/integration/hook_tests/test_model_hooks_bq.py @@ -0,0 +1,150 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + +MODEL_PRE_HOOK = """ + insert into {{this.schema}}.on_model_hook ( + state, + target_name, + target_schema, + target_type, + target_threads, + run_started_at, + invocation_id + ) VALUES ( + 'start', + '{{ target.name }}', + '{{ target.schema }}', + '{{ target.type }}', + {{ target.threads }}, + '{{ run_started_at }}', + '{{ invocation_id }}' + ) +""" + + +MODEL_POST_HOOK = """ + insert into {{this.schema}}.on_model_hook ( + state, + target_name, + target_schema, + target_type, + target_threads, + run_started_at, + invocation_id + ) VALUES ( + 'end', + '{{ target.name }}', + '{{ target.schema }}', + '{{ target.type }}', + {{ target.threads }}, + '{{ run_started_at }}', + '{{ invocation_id }}' + ) +""" + +class TestBigqueryPrePostModelHooks(DBTIntegrationTest): + def setUp(self): + DBTIntegrationTest.setUp(self) + self.run_sql_file("seed_model_bigquery.sql") + + self.fields = [ + 'state', + 'target_name', + 'target_schema', + 'target_threads', + 'target_type', + 'run_started_at', + 'invocation_id' + ] + + @property + def schema(self): + return "model_hooks" + + @property + def profile_config(self): + profile = self.bigquery_profile() + profile['test']['outputs']['default2']['threads'] = 3 + return profile + + @property + def project_config(self): + return { + 'config-version': 2, + 'macro-paths': ['macros'], + 'models': { + 'test': { + 'pre-hook': [MODEL_PRE_HOOK], + 'post-hook':[MODEL_POST_HOOK], + } + } + } + + @property + def models(self): + return "models" + + def get_ctx_vars(self, state): + field_list = ", ".join(self.fields) + query = "select {field_list} from `{schema}.on_model_hook` where state = '{state}'".format(field_list=field_list, schema=self.unique_schema(), state=state) + + vals = self.run_sql(query, fetch='all') + self.assertFalse(len(vals) == 0, 'nothing inserted into hooks table') + self.assertFalse(len(vals) > 1, 'too many rows in hooks table') + ctx = dict(zip(self.fields, vals[0])) + + return ctx + + def check_hooks(self, state): + ctx = self.get_ctx_vars(state) + + self.assertEqual(ctx['state'], state) + self.assertEqual(ctx['target_name'], 'default2') + self.assertEqual(ctx['target_schema'], self.unique_schema()) + self.assertEqual(ctx['target_threads'], 3) + self.assertEqual(ctx['target_type'], 'bigquery') + self.assertTrue(ctx['run_started_at'] is not None and len(ctx['run_started_at']) > 0, 'run_started_at was not set') + self.assertTrue(ctx['invocation_id'] is not None and len(ctx['invocation_id']) > 0, 'invocation_id was not set') + + @use_profile('bigquery') + def test_pre_and_post_model_hooks_bigquery(self): + self.run_dbt(['run']) + + self.check_hooks('start') + self.check_hooks('end') + + +class TestBigqueryPrePostModelHooksOnSeeds(DBTIntegrationTest): + @property + def schema(self): + return "model_hooks_014" + + @property + def models(self): + return "seed-models-bq" + + @property + def project_config(self): + return { + 'config-version': 2, + 'data-paths': ['data'], + 'models': {}, + 'seeds': { + 'post-hook': [ + 'insert into {{ this }} (a, b, c) VALUES (10, 11, 12)', + ], + 'quote_columns': False, + }, + } + + @use_profile('bigquery') + def test_hooks_on_seeds_bigquery(self): + res = self.run_dbt(['seed']) + self.assertEqual(len(res), 1, 'Expected exactly one item') + res = self.run_dbt(['test']) + self.assertEqual(len(res), 1, 'Expected exactly one item') + result = self.run_sql( + 'select a, b, c from `{schema}`.`example_seed` where a = 10', + fetch='all' + ) + self.assertFalse(len(result) == 0, 'nothing inserted into table by hook') + self.assertFalse(len(result) > 1, 'too many rows in table') diff --git a/tests/integration/hook_tests/test_run_hooks_bq.py b/tests/integration/hook_tests/test_run_hooks_bq.py new file mode 100644 index 000000000..37b82735f --- /dev/null +++ b/tests/integration/hook_tests/test_run_hooks_bq.py @@ -0,0 +1,102 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + +class TestBigqueryPrePostRunHooks(DBTIntegrationTest): + + def setUp(self): + DBTIntegrationTest.setUp(self) + self.use_profile('bigquery') + self.use_default_project() + self.run_sql_file("seed_run_bigquery.sql") + + self.fields = [ + 'state', + 'target_name', + 'target_schema', + 'target_threads', + 'target_type', + 'run_started_at', + 'invocation_id' + ] + + @property + def schema(self): + return "run_hooks_014" + + @property + def profile_config(self): + profile = self.bigquery_profile() + profile['test']['outputs']['default2']['threads'] = 3 + return profile + + + @property + def project_config(self): + return { + 'config-version': 2, + 'macro-paths': ['macros'], + 'data-paths': ['data'], + + # The create and drop table statements here validate that these hooks run + # in the same order that they are defined. Drop before create is an error. + # Also check that the table does not exist below. + "on-run-start": [ + "{{ custom_run_hook_bq('start', target, run_started_at, invocation_id) }}", + "create table {{ target.schema }}.start_hook_order_test ( id INT64 )", + "drop table {{ target.schema }}.start_hook_order_test", + ], + "on-run-end": [ + "{{ custom_run_hook_bq('end', target, run_started_at, invocation_id) }}", + "create table {{ target.schema }}.end_hook_order_test ( id INT64 )", + "drop table {{ target.schema }}.end_hook_order_test", + ], + 'seeds': { + 'quote_columns': False, + }, + } + + @property + def models(self): + return "models" + + def get_ctx_vars(self, state): + field_list = ", ".join(self.fields) + query = "select {field_list} from `{schema}.on_run_hook` where state = '{state}'".format(field_list=field_list, schema=self.unique_schema(), state=state) + + vals = self.run_sql(query, fetch='all') + self.assertFalse(len(vals) == 0, 'nothing inserted into on_run_hook table') + self.assertFalse(len(vals) > 1, 'too many rows in hooks table') + ctx = dict(zip(self.fields, vals[0])) + + return ctx + + def check_hooks(self, state): + ctx = self.get_ctx_vars(state) + + self.assertEqual(ctx['state'], state) + self.assertEqual(ctx['target_name'], 'default2') + self.assertEqual(ctx['target_schema'], self.unique_schema()) + self.assertEqual(ctx['target_threads'], 3) + self.assertEqual(ctx['target_type'], 'bigquery') + + self.assertTrue(ctx['run_started_at'] is not None and len(ctx['run_started_at']) > 0, 'run_started_at was not set') + self.assertTrue(ctx['invocation_id'] is not None and len(ctx['invocation_id']) > 0, 'invocation_id was not set') + + @use_profile('bigquery') + def test_bigquery_pre_and_post_run_hooks(self): + self.run_dbt(['run']) + + self.check_hooks('start') + self.check_hooks('end') + + self.assertTableDoesNotExist("start_hook_order_test") + self.assertTableDoesNotExist("end_hook_order_test") + + @use_profile('bigquery') + def test_bigquery_pre_and_post_seed_hooks(self): + self.run_dbt(['seed']) + + self.check_hooks('start') + self.check_hooks('end') + + self.assertTableDoesNotExist("start_hook_order_test") + self.assertTableDoesNotExist("end_hook_order_test") diff --git a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns.sql b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns.sql new file mode 100644 index 000000000..6111d37d0 --- /dev/null +++ b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns.sql @@ -0,0 +1,29 @@ +{{ + config( + materialized='incremental', + unique_key='id', + on_schema_change='append_new_columns' + ) +}} + +{% set string_type = 'string' %} + +WITH source_data AS (SELECT * FROM {{ ref('model_a') }} ) + +{% if is_incremental() %} + +SELECT id, + cast(field1 as {{string_type}}) as field1, + cast(field2 as {{string_type}}) as field2, + cast(field3 as {{string_type}}) as field3, + cast(field4 as {{string_type}}) as field4 +FROM source_data WHERE id NOT IN (SELECT id from {{ this }} ) + +{% else %} + +SELECT id, + cast(field1 as {{string_type}}) as field1, + cast(field2 as {{string_type}}) as field2 +FROM source_data where id <= 3 + +{% endif %} \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_target.sql b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_target.sql new file mode 100644 index 000000000..8f65ed71a --- /dev/null +++ b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_target.sql @@ -0,0 +1,19 @@ +{{ + config(materialized='table') +}} + +{% set string_type = 'string' %} + +with source_data as ( + + select * from {{ ref('model_a') }} + +) + +select id + ,cast(field1 as {{string_type}}) as field1 + ,cast(field2 as {{string_type}}) as field2 + ,cast(CASE WHEN id <= 3 THEN NULL ELSE field3 END as {{string_type}}) AS field3 + ,cast(CASE WHEN id <= 3 THEN NULL ELSE field4 END as {{string_type}}) AS field4 + +from source_data \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/models/incremental_fail.sql b/tests/integration/incremental_schema_tests/models/incremental_fail.sql new file mode 100644 index 000000000..590f5b56d --- /dev/null +++ b/tests/integration/incremental_schema_tests/models/incremental_fail.sql @@ -0,0 +1,19 @@ +{{ + config( + materialized='incremental', + unique_key='id', + on_schema_change='fail' + ) +}} + +WITH source_data AS (SELECT * FROM {{ ref('model_a') }} ) + +{% if is_incremental() %} + +SELECT id, field1, field2 FROM source_data + +{% else %} + +SELECT id, field1, field3 FROm source_data + +{% endif %} \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/models/incremental_ignore.sql b/tests/integration/incremental_schema_tests/models/incremental_ignore.sql new file mode 100644 index 000000000..51dee6022 --- /dev/null +++ b/tests/integration/incremental_schema_tests/models/incremental_ignore.sql @@ -0,0 +1,19 @@ +{{ + config( + materialized='incremental', + unique_key='id', + on_schema_change='ignore' + ) +}} + +WITH source_data AS (SELECT * FROM {{ ref('model_a') }} ) + +{% if is_incremental() %} + +SELECT id, field1, field2, field3, field4 FROM source_data WHERE id NOT IN (SELECT id from {{ this }} ) + +{% else %} + +SELECT id, field1, field2 FROM source_data LIMIT 3 + +{% endif %} \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/models/incremental_ignore_target.sql b/tests/integration/incremental_schema_tests/models/incremental_ignore_target.sql new file mode 100644 index 000000000..92d4564e0 --- /dev/null +++ b/tests/integration/incremental_schema_tests/models/incremental_ignore_target.sql @@ -0,0 +1,15 @@ +{{ + config(materialized='table') +}} + +with source_data as ( + + select * from {{ ref('model_a') }} + +) + +select id + ,field1 + ,field2 + +from source_data \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns.sql b/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns.sql new file mode 100644 index 000000000..9c79d649c --- /dev/null +++ b/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns.sql @@ -0,0 +1,31 @@ +{{ + config( + materialized='incremental', + unique_key='id', + on_schema_change='sync_all_columns' + + ) +}} + +WITH source_data AS (SELECT * FROM {{ ref('model_a') }} ) + +{% set string_type = 'string' %} + +{% if is_incremental() %} + +SELECT id, + cast(field1 as {{string_type}}) as field1, + cast(field3 as {{string_type}}) as field3, -- to validate new fields + cast(field4 as {{string_type}}) AS field4 -- to validate new fields + +FROM source_data WHERE id NOT IN (SELECT id from {{ this }} ) + +{% else %} + +select id, + cast(field1 as {{string_type}}) as field1, + cast(field2 as {{string_type}}) as field2 + +from source_data where id <= 3 + +{% endif %} \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_target.sql b/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_target.sql new file mode 100644 index 000000000..2fcd88121 --- /dev/null +++ b/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_target.sql @@ -0,0 +1,20 @@ +{{ + config(materialized='table') +}} + +with source_data as ( + + select * from {{ ref('model_a') }} + +) + +{% set string_type = 'string' %} + +select id + ,cast(field1 as {{string_type}}) as field1 + --,field2 + ,cast(case when id <= 3 then null else field3 end as {{string_type}}) as field3 + ,cast(case when id <= 3 then null else field4 end as {{string_type}}) as field4 + +from source_data +order by id \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/models/model_a.sql b/tests/integration/incremental_schema_tests/models/model_a.sql new file mode 100644 index 000000000..2a0b2ddaf --- /dev/null +++ b/tests/integration/incremental_schema_tests/models/model_a.sql @@ -0,0 +1,22 @@ +{{ + config(materialized='table') +}} + +with source_data as ( + + select 1 as id, 'aaa' as field1, 'bbb' as field2, 111 as field3, 'TTT' as field4 + union all select 2 as id, 'ccc' as field1, 'ddd' as field2, 222 as field3, 'UUU' as field4 + union all select 3 as id, 'eee' as field1, 'fff' as field2, 333 as field3, 'VVV' as field4 + union all select 4 as id, 'ggg' as field1, 'hhh' as field2, 444 as field3, 'WWW' as field4 + union all select 5 as id, 'iii' as field1, 'jjj' as field2, 555 as field3, 'XXX' as field4 + union all select 6 as id, 'kkk' as field1, 'lll' as field2, 666 as field3, 'YYY' as field4 + +) + +select id + ,field1 + ,field2 + ,field3 + ,field4 + +from source_data \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/models/schema.yml b/tests/integration/incremental_schema_tests/models/schema.yml new file mode 100644 index 000000000..5546314e4 --- /dev/null +++ b/tests/integration/incremental_schema_tests/models/schema.yml @@ -0,0 +1,54 @@ +version: 2 + +models: + - name: model_a + columns: + - name: id + tags: [column_level_tag] + tests: + - unique + + - name: incremental_ignore + columns: + - name: id + tags: [column_level_tag] + tests: + - unique + + - name: incremental_ignore_target + columns: + - name: id + tags: [column_level_tag] + tests: + - unique + + - name: incremental_append_new_columns + columns: + - name: id + tags: [column_level_tag] + tests: + - unique + + - name: incremental_append_new_columns_target + columns: + - name: id + tags: [column_level_tag] + tests: + - unique + + - name: incremental_sync_all_columns + columns: + - name: id + tags: [column_level_tag] + tests: + - unique + + - name: incremental_sync_all_columns_target + columns: + - name: id + tags: [column_leveL_tag] + tests: + - unique + + + \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/test_incremental_schema.py b/tests/integration/incremental_schema_tests/test_incremental_schema.py new file mode 100644 index 000000000..a991fd568 --- /dev/null +++ b/tests/integration/incremental_schema_tests/test_incremental_schema.py @@ -0,0 +1,130 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + + +class TestSelectionExpansion(DBTIntegrationTest): + @property + def schema(self): + return "test_incremental_schema" + + @property + def models(self): + return "models" + + @property + def project_config(self): + return { + "config-version": 2, + "test-paths": ["tests"] + } + + def list_tests_and_assert(self, include, exclude, expected_tests): + list_args = ['ls', '--resource-type', 'test'] + if include: + list_args.extend(('--select', include)) + if exclude: + list_args.extend(('--exclude', exclude)) + listed = self.run_dbt(list_args) + print(listed) + assert len(listed) == len(expected_tests) + test_names = [name.split('.')[2] for name in listed] + assert sorted(test_names) == sorted(expected_tests) + + def run_tests_and_assert( + self, include, exclude, expected_tests, compare_source, compare_target, schema = False, data = False + ): + + run_args = ['run'] + if include: + run_args.extend(('--models', include)) + results_one = self.run_dbt(run_args) + results_two = self.run_dbt(run_args) + + self.assertEqual(len(results_one), 3) + self.assertEqual(len(results_two), 3) + + test_args = ['test'] + if include: + test_args.extend(('--models', include)) + if exclude: + test_args.extend(('--exclude', exclude)) + if schema: + test_args.append('--schema') + if data: + test_args.append('--data') + + results = self.run_dbt(test_args) + tests_run = [r.node.name for r in results] + assert len(tests_run) == len(expected_tests) + assert sorted(tests_run) == sorted(expected_tests) + self.assertTablesEqual(compare_source, compare_target) + + def run_incremental_ignore(self): + select = 'model_a incremental_ignore incremental_ignore_target' + compare_source = 'incremental_ignore' + compare_target = 'incremental_ignore_target' + exclude = None + expected = [ + 'select_from_a', + 'select_from_incremental_ignore', + 'select_from_incremental_ignore_target', + 'unique_model_a_id', + 'unique_incremental_ignore_id', + 'unique_incremental_ignore_target_id' + ] + + self.list_tests_and_assert(select, exclude, expected) + self.run_tests_and_assert(select, exclude, expected, compare_source, compare_target) + + def run_incremental_append_new_columns(self): + select = 'model_a incremental_append_new_columns incremental_append_new_columns_target' + compare_source = 'incremental_append_new_columns' + compare_target = 'incremental_append_new_columns_target' + exclude = None + expected = [ + 'select_from_a', + 'select_from_incremental_append_new_columns', + 'select_from_incremental_append_new_columns_target', + 'unique_model_a_id', + 'unique_incremental_append_new_columns_id', + 'unique_incremental_append_new_columns_target_id' + ] + self.list_tests_and_assert(select, exclude, expected) + self.run_tests_and_assert(select, exclude, expected, compare_source, compare_target) + + def run_incremental_sync_all_columns(self): + select = 'model_a incremental_sync_all_columns incremental_sync_all_columns_target' + compare_source = 'incremental_sync_all_columns' + compare_target = 'incremental_sync_all_columns_target' + exclude = None + expected = [ + 'select_from_a', + 'select_from_incremental_sync_all_columns', + 'select_from_incremental_sync_all_columns_target', + 'unique_model_a_id', + 'unique_incremental_sync_all_columns_id', + 'unique_incremental_sync_all_columns_target_id' + ] + self.list_tests_and_assert(select, exclude, expected) + self.run_tests_and_assert(select, exclude, expected, compare_source, compare_target) + + def run_incremental_fail_on_schema_change(self): + select = 'model_a incremental_fail' + results_one = self.run_dbt(['run', '--models', select, '--full-refresh']) + results_two = self.run_dbt(['run', '--models', select], expect_pass = False) + self.assertIn('Compilation Error', results_two[1].message) + + @use_profile('bigquery') + def test__bigquery__run_incremental_ignore(self): + self.run_incremental_ignore() + + @use_profile('bigquery') + def test__bigquery__run_incremental_append_new_columns(self): + self.run_incremental_append_new_columns() + + @use_profile('bigquery') + def test__bigquery__run_incremental_sync_all_columns(self): + self.run_incremental_sync_all_columns() + + @use_profile('bigquery') + def test__bigquery__run_incremental_fail_on_schema_change(self): + self.run_incremental_fail_on_schema_change() diff --git a/tests/integration/incremental_schema_tests/tests/select_from_a.sql b/tests/integration/incremental_schema_tests/tests/select_from_a.sql new file mode 100644 index 000000000..3dc8f2857 --- /dev/null +++ b/tests/integration/incremental_schema_tests/tests/select_from_a.sql @@ -0,0 +1 @@ +select * from {{ ref('model_a') }} where false diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns.sql new file mode 100644 index 000000000..947e84588 --- /dev/null +++ b/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns.sql @@ -0,0 +1 @@ +select * from {{ ref('incremental_append_new_columns') }} where false \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_target.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_target.sql new file mode 100644 index 000000000..8b86eddd7 --- /dev/null +++ b/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_target.sql @@ -0,0 +1 @@ +select * from {{ ref('incremental_append_new_columns_target') }} where false \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore.sql new file mode 100644 index 000000000..d565c8464 --- /dev/null +++ b/tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore.sql @@ -0,0 +1 @@ +select * from {{ ref('incremental_ignore') }} where false diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore_target.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore_target.sql new file mode 100644 index 000000000..35d535c5c --- /dev/null +++ b/tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore_target.sql @@ -0,0 +1 @@ +select * from {{ ref('incremental_ignore_target') }} where false \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns.sql new file mode 100644 index 000000000..aedc9f803 --- /dev/null +++ b/tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns.sql @@ -0,0 +1 @@ +select * from {{ ref('incremental_sync_all_columns') }} where false \ No newline at end of file diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns_target.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns_target.sql new file mode 100644 index 000000000..4b703c988 --- /dev/null +++ b/tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns_target.sql @@ -0,0 +1 @@ +select * from {{ ref('incremental_sync_all_columns_target') }} where false \ No newline at end of file diff --git a/tests/integration/override_database_test/data/seed.csv b/tests/integration/override_database_test/data/seed.csv new file mode 100644 index 000000000..ae9125c16 --- /dev/null +++ b/tests/integration/override_database_test/data/seed.csv @@ -0,0 +1,6 @@ +id,name +1,a +2,b +3,c +4,d +5,e diff --git a/tests/integration/override_database_test/models/subfolder/view_3.sql b/tests/integration/override_database_test/models/subfolder/view_3.sql new file mode 100644 index 000000000..4b91aa0f2 --- /dev/null +++ b/tests/integration/override_database_test/models/subfolder/view_3.sql @@ -0,0 +1 @@ +select * from {{ ref('seed') }} diff --git a/tests/integration/override_database_test/models/subfolder/view_4.sql b/tests/integration/override_database_test/models/subfolder/view_4.sql new file mode 100644 index 000000000..efa1268fa --- /dev/null +++ b/tests/integration/override_database_test/models/subfolder/view_4.sql @@ -0,0 +1,5 @@ +{{ + config(database=var('alternate_db')) +}} + +select * from {{ ref('seed') }} diff --git a/tests/integration/override_database_test/models/view_1.sql b/tests/integration/override_database_test/models/view_1.sql new file mode 100644 index 000000000..a43f04646 --- /dev/null +++ b/tests/integration/override_database_test/models/view_1.sql @@ -0,0 +1,7 @@ +{# + We are running against a database that must be quoted. + These calls ensure that we trigger an error if we're failing to quote at parse-time +#} +{% do adapter.already_exists(this.schema, this.table) %} +{% do adapter.get_relation(this.database, this.schema, this.table) %} +select * from {{ ref('seed') }} diff --git a/tests/integration/override_database_test/models/view_2.sql b/tests/integration/override_database_test/models/view_2.sql new file mode 100644 index 000000000..9ac6bdad6 --- /dev/null +++ b/tests/integration/override_database_test/models/view_2.sql @@ -0,0 +1,6 @@ +{%- if target.type == 'bigquery' -%} + {{ config(project=var('alternate_db')) }} +{%- else -%} + {{ config(database=var('alternate_db')) }} +{%- endif -%} +select * from {{ ref('seed') }} diff --git a/tests/integration/override_database_test/test_override_database.py b/tests/integration/override_database_test/test_override_database.py new file mode 100644 index 000000000..7e0a85fb2 --- /dev/null +++ b/tests/integration/override_database_test/test_override_database.py @@ -0,0 +1,176 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + +import os + + +class BaseOverrideDatabase(DBTIntegrationTest): + setup_alternate_db = True + @property + def schema(self): + return "override_database" + + @property + def models(self): + return "models" + + @property + def alternative_database(self): + return super().alternative_database + + @property + def project_config(self): + return { + 'config-version': 2, + 'data-paths': ['data'], + 'vars': { + 'alternate_db': self.alternative_database, + }, + 'quoting': { + 'database': True, + }, + 'seeds': { + 'quote_columns': False, + } + } + + +class TestModelOverride(BaseOverrideDatabase): + def run_database_override(self): + func = lambda x: x + + self.run_dbt(['seed']) + + self.assertEqual(len(self.run_dbt(['run'])), 4) + self.assertManyRelationsEqual([ + (func('seed'), self.unique_schema(), self.default_database), + (func('view_2'), self.unique_schema(), self.alternative_database), + (func('view_1'), self.unique_schema(), self.default_database), + (func('view_3'), self.unique_schema(), self.default_database), + (func('view_4'), self.unique_schema(), self.alternative_database), + ]) + + @use_profile('bigquery') + def test_bigquery_database_override(self): + self.run_database_override() + + +class BaseTestProjectModelOverride(BaseOverrideDatabase): + # this is janky, but I really want to access self.default_database in + # project_config + @property + def default_database(self): + target = self._profile_config['test']['target'] + profile = self._profile_config['test']['outputs'][target] + for key in ['database', 'project', 'dbname']: + if key in profile: + database = profile[key] + return database + assert False, 'No profile database found!' + + def run_database_override(self): + self.run_dbt(['seed']) + self.assertEqual(len(self.run_dbt(['run'])), 4) + self.assertExpectedRelations() + + def assertExpectedRelations(self): + func = lambda x: x + + self.assertManyRelationsEqual([ + (func('seed'), self.unique_schema(), self.default_database), + (func('view_2'), self.unique_schema(), self.alternative_database), + (func('view_1'), self.unique_schema(), self.alternative_database), + (func('view_3'), self.unique_schema(), self.default_database), + (func('view_4'), self.unique_schema(), self.alternative_database), + ]) + + +class TestProjectModelOverride(BaseTestProjectModelOverride): + @property + def project_config(self): + return { + 'config-version': 2, + 'vars': { + 'alternate_db': self.alternative_database, + }, + 'models': { + 'database': self.alternative_database, + 'test': { + 'subfolder': { + 'database': self.default_database, + } + } + }, + 'data-paths': ['data'], + 'vars': { + 'alternate_db': self.alternative_database, + }, + 'quoting': { + 'database': True, + }, + 'seeds': { + 'quote_columns': False, + } + } + + @use_profile('bigquery') + def test_bigquery_database_override(self): + self.run_database_override() + + +class TestProjectModelAliasOverride(BaseTestProjectModelOverride): + @property + def project_config(self): + return { + 'config-version': 2, + 'vars': { + 'alternate_db': self.alternative_database, + }, + 'models': { + 'project': self.alternative_database, + 'test': { + 'subfolder': { + 'project': self.default_database, + } + } + }, + 'data-paths': ['data'], + 'vars': { + 'alternate_db': self.alternative_database, + }, + 'quoting': { + 'database': True, + }, + 'seeds': { + 'quote_columns': False, + } + } + + @use_profile('bigquery') + def test_bigquery_project_override(self): + self.run_database_override() + + +class TestProjectSeedOverride(BaseOverrideDatabase): + def run_database_override(self): + func = lambda x: x + + self.use_default_project({ + 'config-version': 2, + 'seeds': { + 'database': self.alternative_database + }, + }) + self.run_dbt(['seed']) + + self.assertEqual(len(self.run_dbt(['run'])), 4) + self.assertManyRelationsEqual([ + (func('seed'), self.unique_schema(), self.alternative_database), + (func('view_2'), self.unique_schema(), self.alternative_database), + (func('view_1'), self.unique_schema(), self.default_database), + (func('view_3'), self.unique_schema(), self.default_database), + (func('view_4'), self.unique_schema(), self.alternative_database), + ]) + + @use_profile('bigquery') + def test_bigquery_database_override(self): + self.run_database_override() diff --git a/tests/integration/persist_docs_tests/data/seed.csv b/tests/integration/persist_docs_tests/data/seed.csv new file mode 100644 index 000000000..1a728c8ab --- /dev/null +++ b/tests/integration/persist_docs_tests/data/seed.csv @@ -0,0 +1,3 @@ +id,name +1,Alice +2,Bob diff --git a/tests/integration/persist_docs_tests/models-bigquery-nested/schema.yml b/tests/integration/persist_docs_tests/models-bigquery-nested/schema.yml new file mode 100644 index 000000000..0311dcb14 --- /dev/null +++ b/tests/integration/persist_docs_tests/models-bigquery-nested/schema.yml @@ -0,0 +1,19 @@ +version: 2 + +models: + - name: table_model_nested + columns: + - name: level_1 + description: level_1 column description + - name: level_1.level_2 + description: level_2 column description + - name: level_1.level_2.level_3_a + description: level_3 column description + - name: view_model_nested + columns: + - name: level_1 + description: level_1 column description + - name: level_1.level_2 + description: level_2 column description + - name: level_1.level_2.level_3_a + description: level_3 column description \ No newline at end of file diff --git a/tests/integration/persist_docs_tests/models-bigquery-nested/table_model_nested.sql b/tests/integration/persist_docs_tests/models-bigquery-nested/table_model_nested.sql new file mode 100644 index 000000000..c2936d4f1 --- /dev/null +++ b/tests/integration/persist_docs_tests/models-bigquery-nested/table_model_nested.sql @@ -0,0 +1,8 @@ +{{ config(materialized='table') }} +SELECT + STRUCT( + STRUCT( + 1 AS level_3_a, + 2 AS level_3_b + ) AS level_2 + ) AS level_1 \ No newline at end of file diff --git a/tests/integration/persist_docs_tests/models-bigquery-nested/view_model_nested.sql b/tests/integration/persist_docs_tests/models-bigquery-nested/view_model_nested.sql new file mode 100644 index 000000000..e3323ddf4 --- /dev/null +++ b/tests/integration/persist_docs_tests/models-bigquery-nested/view_model_nested.sql @@ -0,0 +1,8 @@ +{{ config(materialized='view') }} +SELECT + STRUCT( + STRUCT( + 1 AS level_3_a, + 2 AS level_3_b + ) AS level_2 + ) AS level_1 \ No newline at end of file diff --git a/tests/integration/persist_docs_tests/models-column-missing/missing_column.sql b/tests/integration/persist_docs_tests/models-column-missing/missing_column.sql new file mode 100644 index 000000000..642b0f14a --- /dev/null +++ b/tests/integration/persist_docs_tests/models-column-missing/missing_column.sql @@ -0,0 +1,2 @@ +{{ config(materialized='table') }} +select 1 as id, 'Ed' as name diff --git a/tests/integration/persist_docs_tests/models-column-missing/schema.yml b/tests/integration/persist_docs_tests/models-column-missing/schema.yml new file mode 100644 index 000000000..aa7b4f888 --- /dev/null +++ b/tests/integration/persist_docs_tests/models-column-missing/schema.yml @@ -0,0 +1,8 @@ +version: 2 +models: + - name: missing_column + columns: + - name: id + description: "test id column description" + - name: column_that_does_not_exist + description: "comment that cannot be created" diff --git a/tests/integration/persist_docs_tests/models/my_fun_docs.md b/tests/integration/persist_docs_tests/models/my_fun_docs.md new file mode 100644 index 000000000..f3c0fbf55 --- /dev/null +++ b/tests/integration/persist_docs_tests/models/my_fun_docs.md @@ -0,0 +1,10 @@ +{% docs my_fun_doc %} +name Column description "with double quotes" +and with 'single quotes' as welll as other; +'''abc123''' +reserved -- characters +-- +/* comment */ +Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting + +{% enddocs %} diff --git a/tests/integration/persist_docs_tests/models/no_docs_model.sql b/tests/integration/persist_docs_tests/models/no_docs_model.sql new file mode 100644 index 000000000..e39a7a156 --- /dev/null +++ b/tests/integration/persist_docs_tests/models/no_docs_model.sql @@ -0,0 +1 @@ +select 1 as id, 'Alice' as name diff --git a/tests/integration/persist_docs_tests/models/schema.yml b/tests/integration/persist_docs_tests/models/schema.yml new file mode 100644 index 000000000..5a9091624 --- /dev/null +++ b/tests/integration/persist_docs_tests/models/schema.yml @@ -0,0 +1,70 @@ +version: 2 + +models: + - name: table_model + description: | + Table model description "with double quotes" + and with 'single quotes' as welll as other; + '''abc123''' + reserved -- characters + -- + /* comment */ + Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting + columns: + - name: id + description: | + id Column description "with double quotes" + and with 'single quotes' as welll as other; + '''abc123''' + reserved -- characters + -- + /* comment */ + Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting + - name: name + description: | + Some stuff here and then a call to + {{ doc('my_fun_doc')}} + - name: view_model + description: | + View model description "with double quotes" + and with 'single quotes' as welll as other; + '''abc123''' + reserved -- characters + -- + /* comment */ + Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting + columns: + - name: id + description: | + id Column description "with double quotes" + and with 'single quotes' as welll as other; + '''abc123''' + reserved -- characters + -- + /* comment */ + Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting + +seeds: + - name: seed + description: | + Seed model description "with double quotes" + and with 'single quotes' as welll as other; + '''abc123''' + reserved -- characters + -- + /* comment */ + Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting + columns: + - name: id + description: | + id Column description "with double quotes" + and with 'single quotes' as welll as other; + '''abc123''' + reserved -- characters + -- + /* comment */ + Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting + - name: name + description: | + Some stuff here and then a call to + {{ doc('my_fun_doc')}} diff --git a/tests/integration/persist_docs_tests/models/table_model.sql b/tests/integration/persist_docs_tests/models/table_model.sql new file mode 100644 index 000000000..c0e93c3f3 --- /dev/null +++ b/tests/integration/persist_docs_tests/models/table_model.sql @@ -0,0 +1,2 @@ +{{ config(materialized='table') }} +select 1 as id, 'Joe' as name diff --git a/tests/integration/persist_docs_tests/models/view_model.sql b/tests/integration/persist_docs_tests/models/view_model.sql new file mode 100644 index 000000000..a6f96a16d --- /dev/null +++ b/tests/integration/persist_docs_tests/models/view_model.sql @@ -0,0 +1,2 @@ +{{ config(materialized='view') }} +select 2 as id, 'Bob' as name diff --git a/tests/integration/persist_docs_tests/test_persist_docs.py b/tests/integration/persist_docs_tests/test_persist_docs.py new file mode 100644 index 000000000..5079c035e --- /dev/null +++ b/tests/integration/persist_docs_tests/test_persist_docs.py @@ -0,0 +1,209 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import os + +import json + + +class BasePersistDocsTest(DBTIntegrationTest): + @property + def schema(self): + return "persist_docs" + + @property + def models(self): + return "models" + + def _assert_common_comments(self, *comments): + for comment in comments: + assert '"with double quotes"' in comment + assert """'''abc123'''""" in comment + assert '\n' in comment + assert 'Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting' in comment + assert '/* comment */' in comment + if os.name == 'nt': + assert '--\r\n' in comment or '--\n' in comment + else: + assert '--\n' in comment + + def _assert_has_table_comments(self, table_node): + table_comment = table_node['metadata']['comment'] + assert table_comment.startswith('Table model description') + + table_id_comment = table_node['columns']['id']['comment'] + assert table_id_comment.startswith('id Column description') + + table_name_comment = table_node['columns']['name']['comment'] + assert table_name_comment.startswith( + 'Some stuff here and then a call to') + + self._assert_common_comments( + table_comment, table_id_comment, table_name_comment + ) + + def _assert_has_view_comments(self, view_node, has_node_comments=True, + has_column_comments=True): + view_comment = view_node['metadata']['comment'] + if has_node_comments: + assert view_comment.startswith('View model description') + self._assert_common_comments(view_comment) + else: + assert view_comment is None + + view_id_comment = view_node['columns']['id']['comment'] + if has_column_comments: + assert view_id_comment.startswith('id Column description') + self._assert_common_comments(view_id_comment) + else: + assert view_id_comment is None + + view_name_comment = view_node['columns']['name']['comment'] + assert view_name_comment is None + + +class TestPersistDocsSimple(BasePersistDocsTest): + @property + def project_config(self): + return { + 'config-version': 2, + 'models': { + 'test': { + '+persist_docs': { + "relation": True, + "columns": True, + }, + } + }, + 'seeds': { + 'test': { + '+persist_docs': { + "relation": True, + "columns": True, + }, + } + }, + } + + @use_profile('bigquery') + def test_bigquery_persist_docs(self): + self.run_dbt(['seed']) + self.run_dbt() + desc_map = { + 'seed': 'Seed model description', + 'table_model': 'Table model description', + 'view_model': 'View model description', + } + for node_id in ['seed', 'table_model', 'view_model']: + with self.adapter.connection_named('_test'): + client = self.adapter.connections \ + .get_thread_connection().handle + + table_id = "{}.{}.{}".format( + self.default_database, + self.unique_schema(), + node_id + ) + bq_table = client.get_table(table_id) + + bq_schema = bq_table.schema + + assert bq_table.description.startswith(desc_map[node_id]) + assert bq_schema[0].description.startswith('id Column description ') + if not node_id.startswith('view'): + assert bq_schema[1].description.startswith('Some stuff here and then a call to') + + +class TestPersistDocsNested(BasePersistDocsTest): + @property + def project_config(self): + return { + 'config-version': 2, + 'models': { + 'test': { + '+persist_docs': { + "relation": True, + "columns": True, + }, + } + } + } + + @property + def models(self): + return 'models-bigquery-nested' + + @use_profile('bigquery') + def test_bigquery_persist_docs(self): + """ + run dbt and use the bigquery client from the adapter to check if the + colunmn descriptions are persisted on the test model table and view. + + Next, generate the catalog and check if the comments are also included. + """ + self.run_dbt(['seed']) + self.run_dbt() + + self.run_dbt(['docs', 'generate']) + with open('target/catalog.json') as fp: + catalog_data = json.load(fp) + assert 'nodes' in catalog_data + assert len(catalog_data['nodes']) == 3 # seed, table, and view model + + for node_id in ['table_model_nested', 'view_model_nested']: + # check the descriptions using the api + with self.adapter.connection_named('_test'): + client = self.adapter.connections \ + .get_thread_connection().handle + + table_id = "{}.{}.{}".format( + self.default_database, + self.unique_schema(), + node_id + ) + bq_schema = client.get_table(table_id).schema + + level_1_field = bq_schema[0] + assert level_1_field.description == \ + "level_1 column description" + + level_2_field = level_1_field.fields[0] + assert level_2_field.description == \ + "level_2 column description" + + level_3_field = level_2_field.fields[0] + assert level_3_field.description == \ + "level_3 column description" + + # check the descriptions in the catalog + node = catalog_data['nodes']['model.test.{}'.format(node_id)] + + level_1_column = node['columns']['level_1'] + assert level_1_column['comment'] == "level_1 column description" + + level_2_column = node['columns']['level_1.level_2'] + assert level_2_column['comment'] == "level_2 column description" + + level_3_column = node['columns']['level_1.level_2.level_3_a'] + assert level_3_column['comment'] == "level_3 column description" + + +class TestPersistDocsColumnMissing(BasePersistDocsTest): + @property + def project_config(self): + return { + 'config-version': 2, + 'models': { + 'test': { + '+persist_docs': { + "columns": True, + }, + } + } + } + + @property + def models(self): + return 'models-column-missing' + + @use_profile('bigquery') + def test_bigquery_missing_column(self): + self.run_dbt() diff --git a/tests/integration/query_comments_test/macros/macro.sql b/tests/integration/query_comments_test/macros/macro.sql new file mode 100644 index 000000000..6e8a1900c --- /dev/null +++ b/tests/integration/query_comments_test/macros/macro.sql @@ -0,0 +1,25 @@ +{%- macro query_header_no_args() -%} +{%- set x = "are pretty cool" -%} +{{ "dbt macros" }} +{{ x }} +{%- endmacro -%} + + +{%- macro query_header_args(message) -%} + {%- set comment_dict = dict( + app='dbt++', + macro_version='0.1.0', + dbt_version=dbt_version, + message='blah: '~ message) -%} + {{ return(comment_dict) }} +{%- endmacro -%} + + +{%- macro ordered_to_json(dct) -%} +{{ tojson(dct, sort_keys=True) }} +{%- endmacro %} + + +{% macro invalid_query_header() -%} +{{ "Here is an invalid character for you: */" }} +{% endmacro %} diff --git a/tests/integration/query_comments_test/models/x.sql b/tests/integration/query_comments_test/models/x.sql new file mode 100644 index 000000000..03d6c0abb --- /dev/null +++ b/tests/integration/query_comments_test/models/x.sql @@ -0,0 +1,25 @@ +{% set blacklist = ['pass', 'password', 'keyfile', 'keyfile.json', 'password', 'private_key_passphrase'] %} +{% for key in blacklist %} + {% if key in blacklist and blacklist[key] %} + {% do exceptions.raise_compiler_error('invalid target, found banned key "' ~ key ~ '"') %} + {% endif %} +{% endfor %} + +{% if 'type' not in target %} + {% do exceptions.raise_compiler_error('invalid target, missing "type"') %} +{% endif %} + +{% set required = ['name', 'schema', 'type', 'threads'] %} + +{# Require what we document at https://docs.getdbt.com/docs/target #} + {% do required.extend(['project']) %} +{% endif %} + +{% for value in required %} + {% if value not in target %} + {% do exceptions.raise_compiler_error('invalid target, missing "' ~ value ~ '"') %} + {% endif %} +{% endfor %} + +{% do run_query('select 2 as inner_id') %} +select 1 as outer_id diff --git a/tests/integration/query_comments_test/test_query_comments.py b/tests/integration/query_comments_test/test_query_comments.py new file mode 100644 index 000000000..11a816a21 --- /dev/null +++ b/tests/integration/query_comments_test/test_query_comments.py @@ -0,0 +1,167 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +import io +import json +import os + +import dbt.exceptions +from dbt.version import __version__ as dbt_version +from dbt.logger import log_manager + + +class TestDefaultQueryComments(DBTIntegrationTest): + def matches_comment(self, msg) -> bool: + if not msg.startswith('/* '): + return False + # our blob is the first line of the query comments, minus the comment + json_str = msg.split('\n')[0][3:-3] + data = json.loads(json_str) + return ( + data['app'] == 'dbt' and + data['dbt_version'] == dbt_version and + data['node_id'] == 'model.test.x' + ) + + @property + def project_config(self): + return { + 'config-version': 2, + 'macro-paths': ['macros'] + } + + @property + def schema(self): + return 'dbt_query_comments' + + @staticmethod + def dir(value): + return os.path.normpath(value) + + @property + def models(self): + return self.dir('models') + + def setUp(self): + super().setUp() + self.initial_stdout = log_manager.stdout + self.initial_stderr = log_manager.stderr + self.stringbuf = io.StringIO() + log_manager.set_output_stream(self.stringbuf) + + def tearDown(self): + log_manager.set_output_stream(self.initial_stdout, self.initial_stderr) + super().tearDown() + + def run_get_json(self, expect_pass=True): + self.run_dbt( + ['--debug', '--log-format=json', 'run'], + expect_pass=expect_pass + ) + logs = [] + for line in self.stringbuf.getvalue().split('\n'): + try: + log = json.loads(line) + except ValueError: + continue + + if log['extra'].get('run_state') != 'running': + continue + logs.append(log) + self.assertGreater(len(logs), 0) + return logs + + def query_comment(self, model_name, log): + prefix = 'On {}: '.format(model_name) + + if log['message'].startswith(prefix): + msg = log['message'][len(prefix):] + if msg in {'COMMIT', 'BEGIN', 'ROLLBACK'}: + return None + return msg + return None + + def run_assert_comments(self): + logs = self.run_get_json() + + seen = False + for log in logs: + msg = self.query_comment('model.test.x', log) + if msg is not None and self.matches_comment(msg): + seen = True + + self.assertTrue(seen, 'Never saw a matching log message! Logs:\n{}'.format('\n'.join(l['message'] for l in logs))) + + @use_profile('bigquery') + def test_bigquery_comments(self): + self.run_assert_comments() + + +class TestQueryComments(TestDefaultQueryComments): + @property + def project_config(self): + cfg = super().project_config + cfg.update({'query-comment': 'dbt\nrules!\n'}) + return cfg + + def matches_comment(self, msg) -> bool: + return msg.startswith('/* dbt\nrules! */\n') + + +class TestMacroQueryComments(TestDefaultQueryComments): + @property + def project_config(self): + cfg = super().project_config + cfg.update({'query-comment': '{{ query_header_no_args() }}'}) + return cfg + + def matches_comment(self, msg) -> bool: + start_with = '/* dbt macros\nare pretty cool */\n' + return msg.startswith(start_with) + + +class TestMacroArgsQueryComments(TestDefaultQueryComments): + @property + def project_config(self): + cfg = super().project_config + cfg.update( + {'query-comment': '{{ return(ordered_to_json(query_header_args(target.name))) }}'} + ) + return cfg + + def matches_comment(self, msg) -> bool: + expected_dct = {'app': 'dbt++', 'dbt_version': dbt_version, 'macro_version': '0.1.0', 'message': 'blah: default2'} + expected = '/* {} */\n'.format(json.dumps(expected_dct, sort_keys=True)) + return msg.startswith(expected) + + +class TestMacroInvalidQueryComments(TestDefaultQueryComments): + @property + def project_config(self): + cfg = super().project_config + cfg.update({'query-comment': '{{ invalid_query_header() }}'}) + return cfg + + def run_assert_comments(self): + with self.assertRaises(dbt.exceptions.RuntimeException): + self.run_get_json(expect_pass=False) + + +class TestNullQueryComments(TestDefaultQueryComments): + @property + def project_config(self): + cfg = super().project_config + cfg.update({'query-comment': ''}) + return cfg + + def matches_comment(self, msg) -> bool: + return not ('/*' in msg or '*/' in msg) + + +class TestEmptyQueryComments(TestDefaultQueryComments): + @property + def project_config(self): + cfg = super().project_config + cfg.update({'query-comment': ''}) + return cfg + + def matches_comment(self, msg) -> bool: + return not ('/*' in msg or '*/' in msg) diff --git a/tests/integration/schema_tests_test/models-v2/bq-models/ephemeral_copy.sql b/tests/integration/schema_tests_test/models-v2/bq-models/ephemeral_copy.sql new file mode 100644 index 000000000..73c62165e --- /dev/null +++ b/tests/integration/schema_tests_test/models-v2/bq-models/ephemeral_copy.sql @@ -0,0 +1,8 @@ + +{{ + config( + materialized='ephemeral' + ) +}} + +select * from {{ this.schema }}.seed diff --git a/tests/integration/schema_tests_test/models-v2/bq-models/schema.yml b/tests/integration/schema_tests_test/models-v2/bq-models/schema.yml new file mode 100644 index 000000000..15100c18e --- /dev/null +++ b/tests/integration/schema_tests_test/models-v2/bq-models/schema.yml @@ -0,0 +1,30 @@ +version: 2 + +models: + - name: ephemeral_copy + description: "An ephemeral copy of the table" + columns: + - name: id + description: "The ID" + tests: + - not_null + - unique + - name: favorite_color + tests: + - accepted_values: { values: ['blue', 'green'] } + + # this whole model should pass and run + - name: table_summary + description: "The summary table" + columns: + - name: favorite_color_copy + description: "The favorite color" + tests: + - not_null + - unique + - accepted_values: { values: ['blue', 'green'] } + - relationships: { field: favorite_color, to: ref('ephemeral_copy') } + - name: count + description: "The number of responses for this favorite color" + tests: + - not_null diff --git a/tests/integration/schema_tests_test/models-v2/bq-models/table_summary.sql b/tests/integration/schema_tests_test/models-v2/bq-models/table_summary.sql new file mode 100644 index 000000000..845ce3d58 --- /dev/null +++ b/tests/integration/schema_tests_test/models-v2/bq-models/table_summary.sql @@ -0,0 +1,9 @@ +{{ + config( + materialized='table' + ) +}} + +select favorite_color as favorite_color_copy, count(*) as count +from {{ ref('ephemeral_copy') }} +group by 1 diff --git a/tests/integration/schema_tests_test/models-v2/seed/seed.csv b/tests/integration/schema_tests_test/models-v2/seed/seed.csv new file mode 100644 index 000000000..dc9bdeaf6 --- /dev/null +++ b/tests/integration/schema_tests_test/models-v2/seed/seed.csv @@ -0,0 +1,5 @@ +favorite_color,id,first_name,email,ip_address,updated_at +blue,1,Larry,null,69.135.206.194,2008-09-12 19:08:31 +blue,2,Larry,null,64.210.133.162,1978-05-09 04:15:14 +green,99,Paul,pjohnson2q@umn.edu,183.59.198.197,1991-11-14 12:33:55 +green,100,Frank,fgreene2r@blogspot.com,150.143.68.121,2010-06-12 23:55:39 diff --git a/tests/integration/schema_tests_test/test_schema_v2_tests.py b/tests/integration/schema_tests_test/test_schema_v2_tests.py new file mode 100644 index 000000000..a66f79664 --- /dev/null +++ b/tests/integration/schema_tests_test/test_schema_v2_tests.py @@ -0,0 +1,55 @@ +from tests.integration.base import DBTIntegrationTest, FakeArgs, use_profile +import os + +from dbt.task.test import TestTask +from dbt.exceptions import CompilationException +from dbt.contracts.results import TestStatus + +class TestBQSchemaTests(DBTIntegrationTest): + @property + def schema(self): + return "schema_tests" + + @property + def models(self): + return "models-v2/bq-models" + + @staticmethod + def dir(path): + return os.path.normpath( + os.path.join('models-v2', path)) + + def run_schema_validations(self): + args = FakeArgs() + + test_task = TestTask(args, self.config) + return test_task.run() + + @use_profile('bigquery') + def test_schema_tests_bigquery(self): + self.use_default_project({'data-paths': [self.dir('seed')]}) + self.assertEqual(len(self.run_dbt(['seed'])), 1) + results = self.run_dbt() + self.assertEqual(len(results), 1) + test_results = self.run_schema_validations() + self.assertEqual(len(test_results), 8) + + for result in test_results: + # assert that all deliberately failing tests actually fail + if 'failure' in result.node.name: + self.assertEqual(result.status, 'fail') + self.assertFalse(result.skipped) + self.assertTrue( + result.failures > 0, + 'test {} did not fail'.format(result.node.name) + ) + # assert that actual tests pass + else: + self.assertEqual(result.status, 'pass') + self.assertFalse(result.skipped) + self.assertEqual( + result.failures, 0, + 'test {} failed'.format(result.node.name) + ) + + self.assertEqual(sum(x.failures for x in test_results), 0) diff --git a/tests/integration/simple_copy_test/models-merge-update/incremental_update_cols.sql b/tests/integration/simple_copy_test/models-merge-update/incremental_update_cols.sql new file mode 100644 index 000000000..6f79e08e9 --- /dev/null +++ b/tests/integration/simple_copy_test/models-merge-update/incremental_update_cols.sql @@ -0,0 +1,17 @@ +{{ + config( + materialized = "incremental", + unique_key = "id", + merge_update_columns = ["email", "ip_address"] + ) +}} + + +select * +from {{ ref('seed') }} + +{% if is_incremental() %} + + where load_date > (select max(load_date) from {{this}}) + +{% endif %} diff --git a/tests/integration/simple_copy_test/models/advanced_incremental.sql b/tests/integration/simple_copy_test/models/advanced_incremental.sql new file mode 100644 index 000000000..7f4c12af8 --- /dev/null +++ b/tests/integration/simple_copy_test/models/advanced_incremental.sql @@ -0,0 +1,17 @@ +{{ + config( + materialized = "incremental", + unique_key = "id", + persist_docs = {"relation": true} + ) +}} + + +select * +from {{ ref('seed') }} + +{% if is_incremental() %} + + where id > (select max(id) from {{this}}) + +{% endif %} diff --git a/tests/integration/simple_copy_test/models/compound_sort.sql b/tests/integration/simple_copy_test/models/compound_sort.sql new file mode 100644 index 000000000..64b41ca7e --- /dev/null +++ b/tests/integration/simple_copy_test/models/compound_sort.sql @@ -0,0 +1,9 @@ +{{ + config( + materialized = "table", + sort = 'first_name', + sort_type = 'compound' + ) +}} + +select * from {{ ref('seed') }} diff --git a/tests/integration/simple_copy_test/models/disabled.sql b/tests/integration/simple_copy_test/models/disabled.sql new file mode 100644 index 000000000..1d10a0c8d --- /dev/null +++ b/tests/integration/simple_copy_test/models/disabled.sql @@ -0,0 +1,8 @@ +{{ + config( + materialized = "view", + enabled = False + ) +}} + +select * from {{ ref('seed') }} diff --git a/tests/integration/simple_copy_test/models/empty.sql b/tests/integration/simple_copy_test/models/empty.sql new file mode 100644 index 000000000..3f2ff2d6c --- /dev/null +++ b/tests/integration/simple_copy_test/models/empty.sql @@ -0,0 +1,5 @@ + + + + + diff --git a/tests/integration/simple_copy_test/models/get_and_ref.sql b/tests/integration/simple_copy_test/models/get_and_ref.sql new file mode 100644 index 000000000..89d55c738 --- /dev/null +++ b/tests/integration/simple_copy_test/models/get_and_ref.sql @@ -0,0 +1,3 @@ +{%- do adapter.get_relation(database=target.database, schema=target.schema, identifier='materialized') -%} + +select * from {{ ref('materialized') }} diff --git a/tests/integration/simple_copy_test/models/incremental.sql b/tests/integration/simple_copy_test/models/incremental.sql new file mode 100644 index 000000000..020bf3516 --- /dev/null +++ b/tests/integration/simple_copy_test/models/incremental.sql @@ -0,0 +1,11 @@ +{{ + config( + materialized = "incremental" + ) +}} + +select * from {{ ref('seed') }} + +{% if is_incremental() %} + where id > (select max(id) from {{this}}) +{% endif %} diff --git a/tests/integration/simple_copy_test/models/interleaved_sort.sql b/tests/integration/simple_copy_test/models/interleaved_sort.sql new file mode 100644 index 000000000..147370396 --- /dev/null +++ b/tests/integration/simple_copy_test/models/interleaved_sort.sql @@ -0,0 +1,9 @@ +{{ + config( + materialized = "table", + sort = ['first_name', 'last_name'], + sort_type = 'interleaved' + ) +}} + +select * from {{ ref('seed') }} diff --git a/tests/integration/simple_copy_test/models/materialized.sql b/tests/integration/simple_copy_test/models/materialized.sql new file mode 100644 index 000000000..5b004a514 --- /dev/null +++ b/tests/integration/simple_copy_test/models/materialized.sql @@ -0,0 +1,12 @@ +{{ + config( + materialized = "table" + ) +}} +-- ensure that dbt_utils' relation check will work +{% set relation = ref('seed') %} +{%- if not (relation is mapping and relation.get('metadata', {}).get('type', '').endswith('Relation')) -%} + {%- do exceptions.raise_compiler_error("Macro " ~ macro ~ " expected a Relation but received the value: " ~ relation) -%} +{%- endif -%} +-- this is a unicode character: å +select * from {{ relation }} diff --git a/tests/integration/simple_copy_test/models/schema.yml b/tests/integration/simple_copy_test/models/schema.yml new file mode 100644 index 000000000..9a813eed5 --- /dev/null +++ b/tests/integration/simple_copy_test/models/schema.yml @@ -0,0 +1,7 @@ +version: 2 +models: +- name: disabled + columns: + - name: id + tests: + - unique diff --git a/tests/integration/simple_copy_test/models/view_model.sql b/tests/integration/simple_copy_test/models/view_model.sql new file mode 100644 index 000000000..9838c5b83 --- /dev/null +++ b/tests/integration/simple_copy_test/models/view_model.sql @@ -0,0 +1,7 @@ +{{ + config( + materialized = "view" + ) +}} + +select * from {{ ref('seed') }} diff --git a/tests/integration/simple_copy_test/seed-initial/seed.csv b/tests/integration/simple_copy_test/seed-initial/seed.csv new file mode 100644 index 000000000..640af6c4e --- /dev/null +++ b/tests/integration/simple_copy_test/seed-initial/seed.csv @@ -0,0 +1,101 @@ +id,first_name,last_name,email,gender,ip_address +1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168 +2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35 +3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243 +4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175 +5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136 +6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220 +7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64 +8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13 +9,Gary,Day,gday8@nih.gov,Male,35.81.68.186 +10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100 +11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67 +12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193 +13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5 +14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250 +15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245 +16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54 +17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96 +18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72 +19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174 +20,Andrew,Davis,adavisj@patch.com,Male,9.255.67.25 +21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253 +22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153 +23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201 +24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122 +25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95 +26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52 +27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26 +28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118 +29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28 +30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177 +31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233 +32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203 +33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149 +34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167 +35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110 +36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68 +37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89 +38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81 +39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15 +40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255 +41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140 +42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24 +43,Sean,Mason,smason16@icq.com,Male,159.219.155.249 +44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218 +45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198 +46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18 +47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238 +48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61 +49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21 +50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209 +51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87 +52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142 +53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126 +54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212 +55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194 +56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22 +57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60 +58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50 +59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222 +60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115 +61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155 +62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94 +63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106 +64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68 +65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41 +66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109 +67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77 +68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194 +69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135 +70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87 +71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44 +72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182 +73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241 +74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24 +75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214 +76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199 +77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41 +78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255 +79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144 +80,Rose,King,rking27@ucoz.com,Female,212.123.168.231 +81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188 +82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61 +83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30 +84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192 +85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232 +86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109 +87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156 +88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84 +89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235 +90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53 +91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221 +92,Angela,Rogers,arogers2j@goodreads.com,Female,98.104.132.187 +93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57 +94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189 +95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180 +96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144 +97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117 +98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126 +99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244 +100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88 diff --git a/tests/integration/simple_copy_test/seed-update/seed.csv b/tests/integration/simple_copy_test/seed-update/seed.csv new file mode 100644 index 000000000..5b93306a2 --- /dev/null +++ b/tests/integration/simple_copy_test/seed-update/seed.csv @@ -0,0 +1,201 @@ +id,first_name,last_name,email,gender,ip_address +1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168 +2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35 +3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243 +4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175 +5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136 +6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220 +7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64 +8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13 +9,Gary,Day,gday8@nih.gov,Male,35.81.68.186 +10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100 +11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67 +12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193 +13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5 +14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250 +15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245 +16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54 +17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96 +18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72 +19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174 +20,Andrew,Davis,adavisj@patch.com,Male,9.255.67.25 +21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253 +22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153 +23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201 +24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122 +25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95 +26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52 +27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26 +28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118 +29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28 +30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177 +31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233 +32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203 +33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149 +34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167 +35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110 +36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68 +37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89 +38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81 +39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15 +40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255 +41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140 +42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24 +43,Sean,Mason,smason16@icq.com,Male,159.219.155.249 +44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218 +45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198 +46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18 +47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238 +48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61 +49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21 +50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209 +51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87 +52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142 +53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126 +54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212 +55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194 +56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22 +57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60 +58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50 +59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222 +60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115 +61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155 +62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94 +63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106 +64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68 +65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41 +66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109 +67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77 +68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194 +69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135 +70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87 +71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44 +72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182 +73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241 +74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24 +75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214 +76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199 +77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41 +78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255 +79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144 +80,Rose,King,rking27@ucoz.com,Female,212.123.168.231 +81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188 +82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61 +83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30 +84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192 +85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232 +86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109 +87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156 +88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84 +89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235 +90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53 +91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221 +92,Angela,Rogers,arogers2j@goodreads.com,Female,98.104.132.187 +93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57 +94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189 +95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180 +96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144 +97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117 +98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126 +99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244 +100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88 +101,Michael,Perez,mperez0@chronoengine.com,Male,106.239.70.175 +102,Shawn,Mccoy,smccoy1@reddit.com,Male,24.165.76.182 +103,Kathleen,Payne,kpayne2@cargocollective.com,Female,113.207.168.106 +104,Jimmy,Cooper,jcooper3@cargocollective.com,Male,198.24.63.114 +105,Katherine,Rice,krice4@typepad.com,Female,36.97.186.238 +106,Sarah,Ryan,sryan5@gnu.org,Female,119.117.152.40 +107,Martin,Mcdonald,mmcdonald6@opera.com,Male,8.76.38.115 +108,Frank,Robinson,frobinson7@wunderground.com,Male,186.14.64.194 +109,Jennifer,Franklin,jfranklin8@mail.ru,Female,91.216.3.131 +110,Henry,Welch,hwelch9@list-manage.com,Male,176.35.182.168 +111,Fred,Snyder,fsnydera@reddit.com,Male,217.106.196.54 +112,Amy,Dunn,adunnb@nba.com,Female,95.39.163.195 +113,Kathleen,Meyer,kmeyerc@cdc.gov,Female,164.142.188.214 +114,Steve,Ferguson,sfergusond@reverbnation.com,Male,138.22.204.251 +115,Teresa,Hill,thille@dion.ne.jp,Female,82.84.228.235 +116,Amanda,Harper,aharperf@mail.ru,Female,16.123.56.176 +117,Kimberly,Ray,krayg@xing.com,Female,48.66.48.12 +118,Johnny,Knight,jknighth@jalbum.net,Male,99.30.138.123 +119,Virginia,Freeman,vfreemani@tiny.cc,Female,225.172.182.63 +120,Anna,Austin,aaustinj@diigo.com,Female,62.111.227.148 +121,Willie,Hill,whillk@mail.ru,Male,0.86.232.249 +122,Sean,Harris,sharrisl@zdnet.com,Male,117.165.133.249 +123,Mildred,Adams,madamsm@usatoday.com,Female,163.44.97.46 +124,David,Graham,dgrahamn@zimbio.com,Male,78.13.246.202 +125,Victor,Hunter,vhuntero@ehow.com,Male,64.156.179.139 +126,Aaron,Ruiz,aruizp@weebly.com,Male,34.194.68.78 +127,Benjamin,Brooks,bbrooksq@jalbum.net,Male,20.192.189.107 +128,Lisa,Wilson,lwilsonr@japanpost.jp,Female,199.152.130.217 +129,Benjamin,King,bkings@comsenz.com,Male,29.189.189.213 +130,Christina,Williamson,cwilliamsont@boston.com,Female,194.101.52.60 +131,Jane,Gonzalez,jgonzalezu@networksolutions.com,Female,109.119.12.87 +132,Thomas,Owens,towensv@psu.edu,Male,84.168.213.153 +133,Katherine,Moore,kmoorew@naver.com,Female,183.150.65.24 +134,Jennifer,Stewart,jstewartx@yahoo.com,Female,38.41.244.58 +135,Sara,Tucker,stuckery@topsy.com,Female,181.130.59.184 +136,Harold,Ortiz,hortizz@vkontakte.ru,Male,198.231.63.137 +137,Shirley,James,sjames10@yelp.com,Female,83.27.160.104 +138,Dennis,Johnson,djohnson11@slate.com,Male,183.178.246.101 +139,Louise,Weaver,lweaver12@china.com.cn,Female,1.14.110.18 +140,Maria,Armstrong,marmstrong13@prweb.com,Female,181.142.1.249 +141,Gloria,Cruz,gcruz14@odnoklassniki.ru,Female,178.232.140.243 +142,Diana,Spencer,dspencer15@ifeng.com,Female,125.153.138.244 +143,Kelly,Nguyen,knguyen16@altervista.org,Female,170.13.201.119 +144,Jane,Rodriguez,jrodriguez17@biblegateway.com,Female,12.102.249.81 +145,Scott,Brown,sbrown18@geocities.jp,Male,108.174.99.192 +146,Norma,Cruz,ncruz19@si.edu,Female,201.112.156.197 +147,Marie,Peters,mpeters1a@mlb.com,Female,231.121.197.144 +148,Lillian,Carr,lcarr1b@typepad.com,Female,206.179.164.163 +149,Judy,Nichols,jnichols1c@t-online.de,Female,158.190.209.194 +150,Billy,Long,blong1d@yahoo.com,Male,175.20.23.160 +151,Howard,Reid,hreid1e@exblog.jp,Male,118.99.196.20 +152,Laura,Ferguson,lferguson1f@tuttocitta.it,Female,22.77.87.110 +153,Anne,Bailey,abailey1g@geocities.com,Female,58.144.159.245 +154,Rose,Morgan,rmorgan1h@ehow.com,Female,118.127.97.4 +155,Nicholas,Reyes,nreyes1i@google.ru,Male,50.135.10.252 +156,Joshua,Kennedy,jkennedy1j@house.gov,Male,154.6.163.209 +157,Paul,Watkins,pwatkins1k@upenn.edu,Male,177.236.120.87 +158,Kathryn,Kelly,kkelly1l@businessweek.com,Female,70.28.61.86 +159,Adam,Armstrong,aarmstrong1m@techcrunch.com,Male,133.235.24.202 +160,Norma,Wallace,nwallace1n@phoca.cz,Female,241.119.227.128 +161,Timothy,Reyes,treyes1o@google.cn,Male,86.28.23.26 +162,Elizabeth,Patterson,epatterson1p@sun.com,Female,139.97.159.149 +163,Edward,Gomez,egomez1q@google.fr,Male,158.103.108.255 +164,David,Cox,dcox1r@friendfeed.com,Male,206.80.80.58 +165,Brenda,Wood,bwood1s@over-blog.com,Female,217.207.44.179 +166,Adam,Walker,awalker1t@blogs.com,Male,253.211.54.93 +167,Michael,Hart,mhart1u@wix.com,Male,230.206.200.22 +168,Jesse,Ellis,jellis1v@google.co.uk,Male,213.254.162.52 +169,Janet,Powell,jpowell1w@un.org,Female,27.192.194.86 +170,Helen,Ford,hford1x@creativecommons.org,Female,52.160.102.168 +171,Gerald,Carpenter,gcarpenter1y@about.me,Male,36.30.194.218 +172,Kathryn,Oliver,koliver1z@army.mil,Female,202.63.103.69 +173,Alan,Berry,aberry20@gov.uk,Male,246.157.112.211 +174,Harry,Andrews,handrews21@ameblo.jp,Male,195.108.0.12 +175,Andrea,Hall,ahall22@hp.com,Female,149.162.163.28 +176,Barbara,Wells,bwells23@behance.net,Female,224.70.72.1 +177,Anne,Wells,awells24@apache.org,Female,180.168.81.153 +178,Harry,Harper,hharper25@rediff.com,Male,151.87.130.21 +179,Jack,Ray,jray26@wufoo.com,Male,220.109.38.178 +180,Phillip,Hamilton,phamilton27@joomla.org,Male,166.40.47.30 +181,Shirley,Hunter,shunter28@newsvine.com,Female,97.209.140.194 +182,Arthur,Daniels,adaniels29@reuters.com,Male,5.40.240.86 +183,Virginia,Rodriguez,vrodriguez2a@walmart.com,Female,96.80.164.184 +184,Christina,Ryan,cryan2b@hibu.com,Female,56.35.5.52 +185,Theresa,Mendoza,tmendoza2c@vinaora.com,Female,243.42.0.210 +186,Jason,Cole,jcole2d@ycombinator.com,Male,198.248.39.129 +187,Phillip,Bryant,pbryant2e@rediff.com,Male,140.39.116.251 +188,Adam,Torres,atorres2f@sun.com,Male,101.75.187.135 +189,Margaret,Johnston,mjohnston2g@ucsd.edu,Female,159.30.69.149 +190,Paul,Payne,ppayne2h@hhs.gov,Male,199.234.140.220 +191,Todd,Willis,twillis2i@businessweek.com,Male,191.59.136.214 +192,Willie,Oliver,woliver2j@noaa.gov,Male,44.212.35.197 +193,Frances,Robertson,frobertson2k@go.com,Female,31.117.65.136 +194,Gregory,Hawkins,ghawkins2l@joomla.org,Male,91.3.22.49 +195,Lisa,Perkins,lperkins2m@si.edu,Female,145.95.31.186 +196,Jacqueline,Anderson,janderson2n@cargocollective.com,Female,14.176.0.187 +197,Shirley,Diaz,sdiaz2o@ucla.edu,Female,207.12.95.46 +198,Nicole,Meyer,nmeyer2p@flickr.com,Female,231.79.115.13 +199,Mary,Gray,mgray2q@constantcontact.com,Female,210.116.64.253 +200,Jean,Mcdonald,jmcdonald2r@baidu.com,Female,122.239.235.117 diff --git a/tests/integration/simple_copy_test/seeds-merge-cols-initial/seed.csv b/tests/integration/simple_copy_test/seeds-merge-cols-initial/seed.csv new file mode 100644 index 000000000..1333fd202 --- /dev/null +++ b/tests/integration/simple_copy_test/seeds-merge-cols-initial/seed.csv @@ -0,0 +1,101 @@ +load_date,id,first_name,last_name,email,gender,ip_address +2021-03-05,1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168 +2021-03-05,2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35 +2021-03-05,3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243 +2021-03-05,4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175 +2021-03-05,5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136 +2021-03-05,6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220 +2021-03-05,7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64 +2021-03-05,8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13 +2021-03-05,9,Gary,Day,gday8@nih.gov,Male,35.81.68.186 +2021-03-05,10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100 +2021-03-05,11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67 +2021-03-05,12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193 +2021-03-05,13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5 +2021-03-05,14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250 +2021-03-05,15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245 +2021-03-05,16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54 +2021-03-05,17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96 +2021-03-05,18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72 +2021-03-05,19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174 +2021-03-05,20,Andrew,Davis,adavisj@patch.com,Male,9.255.67.25 +2021-03-05,21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253 +2021-03-05,22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153 +2021-03-05,23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201 +2021-03-05,24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122 +2021-03-05,25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95 +2021-03-05,26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52 +2021-03-05,27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26 +2021-03-05,28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118 +2021-03-05,29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28 +2021-03-05,30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177 +2021-03-05,31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233 +2021-03-05,32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203 +2021-03-05,33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149 +2021-03-05,34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167 +2021-03-05,35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110 +2021-03-05,36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68 +2021-03-05,37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89 +2021-03-05,38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81 +2021-03-05,39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15 +2021-03-05,40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255 +2021-03-05,41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140 +2021-03-05,42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24 +2021-03-05,43,Sean,Mason,smason16@icq.com,Male,159.219.155.249 +2021-03-05,44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218 +2021-03-05,45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198 +2021-03-05,46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18 +2021-03-05,47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238 +2021-03-05,48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61 +2021-03-05,49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21 +2021-03-05,50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209 +2021-03-05,51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87 +2021-03-05,52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142 +2021-03-05,53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126 +2021-03-05,54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212 +2021-03-05,55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194 +2021-03-05,56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22 +2021-03-05,57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60 +2021-03-05,58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50 +2021-03-05,59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222 +2021-03-05,60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115 +2021-03-05,61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155 +2021-03-05,62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94 +2021-03-05,63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106 +2021-03-05,64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68 +2021-03-05,65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41 +2021-03-05,66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109 +2021-03-05,67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77 +2021-03-05,68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194 +2021-03-05,69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135 +2021-03-05,70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87 +2021-03-05,71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44 +2021-03-05,72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182 +2021-03-05,73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241 +2021-03-05,74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24 +2021-03-05,75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214 +2021-03-05,76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199 +2021-03-05,77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41 +2021-03-05,78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255 +2021-03-05,79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144 +2021-03-05,80,Rose,King,rking27@ucoz.com,Female,212.123.168.231 +2021-03-05,81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188 +2021-03-05,82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61 +2021-03-05,83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30 +2021-03-05,84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192 +2021-03-05,85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232 +2021-03-05,86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109 +2021-03-05,87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156 +2021-03-05,88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84 +2021-03-05,89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235 +2021-03-05,90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53 +2021-03-05,91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221 +2021-03-05,92,Angela,Rogers,arogers2j@goodreads.com,Female,98.104.132.187 +2021-03-05,93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57 +2021-03-05,94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189 +2021-03-05,95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180 +2021-03-05,96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144 +2021-03-05,97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117 +2021-03-05,98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126 +2021-03-05,99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244 +2021-03-05,100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88 diff --git a/tests/integration/simple_copy_test/seeds-merge-cols-update/expected_result.csv b/tests/integration/simple_copy_test/seeds-merge-cols-update/expected_result.csv new file mode 100644 index 000000000..b9e0e900b --- /dev/null +++ b/tests/integration/simple_copy_test/seeds-merge-cols-update/expected_result.csv @@ -0,0 +1,201 @@ +load_date,id,first_name,last_name,email,gender,ip_address +2021-03-05,1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168 +2021-03-05,2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35 +2021-03-05,3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243 +2021-03-05,4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175 +2021-03-05,5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136 +2021-03-05,6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220 +2021-03-05,7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64 +2021-03-05,8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13 +2021-03-05,9,Gary,Day,gday8@nih.gov,Male,35.81.68.186 +2021-03-05,10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100 +2021-03-05,11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67 +2021-03-05,12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193 +2021-03-05,13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5 +2021-03-05,14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250 +2021-03-05,15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245 +2021-03-05,16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54 +2021-03-05,17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96 +2021-03-05,18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72 +2021-03-05,19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174 +2021-03-05,20,Andrew,Davis,adavisj@reddit.com,Male,9.255.67.25 +2021-03-05,21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253 +2021-03-05,22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153 +2021-03-05,23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201 +2021-03-05,24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122 +2021-03-05,25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95 +2021-03-05,26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52 +2021-03-05,27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26 +2021-03-05,28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118 +2021-03-05,29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28 +2021-03-05,30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177 +2021-03-05,31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233 +2021-03-05,32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203 +2021-03-05,33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149 +2021-03-05,34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167 +2021-03-05,35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110 +2021-03-05,36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68 +2021-03-05,37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89 +2021-03-05,38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81 +2021-03-05,39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15 +2021-03-05,40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255 +2021-03-05,41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140 +2021-03-05,42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24 +2021-03-05,43,Sean,Mason,smason16@icq.com,Male,159.219.155.249 +2021-03-05,44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218 +2021-03-05,45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198 +2021-03-05,46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18 +2021-03-05,47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238 +2021-03-05,48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61 +2021-03-05,49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21 +2021-03-05,50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209 +2021-03-05,51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87 +2021-03-05,52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142 +2021-03-05,53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126 +2021-03-05,54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212 +2021-03-05,55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194 +2021-03-05,56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22 +2021-03-05,57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60 +2021-03-05,58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50 +2021-03-05,59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222 +2021-03-05,60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115 +2021-03-05,61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155 +2021-03-05,62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94 +2021-03-05,63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106 +2021-03-05,64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68 +2021-03-05,65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41 +2021-03-05,66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109 +2021-03-05,67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77 +2021-03-05,68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194 +2021-03-05,69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135 +2021-03-05,70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87 +2021-03-05,71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44 +2021-03-05,72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182 +2021-03-05,73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241 +2021-03-05,74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24 +2021-03-05,75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214 +2021-03-05,76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199 +2021-03-05,77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41 +2021-03-05,78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255 +2021-03-05,79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144 +2021-03-05,80,Rose,King,rking27@ucoz.com,Female,212.123.168.231 +2021-03-05,81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188 +2021-03-05,82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61 +2021-03-05,83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30 +2021-03-05,84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192 +2021-03-05,85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232 +2021-03-05,86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109 +2021-03-05,87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156 +2021-03-05,88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84 +2021-03-05,89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235 +2021-03-05,90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53 +2021-03-05,91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221 +2021-03-05,92,Angela,Rogers,ascott2j@goodreads.com,Female,98.119.208.155 +2021-03-05,93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57 +2021-03-05,94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189 +2021-03-05,95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180 +2021-03-05,96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144 +2021-03-05,97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117 +2021-03-05,98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126 +2021-03-05,99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244 +2021-03-05,100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88 +2021-03-06,101,Michael,Perez,mperez0@chronoengine.com,Male,106.239.70.175 +2021-03-06,102,Shawn,Mccoy,smccoy1@reddit.com,Male,24.165.76.182 +2021-03-06,103,Kathleen,Payne,kpayne2@cargocollective.com,Female,113.207.168.106 +2021-03-06,104,Jimmy,Cooper,jcooper3@cargocollective.com,Male,198.24.63.114 +2021-03-06,105,Katherine,Rice,krice4@typepad.com,Female,36.97.186.238 +2021-03-06,106,Sarah,Ryan,sryan5@gnu.org,Female,119.117.152.40 +2021-03-06,107,Martin,Mcdonald,mmcdonald6@opera.com,Male,8.76.38.115 +2021-03-06,108,Frank,Robinson,frobinson7@wunderground.com,Male,186.14.64.194 +2021-03-06,109,Jennifer,Franklin,jfranklin8@mail.ru,Female,91.216.3.131 +2021-03-06,110,Henry,Welch,hwelch9@list-manage.com,Male,176.35.182.168 +2021-03-06,111,Fred,Snyder,fsnydera@reddit.com,Male,217.106.196.54 +2021-03-06,112,Amy,Dunn,adunnb@nba.com,Female,95.39.163.195 +2021-03-06,113,Kathleen,Meyer,kmeyerc@cdc.gov,Female,164.142.188.214 +2021-03-06,114,Steve,Ferguson,sfergusond@reverbnation.com,Male,138.22.204.251 +2021-03-06,115,Teresa,Hill,thille@dion.ne.jp,Female,82.84.228.235 +2021-03-06,116,Amanda,Harper,aharperf@mail.ru,Female,16.123.56.176 +2021-03-06,117,Kimberly,Ray,krayg@xing.com,Female,48.66.48.12 +2021-03-06,118,Johnny,Knight,jknighth@jalbum.net,Male,99.30.138.123 +2021-03-06,119,Virginia,Freeman,vfreemani@tiny.cc,Female,225.172.182.63 +2021-03-06,120,Anna,Austin,aaustinj@diigo.com,Female,62.111.227.148 +2021-03-06,121,Willie,Hill,whillk@mail.ru,Male,0.86.232.249 +2021-03-06,122,Sean,Harris,sharrisl@zdnet.com,Male,117.165.133.249 +2021-03-06,123,Mildred,Adams,madamsm@usatoday.com,Female,163.44.97.46 +2021-03-06,124,David,Graham,dgrahamn@zimbio.com,Male,78.13.246.202 +2021-03-06,125,Victor,Hunter,vhuntero@ehow.com,Male,64.156.179.139 +2021-03-06,126,Aaron,Ruiz,aruizp@weebly.com,Male,34.194.68.78 +2021-03-06,127,Benjamin,Brooks,bbrooksq@jalbum.net,Male,20.192.189.107 +2021-03-06,128,Lisa,Wilson,lwilsonr@japanpost.jp,Female,199.152.130.217 +2021-03-06,129,Benjamin,King,bkings@comsenz.com,Male,29.189.189.213 +2021-03-06,130,Christina,Williamson,cwilliamsont@boston.com,Female,194.101.52.60 +2021-03-06,131,Jane,Gonzalez,jgonzalezu@networksolutions.com,Female,109.119.12.87 +2021-03-06,132,Thomas,Owens,towensv@psu.edu,Male,84.168.213.153 +2021-03-06,133,Katherine,Moore,kmoorew@naver.com,Female,183.150.65.24 +2021-03-06,134,Jennifer,Stewart,jstewartx@yahoo.com,Female,38.41.244.58 +2021-03-06,135,Sara,Tucker,stuckery@topsy.com,Female,181.130.59.184 +2021-03-06,136,Harold,Ortiz,hortizz@vkontakte.ru,Male,198.231.63.137 +2021-03-06,137,Shirley,James,sjames10@yelp.com,Female,83.27.160.104 +2021-03-06,138,Dennis,Johnson,djohnson11@slate.com,Male,183.178.246.101 +2021-03-06,139,Louise,Weaver,lweaver12@china.com.cn,Female,1.14.110.18 +2021-03-06,140,Maria,Armstrong,marmstrong13@prweb.com,Female,181.142.1.249 +2021-03-06,141,Gloria,Cruz,gcruz14@odnoklassniki.ru,Female,178.232.140.243 +2021-03-06,142,Diana,Spencer,dspencer15@ifeng.com,Female,125.153.138.244 +2021-03-06,143,Kelly,Nguyen,knguyen16@altervista.org,Female,170.13.201.119 +2021-03-06,144,Jane,Rodriguez,jrodriguez17@biblegateway.com,Female,12.102.249.81 +2021-03-06,145,Scott,Brown,sbrown18@geocities.jp,Male,108.174.99.192 +2021-03-06,146,Norma,Cruz,ncruz19@si.edu,Female,201.112.156.197 +2021-03-06,147,Marie,Peters,mpeters1a@mlb.com,Female,231.121.197.144 +2021-03-06,148,Lillian,Carr,lcarr1b@typepad.com,Female,206.179.164.163 +2021-03-06,149,Judy,Nichols,jnichols1c@t-online.de,Female,158.190.209.194 +2021-03-06,150,Billy,Long,blong1d@yahoo.com,Male,175.20.23.160 +2021-03-06,151,Howard,Reid,hreid1e@exblog.jp,Male,118.99.196.20 +2021-03-06,152,Laura,Ferguson,lferguson1f@tuttocitta.it,Female,22.77.87.110 +2021-03-06,153,Anne,Bailey,abailey1g@geocities.com,Female,58.144.159.245 +2021-03-06,154,Rose,Morgan,rmorgan1h@ehow.com,Female,118.127.97.4 +2021-03-06,155,Nicholas,Reyes,nreyes1i@google.ru,Male,50.135.10.252 +2021-03-06,156,Joshua,Kennedy,jkennedy1j@house.gov,Male,154.6.163.209 +2021-03-06,157,Paul,Watkins,pwatkins1k@upenn.edu,Male,177.236.120.87 +2021-03-06,158,Kathryn,Kelly,kkelly1l@businessweek.com,Female,70.28.61.86 +2021-03-06,159,Adam,Armstrong,aarmstrong1m@techcrunch.com,Male,133.235.24.202 +2021-03-06,160,Norma,Wallace,nwallace1n@phoca.cz,Female,241.119.227.128 +2021-03-06,161,Timothy,Reyes,treyes1o@google.cn,Male,86.28.23.26 +2021-03-06,162,Elizabeth,Patterson,epatterson1p@sun.com,Female,139.97.159.149 +2021-03-06,163,Edward,Gomez,egomez1q@google.fr,Male,158.103.108.255 +2021-03-06,164,David,Cox,dcox1r@friendfeed.com,Male,206.80.80.58 +2021-03-06,165,Brenda,Wood,bwood1s@over-blog.com,Female,217.207.44.179 +2021-03-06,166,Adam,Walker,awalker1t@blogs.com,Male,253.211.54.93 +2021-03-06,167,Michael,Hart,mhart1u@wix.com,Male,230.206.200.22 +2021-03-06,168,Jesse,Ellis,jellis1v@google.co.uk,Male,213.254.162.52 +2021-03-06,169,Janet,Powell,jpowell1w@un.org,Female,27.192.194.86 +2021-03-06,170,Helen,Ford,hford1x@creativecommons.org,Female,52.160.102.168 +2021-03-06,171,Gerald,Carpenter,gcarpenter1y@about.me,Male,36.30.194.218 +2021-03-06,172,Kathryn,Oliver,koliver1z@army.mil,Female,202.63.103.69 +2021-03-06,173,Alan,Berry,aberry20@gov.uk,Male,246.157.112.211 +2021-03-06,174,Harry,Andrews,handrews21@ameblo.jp,Male,195.108.0.12 +2021-03-06,175,Andrea,Hall,ahall22@hp.com,Female,149.162.163.28 +2021-03-06,176,Barbara,Wells,bwells23@behance.net,Female,224.70.72.1 +2021-03-06,177,Anne,Wells,awells24@apache.org,Female,180.168.81.153 +2021-03-06,178,Harry,Harper,hharper25@rediff.com,Male,151.87.130.21 +2021-03-06,179,Jack,Ray,jray26@wufoo.com,Male,220.109.38.178 +2021-03-06,180,Phillip,Hamilton,phamilton27@joomla.org,Male,166.40.47.30 +2021-03-06,181,Shirley,Hunter,shunter28@newsvine.com,Female,97.209.140.194 +2021-03-06,182,Arthur,Daniels,adaniels29@reuters.com,Male,5.40.240.86 +2021-03-06,183,Virginia,Rodriguez,vrodriguez2a@walmart.com,Female,96.80.164.184 +2021-03-06,184,Christina,Ryan,cryan2b@hibu.com,Female,56.35.5.52 +2021-03-06,185,Theresa,Mendoza,tmendoza2c@vinaora.com,Female,243.42.0.210 +2021-03-06,186,Jason,Cole,jcole2d@ycombinator.com,Male,198.248.39.129 +2021-03-06,187,Phillip,Bryant,pbryant2e@rediff.com,Male,140.39.116.251 +2021-03-06,188,Adam,Torres,atorres2f@sun.com,Male,101.75.187.135 +2021-03-06,189,Margaret,Johnston,mjohnston2g@ucsd.edu,Female,159.30.69.149 +2021-03-06,190,Paul,Payne,ppayne2h@hhs.gov,Male,199.234.140.220 +2021-03-06,191,Todd,Willis,twillis2i@businessweek.com,Male,191.59.136.214 +2021-03-06,192,Willie,Oliver,woliver2j@noaa.gov,Male,44.212.35.197 +2021-03-06,193,Frances,Robertson,frobertson2k@go.com,Female,31.117.65.136 +2021-03-06,194,Gregory,Hawkins,ghawkins2l@joomla.org,Male,91.3.22.49 +2021-03-06,195,Lisa,Perkins,lperkins2m@si.edu,Female,145.95.31.186 +2021-03-06,196,Jacqueline,Anderson,janderson2n@cargocollective.com,Female,14.176.0.187 +2021-03-06,197,Shirley,Diaz,sdiaz2o@ucla.edu,Female,207.12.95.46 +2021-03-06,198,Nicole,Meyer,nmeyer2p@flickr.com,Female,231.79.115.13 +2021-03-06,199,Mary,Gray,mgray2q@constantcontact.com,Female,210.116.64.253 +2021-03-06,200,Jean,Mcdonald,jmcdonald2r@baidu.com,Female,122.239.235.117 diff --git a/tests/integration/simple_copy_test/seeds-merge-cols-update/seed.csv b/tests/integration/simple_copy_test/seeds-merge-cols-update/seed.csv new file mode 100644 index 000000000..2a2ff674c --- /dev/null +++ b/tests/integration/simple_copy_test/seeds-merge-cols-update/seed.csv @@ -0,0 +1,204 @@ +load_date,id,first_name,last_name,email,gender,ip_address +2021-03-05,1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168 +2021-03-05,2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35 +2021-03-05,3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243 +2021-03-05,4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175 +2021-03-05,5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136 +2021-03-05,6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220 +2021-03-05,7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64 +2021-03-05,8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13 +2021-03-05,9,Gary,Day,gday8@nih.gov,Male,35.81.68.186 +2021-03-05,10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100 +2021-03-05,11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67 +2021-03-05,12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193 +2021-03-05,13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5 +2021-03-05,14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250 +2021-03-05,15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245 +2021-03-05,16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54 +2021-03-05,17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96 +2021-03-05,18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72 +2021-03-05,19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174 +2021-03-05,20,Andrew,Davis,adavisj@patch.com,Male,9.255.67.25 +2021-03-05,21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253 +2021-03-05,22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153 +2021-03-05,23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201 +2021-03-05,24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122 +2021-03-05,25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95 +2021-03-05,26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52 +2021-03-05,27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26 +2021-03-05,28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118 +2021-03-05,29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28 +2021-03-05,30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177 +2021-03-05,31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233 +2021-03-05,32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203 +2021-03-05,33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149 +2021-03-05,34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167 +2021-03-05,35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110 +2021-03-05,36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68 +2021-03-05,37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89 +2021-03-05,38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81 +2021-03-05,39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15 +2021-03-05,40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255 +2021-03-05,41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140 +2021-03-05,42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24 +2021-03-05,43,Sean,Mason,smason16@icq.com,Male,159.219.155.249 +2021-03-05,44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218 +2021-03-05,45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198 +2021-03-05,46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18 +2021-03-05,47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238 +2021-03-05,48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61 +2021-03-05,49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21 +2021-03-05,50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209 +2021-03-05,51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87 +2021-03-05,52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142 +2021-03-05,53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126 +2021-03-05,54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212 +2021-03-05,55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194 +2021-03-05,56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22 +2021-03-05,57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60 +2021-03-05,58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50 +2021-03-05,59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222 +2021-03-05,60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115 +2021-03-05,61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155 +2021-03-05,62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94 +2021-03-05,63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106 +2021-03-05,64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68 +2021-03-05,65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41 +2021-03-05,66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109 +2021-03-05,67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77 +2021-03-05,68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194 +2021-03-05,69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135 +2021-03-05,70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87 +2021-03-05,71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44 +2021-03-05,72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182 +2021-03-05,73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241 +2021-03-05,74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24 +2021-03-05,75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214 +2021-03-05,76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199 +2021-03-05,77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41 +2021-03-05,78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255 +2021-03-05,79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144 +2021-03-05,80,Rose,King,rking27@ucoz.com,Female,212.123.168.231 +2021-03-05,81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188 +2021-03-05,82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61 +2021-03-05,83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30 +2021-03-05,84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192 +2021-03-05,85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232 +2021-03-05,86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109 +2021-03-05,87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156 +2021-03-05,88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84 +2021-03-05,89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235 +2021-03-05,90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53 +2021-03-05,91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221 +2021-03-05,92,Angela,Rogers,arogers2j@goodreads.com,Female,98.104.132.187 +2021-03-05,93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57 +2021-03-05,94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189 +2021-03-05,95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180 +2021-03-05,96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144 +2021-03-05,97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117 +2021-03-05,98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126 +2021-03-05,99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244 +2021-03-05,100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88 +2021-03-06,20,Andrew,Davis,adavisj@reddit.com,Male,9.255.67.25 +2021-03-06,83,Josh,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30 +2021-03-06,92,Angela,Scott,ascott2j@goodreads.com,Female,98.119.208.155 +2021-03-06,101,Michael,Perez,mperez0@chronoengine.com,Male,106.239.70.175 +2021-03-06,102,Shawn,Mccoy,smccoy1@reddit.com,Male,24.165.76.182 +2021-03-06,103,Kathleen,Payne,kpayne2@cargocollective.com,Female,113.207.168.106 +2021-03-06,104,Jimmy,Cooper,jcooper3@cargocollective.com,Male,198.24.63.114 +2021-03-06,105,Katherine,Rice,krice4@typepad.com,Female,36.97.186.238 +2021-03-06,106,Sarah,Ryan,sryan5@gnu.org,Female,119.117.152.40 +2021-03-06,107,Martin,Mcdonald,mmcdonald6@opera.com,Male,8.76.38.115 +2021-03-06,108,Frank,Robinson,frobinson7@wunderground.com,Male,186.14.64.194 +2021-03-06,109,Jennifer,Franklin,jfranklin8@mail.ru,Female,91.216.3.131 +2021-03-06,110,Henry,Welch,hwelch9@list-manage.com,Male,176.35.182.168 +2021-03-06,111,Fred,Snyder,fsnydera@reddit.com,Male,217.106.196.54 +2021-03-06,112,Amy,Dunn,adunnb@nba.com,Female,95.39.163.195 +2021-03-06,113,Kathleen,Meyer,kmeyerc@cdc.gov,Female,164.142.188.214 +2021-03-06,114,Steve,Ferguson,sfergusond@reverbnation.com,Male,138.22.204.251 +2021-03-06,115,Teresa,Hill,thille@dion.ne.jp,Female,82.84.228.235 +2021-03-06,116,Amanda,Harper,aharperf@mail.ru,Female,16.123.56.176 +2021-03-06,117,Kimberly,Ray,krayg@xing.com,Female,48.66.48.12 +2021-03-06,118,Johnny,Knight,jknighth@jalbum.net,Male,99.30.138.123 +2021-03-06,119,Virginia,Freeman,vfreemani@tiny.cc,Female,225.172.182.63 +2021-03-06,120,Anna,Austin,aaustinj@diigo.com,Female,62.111.227.148 +2021-03-06,121,Willie,Hill,whillk@mail.ru,Male,0.86.232.249 +2021-03-06,122,Sean,Harris,sharrisl@zdnet.com,Male,117.165.133.249 +2021-03-06,123,Mildred,Adams,madamsm@usatoday.com,Female,163.44.97.46 +2021-03-06,124,David,Graham,dgrahamn@zimbio.com,Male,78.13.246.202 +2021-03-06,125,Victor,Hunter,vhuntero@ehow.com,Male,64.156.179.139 +2021-03-06,126,Aaron,Ruiz,aruizp@weebly.com,Male,34.194.68.78 +2021-03-06,127,Benjamin,Brooks,bbrooksq@jalbum.net,Male,20.192.189.107 +2021-03-06,128,Lisa,Wilson,lwilsonr@japanpost.jp,Female,199.152.130.217 +2021-03-06,129,Benjamin,King,bkings@comsenz.com,Male,29.189.189.213 +2021-03-06,130,Christina,Williamson,cwilliamsont@boston.com,Female,194.101.52.60 +2021-03-06,131,Jane,Gonzalez,jgonzalezu@networksolutions.com,Female,109.119.12.87 +2021-03-06,132,Thomas,Owens,towensv@psu.edu,Male,84.168.213.153 +2021-03-06,133,Katherine,Moore,kmoorew@naver.com,Female,183.150.65.24 +2021-03-06,134,Jennifer,Stewart,jstewartx@yahoo.com,Female,38.41.244.58 +2021-03-06,135,Sara,Tucker,stuckery@topsy.com,Female,181.130.59.184 +2021-03-06,136,Harold,Ortiz,hortizz@vkontakte.ru,Male,198.231.63.137 +2021-03-06,137,Shirley,James,sjames10@yelp.com,Female,83.27.160.104 +2021-03-06,138,Dennis,Johnson,djohnson11@slate.com,Male,183.178.246.101 +2021-03-06,139,Louise,Weaver,lweaver12@china.com.cn,Female,1.14.110.18 +2021-03-06,140,Maria,Armstrong,marmstrong13@prweb.com,Female,181.142.1.249 +2021-03-06,141,Gloria,Cruz,gcruz14@odnoklassniki.ru,Female,178.232.140.243 +2021-03-06,142,Diana,Spencer,dspencer15@ifeng.com,Female,125.153.138.244 +2021-03-06,143,Kelly,Nguyen,knguyen16@altervista.org,Female,170.13.201.119 +2021-03-06,144,Jane,Rodriguez,jrodriguez17@biblegateway.com,Female,12.102.249.81 +2021-03-06,145,Scott,Brown,sbrown18@geocities.jp,Male,108.174.99.192 +2021-03-06,146,Norma,Cruz,ncruz19@si.edu,Female,201.112.156.197 +2021-03-06,147,Marie,Peters,mpeters1a@mlb.com,Female,231.121.197.144 +2021-03-06,148,Lillian,Carr,lcarr1b@typepad.com,Female,206.179.164.163 +2021-03-06,149,Judy,Nichols,jnichols1c@t-online.de,Female,158.190.209.194 +2021-03-06,150,Billy,Long,blong1d@yahoo.com,Male,175.20.23.160 +2021-03-06,151,Howard,Reid,hreid1e@exblog.jp,Male,118.99.196.20 +2021-03-06,152,Laura,Ferguson,lferguson1f@tuttocitta.it,Female,22.77.87.110 +2021-03-06,153,Anne,Bailey,abailey1g@geocities.com,Female,58.144.159.245 +2021-03-06,154,Rose,Morgan,rmorgan1h@ehow.com,Female,118.127.97.4 +2021-03-06,155,Nicholas,Reyes,nreyes1i@google.ru,Male,50.135.10.252 +2021-03-06,156,Joshua,Kennedy,jkennedy1j@house.gov,Male,154.6.163.209 +2021-03-06,157,Paul,Watkins,pwatkins1k@upenn.edu,Male,177.236.120.87 +2021-03-06,158,Kathryn,Kelly,kkelly1l@businessweek.com,Female,70.28.61.86 +2021-03-06,159,Adam,Armstrong,aarmstrong1m@techcrunch.com,Male,133.235.24.202 +2021-03-06,160,Norma,Wallace,nwallace1n@phoca.cz,Female,241.119.227.128 +2021-03-06,161,Timothy,Reyes,treyes1o@google.cn,Male,86.28.23.26 +2021-03-06,162,Elizabeth,Patterson,epatterson1p@sun.com,Female,139.97.159.149 +2021-03-06,163,Edward,Gomez,egomez1q@google.fr,Male,158.103.108.255 +2021-03-06,164,David,Cox,dcox1r@friendfeed.com,Male,206.80.80.58 +2021-03-06,165,Brenda,Wood,bwood1s@over-blog.com,Female,217.207.44.179 +2021-03-06,166,Adam,Walker,awalker1t@blogs.com,Male,253.211.54.93 +2021-03-06,167,Michael,Hart,mhart1u@wix.com,Male,230.206.200.22 +2021-03-06,168,Jesse,Ellis,jellis1v@google.co.uk,Male,213.254.162.52 +2021-03-06,169,Janet,Powell,jpowell1w@un.org,Female,27.192.194.86 +2021-03-06,170,Helen,Ford,hford1x@creativecommons.org,Female,52.160.102.168 +2021-03-06,171,Gerald,Carpenter,gcarpenter1y@about.me,Male,36.30.194.218 +2021-03-06,172,Kathryn,Oliver,koliver1z@army.mil,Female,202.63.103.69 +2021-03-06,173,Alan,Berry,aberry20@gov.uk,Male,246.157.112.211 +2021-03-06,174,Harry,Andrews,handrews21@ameblo.jp,Male,195.108.0.12 +2021-03-06,175,Andrea,Hall,ahall22@hp.com,Female,149.162.163.28 +2021-03-06,176,Barbara,Wells,bwells23@behance.net,Female,224.70.72.1 +2021-03-06,177,Anne,Wells,awells24@apache.org,Female,180.168.81.153 +2021-03-06,178,Harry,Harper,hharper25@rediff.com,Male,151.87.130.21 +2021-03-06,179,Jack,Ray,jray26@wufoo.com,Male,220.109.38.178 +2021-03-06,180,Phillip,Hamilton,phamilton27@joomla.org,Male,166.40.47.30 +2021-03-06,181,Shirley,Hunter,shunter28@newsvine.com,Female,97.209.140.194 +2021-03-06,182,Arthur,Daniels,adaniels29@reuters.com,Male,5.40.240.86 +2021-03-06,183,Virginia,Rodriguez,vrodriguez2a@walmart.com,Female,96.80.164.184 +2021-03-06,184,Christina,Ryan,cryan2b@hibu.com,Female,56.35.5.52 +2021-03-06,185,Theresa,Mendoza,tmendoza2c@vinaora.com,Female,243.42.0.210 +2021-03-06,186,Jason,Cole,jcole2d@ycombinator.com,Male,198.248.39.129 +2021-03-06,187,Phillip,Bryant,pbryant2e@rediff.com,Male,140.39.116.251 +2021-03-06,188,Adam,Torres,atorres2f@sun.com,Male,101.75.187.135 +2021-03-06,189,Margaret,Johnston,mjohnston2g@ucsd.edu,Female,159.30.69.149 +2021-03-06,190,Paul,Payne,ppayne2h@hhs.gov,Male,199.234.140.220 +2021-03-06,191,Todd,Willis,twillis2i@businessweek.com,Male,191.59.136.214 +2021-03-06,192,Willie,Oliver,woliver2j@noaa.gov,Male,44.212.35.197 +2021-03-06,193,Frances,Robertson,frobertson2k@go.com,Female,31.117.65.136 +2021-03-06,194,Gregory,Hawkins,ghawkins2l@joomla.org,Male,91.3.22.49 +2021-03-06,195,Lisa,Perkins,lperkins2m@si.edu,Female,145.95.31.186 +2021-03-06,196,Jacqueline,Anderson,janderson2n@cargocollective.com,Female,14.176.0.187 +2021-03-06,197,Shirley,Diaz,sdiaz2o@ucla.edu,Female,207.12.95.46 +2021-03-06,198,Nicole,Meyer,nmeyer2p@flickr.com,Female,231.79.115.13 +2021-03-06,199,Mary,Gray,mgray2q@constantcontact.com,Female,210.116.64.253 +2021-03-06,200,Jean,Mcdonald,jmcdonald2r@baidu.com,Female,122.239.235.117 diff --git a/tests/integration/simple_copy_test/test_simple_copy.py b/tests/integration/simple_copy_test/test_simple_copy.py new file mode 100644 index 000000000..39733c4e6 --- /dev/null +++ b/tests/integration/simple_copy_test/test_simple_copy.py @@ -0,0 +1,96 @@ +import json +import os +from pytest import mark + +from tests.integration.base import DBTIntegrationTest, use_profile + + +class BaseTestSimpleCopy(DBTIntegrationTest): + @property + def schema(self): + return "simple_copy" + + @staticmethod + def dir(path): + return path.lstrip('/') + + @property + def models(self): + return self.dir("models") + + @property + def project_config(self): + return self.seed_quote_cfg_with({ + 'profile': '{{ "tes" ~ "t" }}' + }) + + def seed_quote_cfg_with(self, extra): + cfg = { + 'config-version': 2, + 'seeds': { + 'quote_columns': False, + } + } + cfg.update(extra) + return cfg + + +class TestSimpleCopy(BaseTestSimpleCopy): + + @property + def project_config(self): + return self.seed_quote_cfg_with({"data-paths": [self.dir("seed-initial")]}) + + @use_profile("bigquery") + def test__bigquery__simple_copy(self): + results = self.run_dbt(["seed"]) + self.assertEqual(len(results), 1) + results = self.run_dbt() + self.assertEqual(len(results), 7) + + self.assertTablesEqual("seed", "view_model") + self.assertTablesEqual("seed", "incremental") + self.assertTablesEqual("seed", "materialized") + self.assertTablesEqual("seed", "get_and_ref") + + self.use_default_project({"data-paths": [self.dir("seed-update")]}) + + results = self.run_dbt(["seed"]) + self.assertEqual(len(results), 1) + results = self.run_dbt() + self.assertEqual(len(results), 7) + + self.assertTablesEqual("seed", "view_model") + self.assertTablesEqual("seed", "incremental") + self.assertTablesEqual("seed", "materialized") + self.assertTablesEqual("seed", "get_and_ref") + + +class TestIncrementalMergeColumns(BaseTestSimpleCopy): + @property + def models(self): + return self.dir("models-merge-update") + + @property + def project_config(self): + return { + "seeds": { + "quote_columns": False + } + } + + def seed_and_run(self): + self.run_dbt(["seed"]) + self.run_dbt(["run"]) + + @use_profile("bigquery") + def test__bigquery__incremental_merge_columns(self): + self.use_default_project({ + "data-paths": ["seeds-merge-cols-initial"] + }) + self.seed_and_run() + self.use_default_project({ + "data-paths": ["seeds-merge-cols-update"] + }) + self.seed_and_run() + self.assertTablesEqual("incremental_update_cols", "expected_result") diff --git a/tests/integration/simple_seed_test/data-config/seed_disabled.csv b/tests/integration/simple_seed_test/data-config/seed_disabled.csv new file mode 100644 index 000000000..0227dd609 --- /dev/null +++ b/tests/integration/simple_seed_test/data-config/seed_disabled.csv @@ -0,0 +1,21 @@ +id,first_name,email,ip_address,birthday +1,Larry,lking0@miitbeian.gov.cn,69.135.206.194,2008-09-12 19:08:31 +2,Larry,lperkins1@toplist.cz,64.210.133.162,1978-05-09 04:15:14 +3,Anna,amontgomery2@miitbeian.gov.cn,168.104.64.114,2011-10-16 04:07:57 +4,Sandra,sgeorge3@livejournal.com,229.235.252.98,1973-07-19 10:52:43 +5,Fred,fwoods4@google.cn,78.229.170.124,2012-09-30 16:38:29 +6,Stephen,shanson5@livejournal.com,182.227.157.105,1995-11-07 21:40:50 +7,William,wmartinez6@upenn.edu,135.139.249.50,1982-09-05 03:11:59 +8,Jessica,jlong7@hao123.com,203.62.178.210,1991-10-16 11:03:15 +9,Douglas,dwhite8@tamu.edu,178.187.247.1,1979-10-01 09:49:48 +10,Lisa,lcoleman9@nydailynews.com,168.234.128.249,2011-05-26 07:45:49 +11,Ralph,rfieldsa@home.pl,55.152.163.149,1972-11-18 19:06:11 +12,Louise,lnicholsb@samsung.com,141.116.153.154,2014-11-25 20:56:14 +13,Clarence,cduncanc@sfgate.com,81.171.31.133,2011-11-17 07:02:36 +14,Daniel,dfranklind@omniture.com,8.204.211.37,1980-09-13 00:09:04 +15,Katherine,klanee@auda.org.au,176.96.134.59,1997-08-22 19:36:56 +16,Billy,bwardf@wikia.com,214.108.78.85,2003-10-19 02:14:47 +17,Annie,agarzag@ocn.ne.jp,190.108.42.70,1988-10-28 15:12:35 +18,Shirley,scolemanh@fastcompany.com,109.251.164.84,1988-08-24 10:50:57 +19,Roger,rfrazieri@scribd.com,38.145.218.108,1985-12-31 15:17:15 +20,Lillian,lstanleyj@goodreads.com,47.57.236.17,1970-06-08 02:09:05 diff --git a/tests/integration/simple_seed_test/data-config/seed_enabled.csv b/tests/integration/simple_seed_test/data-config/seed_enabled.csv new file mode 100644 index 000000000..0227dd609 --- /dev/null +++ b/tests/integration/simple_seed_test/data-config/seed_enabled.csv @@ -0,0 +1,21 @@ +id,first_name,email,ip_address,birthday +1,Larry,lking0@miitbeian.gov.cn,69.135.206.194,2008-09-12 19:08:31 +2,Larry,lperkins1@toplist.cz,64.210.133.162,1978-05-09 04:15:14 +3,Anna,amontgomery2@miitbeian.gov.cn,168.104.64.114,2011-10-16 04:07:57 +4,Sandra,sgeorge3@livejournal.com,229.235.252.98,1973-07-19 10:52:43 +5,Fred,fwoods4@google.cn,78.229.170.124,2012-09-30 16:38:29 +6,Stephen,shanson5@livejournal.com,182.227.157.105,1995-11-07 21:40:50 +7,William,wmartinez6@upenn.edu,135.139.249.50,1982-09-05 03:11:59 +8,Jessica,jlong7@hao123.com,203.62.178.210,1991-10-16 11:03:15 +9,Douglas,dwhite8@tamu.edu,178.187.247.1,1979-10-01 09:49:48 +10,Lisa,lcoleman9@nydailynews.com,168.234.128.249,2011-05-26 07:45:49 +11,Ralph,rfieldsa@home.pl,55.152.163.149,1972-11-18 19:06:11 +12,Louise,lnicholsb@samsung.com,141.116.153.154,2014-11-25 20:56:14 +13,Clarence,cduncanc@sfgate.com,81.171.31.133,2011-11-17 07:02:36 +14,Daniel,dfranklind@omniture.com,8.204.211.37,1980-09-13 00:09:04 +15,Katherine,klanee@auda.org.au,176.96.134.59,1997-08-22 19:36:56 +16,Billy,bwardf@wikia.com,214.108.78.85,2003-10-19 02:14:47 +17,Annie,agarzag@ocn.ne.jp,190.108.42.70,1988-10-28 15:12:35 +18,Shirley,scolemanh@fastcompany.com,109.251.164.84,1988-08-24 10:50:57 +19,Roger,rfrazieri@scribd.com,38.145.218.108,1985-12-31 15:17:15 +20,Lillian,lstanleyj@goodreads.com,47.57.236.17,1970-06-08 02:09:05 diff --git a/tests/integration/simple_seed_test/data-config/seed_tricky.csv b/tests/integration/simple_seed_test/data-config/seed_tricky.csv new file mode 100644 index 000000000..3e90a18e3 --- /dev/null +++ b/tests/integration/simple_seed_test/data-config/seed_tricky.csv @@ -0,0 +1,7 @@ +id,id_str,a_bool,looks_like_a_bool,a_date,looks_like_a_date,relative,weekday +1,1,true,true,2019-01-01 12:32:30,2019-01-01 12:32:30,tomorrow,Saturday +2,2,True,True,2019-01-01 12:32:31,2019-01-01 12:32:31,today,Sunday +3,3,TRUE,TRUE,2019-01-01 12:32:32,2019-01-01 12:32:32,yesterday,Monday +4,4,false,false,2019-01-01 01:32:32,2019-01-01 01:32:32,tomorrow,Saturday +5,5,False,False,2019-01-01 01:32:32,2019-01-01 01:32:32,today,Sunday +6,6,FALSE,FALSE,2019-01-01 01:32:32,2019-01-01 01:32:32,yesterday,Monday diff --git a/tests/integration/simple_seed_test/macros/schema_test.sql b/tests/integration/simple_seed_test/macros/schema_test.sql new file mode 100644 index 000000000..5c7f25964 --- /dev/null +++ b/tests/integration/simple_seed_test/macros/schema_test.sql @@ -0,0 +1,22 @@ + +{% test column_type(model, column_name, type) %} + + {% set cols = adapter.get_columns_in_relation(model) %} + + {% set col_types = {} %} + {% for col in cols %} + {% do col_types.update({col.name: col.data_type}) %} + {% endfor %} + + {% set validation_message = 'Got a column type of ' ~ col_types.get(column_name) ~ ', expected ' ~ type %} + + {% set val = 0 if col_types.get(column_name) == type else 1 %} + {% if val == 1 and execute %} + {{ log(validation_message, info=True) }} + {% endif %} + + select '{{ validation_message }}' as validation_error + from (select true) as nothing + where {{ val }} = 1 + +{% endtest %} diff --git a/tests/integration/simple_seed_test/models-bq/schema.yml b/tests/integration/simple_seed_test/models-bq/schema.yml new file mode 100644 index 000000000..019a9524f --- /dev/null +++ b/tests/integration/simple_seed_test/models-bq/schema.yml @@ -0,0 +1,47 @@ +version: 2 +seeds: +- name: seed_enabled + columns: + - name: birthday + tests: + - column_type: + type: STRING + - name: id + tests: + - column_type: + type: FLOAT64 + +- name: seed_tricky + columns: + - name: id + tests: + - column_type: + type: INT64 + - name: id_str + tests: + - column_type: + type: STRING + - name: a_bool + tests: + - column_type: + type: BOOLEAN + - name: looks_like_a_bool + tests: + - column_type: + type: STRING + - name: a_date + tests: + - column_type: + type: DATETIME + - name: looks_like_a_date + tests: + - column_type: + type: STRING + - name: relative + tests: + - column_type: + type: STRING + - name: weekday + tests: + - column_type: + type: STRING diff --git a/tests/integration/simple_seed_test/test_seed_type_override.py b/tests/integration/simple_seed_test/test_seed_type_override.py new file mode 100644 index 000000000..824226816 --- /dev/null +++ b/tests/integration/simple_seed_test/test_seed_type_override.py @@ -0,0 +1,60 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + + +class TestSimpleSeedColumnOverride(DBTIntegrationTest): + + @property + def schema(self): + return "simple_seed" + + @property + def project_config(self): + return { + 'config-version': 2, + 'data-paths': ['data-config'], + 'macro-paths': ['macros'], + 'seeds': { + 'test': { + 'enabled': False, + 'quote_columns': True, + 'seed_enabled': { + 'enabled': True, + '+column_types': self.seed_enabled_types() + }, + 'seed_tricky': { + 'enabled': True, + '+column_types': self.seed_tricky_types(), + }, + }, + }, + } + + +class TestSimpleSeedColumnOverrideBQ(TestSimpleSeedColumnOverride): + @property + def models(self): + return "models-bq" + + def seed_enabled_types(self): + return { + "id": "FLOAT64", + "birthday": "STRING", + } + + def seed_tricky_types(self): + return { + 'id_str': 'STRING', + 'looks_like_a_bool': 'STRING', + 'looks_like_a_date': 'STRING', + } + + @property + def profile_config(self): + return self.bigquery_profile() + + @use_profile('bigquery') + def test_bigquery_simple_seed_with_column_override_bigquery(self): + results = self.run_dbt(["seed", "--show"]) + self.assertEqual(len(results), 2) + results = self.run_dbt(["test"]) + self.assertEqual(len(results), 10) diff --git a/tests/integration/simple_snapshot_test/add_column_to_source_bq.sql b/tests/integration/simple_snapshot_test/add_column_to_source_bq.sql new file mode 100644 index 000000000..e1babb82c --- /dev/null +++ b/tests/integration/simple_snapshot_test/add_column_to_source_bq.sql @@ -0,0 +1,56 @@ + +create or replace table {schema}.seed as ( + + select *, + [ + struct( + 1 as field_1, + 2 as field_2 + ), + struct( + 3 as field_1, + 4 as field_2 + ) + ] as repeated_nested_field, + + struct( + 1 as field_1, + 2 as field_2 + ) as nested_field, + + [ + 1, + 2 + ] as repeated_field + + from {schema}.seed + +); + +create or replace table {schema}.snapshot_expected as ( + + select *, + [ + struct( + 1 as field_1, + 2 as field_2 + ), + struct( + 3 as field_1, + 4 as field_2 + ) + ] as repeated_nested_field, + + struct( + 1 as field_1, + 2 as field_2 + ) as nested_field, + + [ + 1, + 2 + ] as repeated_field + + from {schema}.snapshot_expected + +); diff --git a/tests/integration/simple_snapshot_test/check-snapshots-expected/check_snapshots_test_current.sql b/tests/integration/simple_snapshot_test/check-snapshots-expected/check_snapshots_test_current.sql new file mode 100644 index 000000000..414afb472 --- /dev/null +++ b/tests/integration/simple_snapshot_test/check-snapshots-expected/check_snapshots_test_current.sql @@ -0,0 +1,51 @@ + + +with query as ( + + -- check that the current value for id=1 is red + select case when ( + select count(*) + from {{ ref('check_cols_cycle') }} + where id = 1 and color = 'red' and dbt_valid_to is null + ) = 1 then 0 else 1 end as failures + + union all + + -- check that the previous 'red' value for id=1 is invalidated + select case when ( + select count(*) + from {{ ref('check_cols_cycle') }} + where id = 1 and color = 'red' and dbt_valid_to is not null + ) = 1 then 0 else 1 end as failures + + union all + + -- check that there's only one current record for id=2 + select case when ( + select count(*) + from {{ ref('check_cols_cycle') }} + where id = 2 and color = 'pink' and dbt_valid_to is null + ) = 1 then 0 else 1 end as failures + + union all + + -- check that the previous value for id=2 is represented + select case when ( + select count(*) + from {{ ref('check_cols_cycle') }} + where id = 2 and color = 'green' and dbt_valid_to is not null + ) = 1 then 0 else 1 end as failures + + union all + + -- check that there are 5 records total in the table + select case when ( + select count(*) + from {{ ref('check_cols_cycle') }} + ) = 5 then 0 else 1 end as failures + +) + +select * +from query +where failures = 1 diff --git a/tests/integration/simple_snapshot_test/check-snapshots/check_cols_cycle.sql b/tests/integration/simple_snapshot_test/check-snapshots/check_cols_cycle.sql new file mode 100644 index 000000000..8b36f35a1 --- /dev/null +++ b/tests/integration/simple_snapshot_test/check-snapshots/check_cols_cycle.sql @@ -0,0 +1,33 @@ + +{% snapshot check_cols_cycle %} + + {{ + config( + target_database=database, + target_schema=schema, + unique_key='id', + strategy='check', + check_cols=['color'] + ) + }} + + {% if var('version') == 1 %} + + select 1 as id, 'red' as color union all + select 2 as id, 'green' as color + + {% elif var('version') == 2 %} + + select 1 as id, 'blue' as color union all + select 2 as id, 'green' as color + + {% elif var('version') == 3 %} + + select 1 as id, 'red' as color union all + select 2 as id, 'pink' as color + + {% else %} + {% do exceptions.raise_compiler_error("Got bad version: " ~ var('version')) %} + {% endif %} + +{% endsnapshot %} diff --git a/tests/integration/simple_snapshot_test/custom-snapshot-macros/custom.sql b/tests/integration/simple_snapshot_test/custom-snapshot-macros/custom.sql new file mode 100644 index 000000000..4347088e4 --- /dev/null +++ b/tests/integration/simple_snapshot_test/custom-snapshot-macros/custom.sql @@ -0,0 +1,18 @@ +{# A "custom" strategy that's really just the timestamp one #} +{% macro snapshot_custom_strategy(node, snapshotted_rel, current_rel, config, target_exists) %} + {% set primary_key = config['unique_key'] %} + {% set updated_at = config['updated_at'] %} + + {% set row_changed_expr -%} + ({{ snapshotted_rel }}.{{ updated_at }} < {{ current_rel }}.{{ updated_at }}) + {%- endset %} + + {% set scd_id_expr = snapshot_hash_arguments([primary_key, updated_at]) %} + + {% do return({ + "unique_key": primary_key, + "updated_at": updated_at, + "row_changed": row_changed_expr, + "scd_id": scd_id_expr + }) %} +{% endmacro %} diff --git a/tests/integration/simple_snapshot_test/data/seed.csv b/tests/integration/simple_snapshot_test/data/seed.csv new file mode 100644 index 000000000..9da8d46ff --- /dev/null +++ b/tests/integration/simple_snapshot_test/data/seed.csv @@ -0,0 +1,4 @@ +id,first_name +1,Judith +2,Arthur +3,Rachel diff --git a/tests/integration/simple_snapshot_test/data/seed_newcol.csv b/tests/integration/simple_snapshot_test/data/seed_newcol.csv new file mode 100644 index 000000000..005517bda --- /dev/null +++ b/tests/integration/simple_snapshot_test/data/seed_newcol.csv @@ -0,0 +1,4 @@ +id,first_name,last_name +1,Judith,Kennedy +2,Arthur,Kelly +3,Rachel,Moreno diff --git a/tests/integration/simple_snapshot_test/invalidate_bigquery.sql b/tests/integration/simple_snapshot_test/invalidate_bigquery.sql new file mode 100644 index 000000000..d4641d451 --- /dev/null +++ b/tests/integration/simple_snapshot_test/invalidate_bigquery.sql @@ -0,0 +1,12 @@ + +-- update records 11 - 21. Change email and updated_at field +update {database}.{schema}.seed set + updated_at = timestamp_add(updated_at, interval 1 hour), + email = case when id = 20 then 'pfoxj@creativecommons.org' else concat('new_', email) end +where id >= 10 and id <= 20; + + +-- invalidate records 11 - 21 +update {database}.{schema}.snapshot_expected set + dbt_valid_to = timestamp_add(updated_at, interval 1 hour) +where id >= 10 and id <= 20; diff --git a/tests/integration/simple_snapshot_test/macros/test_no_overlaps.sql b/tests/integration/simple_snapshot_test/macros/test_no_overlaps.sql new file mode 100644 index 000000000..6d432193c --- /dev/null +++ b/tests/integration/simple_snapshot_test/macros/test_no_overlaps.sql @@ -0,0 +1,85 @@ +{% macro get_snapshot_unique_id() -%} + {{ return(adapter.dispatch('get_snapshot_unique_id')()) }} +{%- endmacro %} + +{% macro default__get_snapshot_unique_id() -%} + {% do return("id || '-' || first_name") %} +{%- endmacro %} + + +{% macro bigquery__get_snapshot_unique_id() -%} + {%- do return('concat(cast(id as string), "-", first_name)') -%} +{%- endmacro %} + +{# + mostly copy+pasted from dbt_utils, but I removed some parameters and added + a query that calls get_snapshot_unique_id +#} +{% test mutually_exclusive_ranges(model) %} + +with base as ( + select {{ get_snapshot_unique_id() }} as dbt_unique_id, + * + from {{ model }} +), +window_functions as ( + + select + dbt_valid_from as lower_bound, + coalesce(dbt_valid_to, '2099-1-1T00:00:01') as upper_bound, + + lead(dbt_valid_from) over ( + partition by dbt_unique_id + order by dbt_valid_from + ) as next_lower_bound, + + row_number() over ( + partition by dbt_unique_id + order by dbt_valid_from desc + ) = 1 as is_last_record + + from base + +), + +calc as ( + -- We want to return records where one of our assumptions fails, so we'll use + -- the `not` function with `and` statements so we can write our assumptions nore cleanly + select + *, + + -- For each record: lower_bound should be < upper_bound. + -- Coalesce it to return an error on the null case (implicit assumption + -- these columns are not_null) + coalesce( + lower_bound < upper_bound, + is_last_record + ) as lower_bound_less_than_upper_bound, + + -- For each record: upper_bound {{ allow_gaps_operator }} the next lower_bound. + -- Coalesce it to handle null cases for the last record. + coalesce( + upper_bound = next_lower_bound, + is_last_record, + false + ) as upper_bound_equal_to_next_lower_bound + + from window_functions + +), + +validation_errors as ( + + select + * + from calc + + where not( + -- THE FOLLOWING SHOULD BE TRUE -- + lower_bound_less_than_upper_bound + and upper_bound_equal_to_next_lower_bound + ) +) + +select * from validation_errors +{% endtest %} diff --git a/tests/integration/simple_snapshot_test/models/.gitkeep b/tests/integration/simple_snapshot_test/models/.gitkeep new file mode 100644 index 000000000..e69de29bb diff --git a/tests/integration/simple_snapshot_test/models/ref_snapshot.sql b/tests/integration/simple_snapshot_test/models/ref_snapshot.sql new file mode 100644 index 000000000..c453929ce --- /dev/null +++ b/tests/integration/simple_snapshot_test/models/ref_snapshot.sql @@ -0,0 +1 @@ +select * from {{ ref('snapshot_actual') }} diff --git a/tests/integration/simple_snapshot_test/models/schema.yml b/tests/integration/simple_snapshot_test/models/schema.yml new file mode 100644 index 000000000..259e55b95 --- /dev/null +++ b/tests/integration/simple_snapshot_test/models/schema.yml @@ -0,0 +1,5 @@ +version: 2 +snapshots: + - name: snapshot_actual + tests: + - mutually_exclusive_ranges diff --git a/tests/integration/simple_snapshot_test/seed.sql b/tests/integration/simple_snapshot_test/seed.sql new file mode 100644 index 000000000..8f3422e36 --- /dev/null +++ b/tests/integration/simple_snapshot_test/seed.sql @@ -0,0 +1,220 @@ +create table {database}.{schema}.seed ( + id INTEGER, + first_name VARCHAR(50), + last_name VARCHAR(50), + email VARCHAR(50), + gender VARCHAR(50), + ip_address VARCHAR(20), + updated_at TIMESTAMP WITHOUT TIME ZONE +); + +create table {database}.{schema}.snapshot_expected ( + id INTEGER, + first_name VARCHAR(50), + last_name VARCHAR(50), + email VARCHAR(50), + gender VARCHAR(50), + ip_address VARCHAR(20), + + -- snapshotting fields + updated_at TIMESTAMP WITHOUT TIME ZONE, + dbt_valid_from TIMESTAMP WITHOUT TIME ZONE, + dbt_valid_to TIMESTAMP WITHOUT TIME ZONE, + dbt_scd_id VARCHAR(32), + dbt_updated_at TIMESTAMP WITHOUT TIME ZONE +); + + +-- seed inserts +insert into {database}.{schema}.seed (id, first_name, last_name, email, gender, ip_address, updated_at) values +(1, 'Judith', 'Kennedy', 'jkennedy0@phpbb.com', 'Female', '54.60.24.128', '2015-12-24 12:19:28'), +(2, 'Arthur', 'Kelly', 'akelly1@eepurl.com', 'Male', '62.56.24.215', '2015-10-28 16:22:15'), +(3, 'Rachel', 'Moreno', 'rmoreno2@msu.edu', 'Female', '31.222.249.23', '2016-04-05 02:05:30'), +(4, 'Ralph', 'Turner', 'rturner3@hp.com', 'Male', '157.83.76.114', '2016-08-08 00:06:51'), +(5, 'Laura', 'Gonzales', 'lgonzales4@howstuffworks.com', 'Female', '30.54.105.168', '2016-09-01 08:25:38'), +(6, 'Katherine', 'Lopez', 'klopez5@yahoo.co.jp', 'Female', '169.138.46.89', '2016-08-30 18:52:11'), +(7, 'Jeremy', 'Hamilton', 'jhamilton6@mozilla.org', 'Male', '231.189.13.133', '2016-07-17 02:09:46'), +(8, 'Heather', 'Rose', 'hrose7@goodreads.com', 'Female', '87.165.201.65', '2015-12-29 22:03:56'), +(9, 'Gregory', 'Kelly', 'gkelly8@trellian.com', 'Male', '154.209.99.7', '2016-03-24 21:18:16'), +(10, 'Rachel', 'Lopez', 'rlopez9@themeforest.net', 'Female', '237.165.82.71', '2016-08-20 15:44:49'), +(11, 'Donna', 'Welch', 'dwelcha@shutterfly.com', 'Female', '103.33.110.138', '2016-02-27 01:41:48'), +(12, 'Russell', 'Lawrence', 'rlawrenceb@qq.com', 'Male', '189.115.73.4', '2016-06-11 03:07:09'), +(13, 'Michelle', 'Montgomery', 'mmontgomeryc@scientificamerican.com', 'Female', '243.220.95.82', '2016-06-18 16:27:19'), +(14, 'Walter', 'Castillo', 'wcastillod@pagesperso-orange.fr', 'Male', '71.159.238.196', '2016-10-06 01:55:44'), +(15, 'Robin', 'Mills', 'rmillse@vkontakte.ru', 'Female', '172.190.5.50', '2016-10-31 11:41:21'), +(16, 'Raymond', 'Holmes', 'rholmesf@usgs.gov', 'Male', '148.153.166.95', '2016-10-03 08:16:38'), +(17, 'Gary', 'Bishop', 'gbishopg@plala.or.jp', 'Male', '161.108.182.13', '2016-08-29 19:35:20'), +(18, 'Anna', 'Riley', 'arileyh@nasa.gov', 'Female', '253.31.108.22', '2015-12-11 04:34:27'), +(19, 'Sarah', 'Knight', 'sknighti@foxnews.com', 'Female', '222.220.3.177', '2016-09-26 00:49:06'), +(20, 'Phyllis', 'Fox', null, 'Female', '163.191.232.95', '2016-08-21 10:35:19'); + + +-- populate snapshot table +insert into {database}.{schema}.snapshot_expected ( + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + dbt_valid_from, + dbt_valid_to, + dbt_updated_at, + dbt_scd_id +) + +select + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + -- fields added by snapshotting + updated_at as dbt_valid_from, + null::timestamp as dbt_valid_to, + updated_at as dbt_updated_at, + md5(id || '-' || first_name || '|' || updated_at::text) as dbt_scd_id +from {database}.{schema}.seed; + +create table {database}.{schema}.snapshot_castillo_expected ( + id INTEGER, + first_name VARCHAR(50), + last_name VARCHAR(50), + email VARCHAR(50), + gender VARCHAR(50), + ip_address VARCHAR(20), + + -- snapshotting fields + "1-updated_at" TIMESTAMP WITHOUT TIME ZONE, + dbt_valid_from TIMESTAMP WITHOUT TIME ZONE, + dbt_valid_to TIMESTAMP WITHOUT TIME ZONE, + dbt_scd_id VARCHAR(32), + dbt_updated_at TIMESTAMP WITHOUT TIME ZONE + +); + +-- one entry +insert into {database}.{schema}.snapshot_castillo_expected ( + id, + first_name, + last_name, + email, + gender, + ip_address, + "1-updated_at", + dbt_valid_from, + dbt_valid_to, + dbt_updated_at, + dbt_scd_id +) + +select + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + -- fields added by snapshotting + updated_at as dbt_valid_from, + null::timestamp as dbt_valid_to, + updated_at as dbt_updated_at, + md5(id || '-' || first_name || '|' || updated_at::text) as dbt_scd_id +from {database}.{schema}.seed where last_name = 'Castillo'; + +create table {database}.{schema}.snapshot_alvarez_expected ( + id INTEGER, + first_name VARCHAR(50), + last_name VARCHAR(50), + email VARCHAR(50), + gender VARCHAR(50), + ip_address VARCHAR(20), + + -- snapshotting fields + updated_at TIMESTAMP WITHOUT TIME ZONE, + dbt_valid_from TIMESTAMP WITHOUT TIME ZONE, + dbt_valid_to TIMESTAMP WITHOUT TIME ZONE, + dbt_scd_id VARCHAR(32), + dbt_updated_at TIMESTAMP WITHOUT TIME ZONE +); + +-- 0 entries +insert into {database}.{schema}.snapshot_alvarez_expected ( + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + dbt_valid_from, + dbt_valid_to, + dbt_updated_at, + dbt_scd_id +) + +select + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + -- fields added by snapshotting + updated_at as dbt_valid_from, + null::timestamp as dbt_valid_to, + updated_at as dbt_updated_at, + md5(id || '-' || first_name || '|' || updated_at::text) as dbt_scd_id +from {database}.{schema}.seed where last_name = 'Alvarez'; + +create table {database}.{schema}.snapshot_kelly_expected ( + id INTEGER, + first_name VARCHAR(50), + last_name VARCHAR(50), + email VARCHAR(50), + gender VARCHAR(50), + ip_address VARCHAR(20), + + -- snapshotting fields + updated_at TIMESTAMP WITHOUT TIME ZONE, + dbt_valid_from TIMESTAMP WITHOUT TIME ZONE, + dbt_valid_to TIMESTAMP WITHOUT TIME ZONE, + dbt_scd_id VARCHAR(32), + dbt_updated_at TIMESTAMP WITHOUT TIME ZONE +); + + +-- 2 entries +insert into {database}.{schema}.snapshot_kelly_expected ( + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + dbt_valid_from, + dbt_valid_to, + dbt_updated_at, + dbt_scd_id +) + +select + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + -- fields added by snapshotting + updated_at as dbt_valid_from, + null::timestamp as dbt_valid_to, + updated_at as dbt_updated_at, + md5(id || '-' || first_name || '|' || updated_at::text) as dbt_scd_id +from {database}.{schema}.seed where last_name = 'Kelly'; diff --git a/tests/integration/simple_snapshot_test/seed_bq.sql b/tests/integration/simple_snapshot_test/seed_bq.sql new file mode 100644 index 000000000..5ea93fee4 --- /dev/null +++ b/tests/integration/simple_snapshot_test/seed_bq.sql @@ -0,0 +1,81 @@ +create table {database}.{schema}.seed ( + id INT64, + first_name STRING, + last_name STRING, + email STRING, + gender STRING, + ip_address STRING, + updated_at TIMESTAMP +); + +create table {database}.{schema}.snapshot_expected ( + id INT64, + first_name STRING, + last_name STRING, + email STRING, + gender STRING, + ip_address STRING, + + -- snapshotting fields + updated_at TIMESTAMP, + dbt_valid_from TIMESTAMP, + dbt_valid_to TIMESTAMP, + dbt_scd_id STRING, + dbt_updated_at TIMESTAMP +); + + +-- seed inserts +insert {database}.{schema}.seed (id, first_name, last_name, email, gender, ip_address, updated_at) values +(1, 'Judith', 'Kennedy', 'jkennedy0@phpbb.com', 'Female', '54.60.24.128', '2015-12-24 12:19:28'), +(2, 'Arthur', 'Kelly', 'akelly1@eepurl.com', 'Male', '62.56.24.215', '2015-10-28 16:22:15'), +(3, 'Rachel', 'Moreno', 'rmoreno2@msu.edu', 'Female', '31.222.249.23', '2016-04-05 02:05:30'), +(4, 'Ralph', 'Turner', 'rturner3@hp.com', 'Male', '157.83.76.114', '2016-08-08 00:06:51'), +(5, 'Laura', 'Gonzales', 'lgonzales4@howstuffworks.com', 'Female', '30.54.105.168', '2016-09-01 08:25:38'), +(6, 'Katherine', 'Lopez', 'klopez5@yahoo.co.jp', 'Female', '169.138.46.89', '2016-08-30 18:52:11'), +(7, 'Jeremy', 'Hamilton', 'jhamilton6@mozilla.org', 'Male', '231.189.13.133', '2016-07-17 02:09:46'), +(8, 'Heather', 'Rose', 'hrose7@goodreads.com', 'Female', '87.165.201.65', '2015-12-29 22:03:56'), +(9, 'Gregory', 'Kelly', 'gkelly8@trellian.com', 'Male', '154.209.99.7', '2016-03-24 21:18:16'), +(10, 'Rachel', 'Lopez', 'rlopez9@themeforest.net', 'Female', '237.165.82.71', '2016-08-20 15:44:49'), +(11, 'Donna', 'Welch', 'dwelcha@shutterfly.com', 'Female', '103.33.110.138', '2016-02-27 01:41:48'), +(12, 'Russell', 'Lawrence', 'rlawrenceb@qq.com', 'Male', '189.115.73.4', '2016-06-11 03:07:09'), +(13, 'Michelle', 'Montgomery', 'mmontgomeryc@scientificamerican.com', 'Female', '243.220.95.82', '2016-06-18 16:27:19'), +(14, 'Walter', 'Castillo', 'wcastillod@pagesperso-orange.fr', 'Male', '71.159.238.196', '2016-10-06 01:55:44'), +(15, 'Robin', 'Mills', 'rmillse@vkontakte.ru', 'Female', '172.190.5.50', '2016-10-31 11:41:21'), +(16, 'Raymond', 'Holmes', 'rholmesf@usgs.gov', 'Male', '148.153.166.95', '2016-10-03 08:16:38'), +(17, 'Gary', 'Bishop', 'gbishopg@plala.or.jp', 'Male', '161.108.182.13', '2016-08-29 19:35:20'), +(18, 'Anna', 'Riley', 'arileyh@nasa.gov', 'Female', '253.31.108.22', '2015-12-11 04:34:27'), +(19, 'Sarah', 'Knight', 'sknighti@foxnews.com', 'Female', '222.220.3.177', '2016-09-26 00:49:06'), +(20, 'Phyllis', 'Fox', null, 'Female', '163.191.232.95', '2016-08-21 10:35:19'); + + +-- populate snapshot table +insert {database}.{schema}.snapshot_expected ( + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + dbt_valid_from, + dbt_valid_to, + dbt_updated_at, + dbt_scd_id +) + +select + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + -- fields added by snapshotting + updated_at as dbt_valid_from, + cast(null as timestamp) as dbt_valid_to, + updated_at as dbt_updated_at, + to_hex(md5(concat(cast(id as string), '-', first_name, '|', cast(updated_at as string)))) as dbt_scd_id +from {database}.{schema}.seed; + diff --git a/tests/integration/simple_snapshot_test/test-check-col-snapshots-bq/snapshot.sql b/tests/integration/simple_snapshot_test/test-check-col-snapshots-bq/snapshot.sql new file mode 100644 index 000000000..9c8459756 --- /dev/null +++ b/tests/integration/simple_snapshot_test/test-check-col-snapshots-bq/snapshot.sql @@ -0,0 +1,29 @@ +{% snapshot snapshot_actual %} + {# this used to be check_cols=('email',), which ought to be totally valid, + but is not because type systems are hard. #} + {{ + config( + target_project=var('target_database', database), + target_dataset=var('target_schema', schema), + unique_key='concat(cast(id as string) , "-", first_name)', + strategy='check', + check_cols=['email'], + ) + }} + select * from `{{target.database}}`.`{{schema}}`.seed +{% endsnapshot %} + + +{# This should be exactly the same #} +{% snapshot snapshot_checkall %} + {{ + config( + target_project=var('target_database', database), + target_dataset=var('target_schema', schema), + unique_key='concat(cast(id as string) , "-", first_name)', + strategy='check', + check_cols='all', + ) + }} + select * from `{{target.database}}`.`{{schema}}`.seed +{% endsnapshot %} diff --git a/tests/integration/simple_snapshot_test/test-check-col-snapshots-noconfig/snapshot.sql b/tests/integration/simple_snapshot_test/test-check-col-snapshots-noconfig/snapshot.sql new file mode 100644 index 000000000..daf4cf312 --- /dev/null +++ b/tests/integration/simple_snapshot_test/test-check-col-snapshots-noconfig/snapshot.sql @@ -0,0 +1,9 @@ +{% snapshot snapshot_actual %} + select * from {{target.database}}.{{schema}}.seed +{% endsnapshot %} + +{# This should be exactly the same #} +{% snapshot snapshot_checkall %} + {{ config(check_cols='all') }} + select * from {{target.database}}.{{schema}}.seed +{% endsnapshot %} diff --git a/tests/integration/simple_snapshot_test/test-check-col-snapshots/snapshot.sql b/tests/integration/simple_snapshot_test/test-check-col-snapshots/snapshot.sql new file mode 100644 index 000000000..dd85ed753 --- /dev/null +++ b/tests/integration/simple_snapshot_test/test-check-col-snapshots/snapshot.sql @@ -0,0 +1,28 @@ +{% snapshot snapshot_actual %} + + {{ + config( + target_database=var('target_database', database), + target_schema=schema, + unique_key='id || ' ~ "'-'" ~ ' || first_name', + strategy='check', + check_cols=['email'], + ) + }} + select * from {{target.database}}.{{schema}}.seed + +{% endsnapshot %} + +{# This should be exactly the same #} +{% snapshot snapshot_checkall %} + {{ + config( + target_database=var('target_database', database), + target_schema=schema, + unique_key='id || ' ~ "'-'" ~ ' || first_name', + strategy='check', + check_cols='all', + ) + }} + select * from {{target.database}}.{{schema}}.seed +{% endsnapshot %} diff --git a/tests/integration/simple_snapshot_test/test-snapshots-bq/snapshot.sql b/tests/integration/simple_snapshot_test/test-snapshots-bq/snapshot.sql new file mode 100644 index 000000000..7ffdedbcc --- /dev/null +++ b/tests/integration/simple_snapshot_test/test-snapshots-bq/snapshot.sql @@ -0,0 +1,19 @@ +{% snapshot snapshot_actual %} + + {{ + config( + target_project=var('target_database', database), + target_dataset=var('target_schema', schema), + unique_key='concat(cast(id as string) , "-", first_name)', + strategy='timestamp', + updated_at='updated_at', + ) + }} + + {% if var('invalidate_hard_deletes', 'false') | as_bool %} + {{ config(invalidate_hard_deletes=True) }} + {% endif %} + + select * from `{{target.database}}`.`{{schema}}`.seed + +{% endsnapshot %} diff --git a/tests/integration/simple_snapshot_test/test-snapshots-checkall/snapshot.sql b/tests/integration/simple_snapshot_test/test-snapshots-checkall/snapshot.sql new file mode 100644 index 000000000..b9cd002ca --- /dev/null +++ b/tests/integration/simple_snapshot_test/test-snapshots-checkall/snapshot.sql @@ -0,0 +1,4 @@ +{% snapshot my_snapshot %} + {{ config(check_cols='all', unique_key='id', strategy='check', target_database=database, target_schema=schema) }} + select * from {{ ref(var('seed_name', 'seed')) }} +{% endsnapshot %} diff --git a/tests/integration/simple_snapshot_test/test-snapshots-pg/snapshot.sql b/tests/integration/simple_snapshot_test/test-snapshots-pg/snapshot.sql new file mode 100644 index 000000000..ae5aac087 --- /dev/null +++ b/tests/integration/simple_snapshot_test/test-snapshots-pg/snapshot.sql @@ -0,0 +1,19 @@ +{% snapshot snapshot_actual %} + + {{ + config( + target_database=var('target_database', database), + target_schema=var('target_schema', schema), + unique_key='id || ' ~ "'-'" ~ ' || first_name', + strategy='timestamp', + updated_at='updated_at', + ) + }} + + {% if var('invalidate_hard_deletes', 'false') | as_bool %} + {{ config(invalidate_hard_deletes=True) }} + {% endif %} + + select * from {{target.database}}.{{target.schema}}.seed + +{% endsnapshot %} diff --git a/tests/integration/simple_snapshot_test/test-snapshots-select-noconfig/snapshot.sql b/tests/integration/simple_snapshot_test/test-snapshots-select-noconfig/snapshot.sql new file mode 100644 index 000000000..a62218b2c --- /dev/null +++ b/tests/integration/simple_snapshot_test/test-snapshots-select-noconfig/snapshot.sql @@ -0,0 +1,41 @@ +{% snapshot snapshot_actual %} + + {{ + config( + target_database=var('target_database', database), + target_schema=var('target_schema', schema), + ) + }} + select * from {{target.database}}.{{target.schema}}.seed + +{% endsnapshot %} + +{% snapshot snapshot_castillo %} + + {{ + config( + target_database=var('target_database', database), + updated_at='"1-updated_at"', + ) + }} + select id,first_name,last_name,email,gender,ip_address,updated_at as "1-updated_at" from {{target.database}}.{{schema}}.seed where last_name = 'Castillo' + +{% endsnapshot %} + +{% snapshot snapshot_alvarez %} + + {{ + config( + target_database=var('target_database', database), + ) + }} + select * from {{target.database}}.{{schema}}.seed where last_name = 'Alvarez' + +{% endsnapshot %} + + +{% snapshot snapshot_kelly %} + {# This has no target_database set, which is allowed! #} + select * from {{target.database}}.{{schema}}.seed where last_name = 'Kelly' + +{% endsnapshot %} diff --git a/tests/integration/simple_snapshot_test/test-snapshots-select/snapshot.sql b/tests/integration/simple_snapshot_test/test-snapshots-select/snapshot.sql new file mode 100644 index 000000000..06245f36f --- /dev/null +++ b/tests/integration/simple_snapshot_test/test-snapshots-select/snapshot.sql @@ -0,0 +1,44 @@ +{% snapshot snapshot_castillo %} + + {{ + config( + target_database=var('target_database', database), + target_schema=schema, + unique_key='id || ' ~ "'-'" ~ ' || first_name', + strategy='timestamp', + updated_at='"1-updated_at"', + ) + }} + select id,first_name,last_name,email,gender,ip_address,updated_at as "1-updated_at" from {{target.database}}.{{schema}}.seed where last_name = 'Castillo' + +{% endsnapshot %} + +{% snapshot snapshot_alvarez %} + + {{ + config( + target_database=var('target_database', database), + target_schema=schema, + unique_key='id || ' ~ "'-'" ~ ' || first_name', + strategy='timestamp', + updated_at='updated_at', + ) + }} + select * from {{target.database}}.{{schema}}.seed where last_name = 'Alvarez' + +{% endsnapshot %} + + +{% snapshot snapshot_kelly %} + {# This has no target_database set, which is allowed! #} + {{ + config( + target_schema=schema, + unique_key='id || ' ~ "'-'" ~ ' || first_name', + strategy='timestamp', + updated_at='updated_at', + ) + }} + select * from {{target.database}}.{{schema}}.seed where last_name = 'Kelly' + +{% endsnapshot %} diff --git a/tests/integration/simple_snapshot_test/test_simple_snapshot.py b/tests/integration/simple_snapshot_test/test_simple_snapshot.py new file mode 100644 index 000000000..5db2d02e6 --- /dev/null +++ b/tests/integration/simple_snapshot_test/test_simple_snapshot.py @@ -0,0 +1,566 @@ +from tests.integration.base import DBTIntegrationTest, use_profile +from datetime import datetime +import pytz +import dbt.exceptions + + +class BaseSimpleSnapshotTest(DBTIntegrationTest): + NUM_SNAPSHOT_MODELS = 1 + + @property + def schema(self): + return "simple_snapshot" + + @property + def models(self): + return "models" + + def run_snapshot(self): + return self.run_dbt(['snapshot']) + + def dbt_run_seed_snapshot(self): + self.run_sql_file('seed.sql') + + results = self.run_snapshot() + self.assertEqual(len(results), self.NUM_SNAPSHOT_MODELS) + + def assert_case_tables_equal(self, actual, expected): + # this does something different on snowflake, but here it's just assertTablesEqual + self.assertTablesEqual(actual, expected) + + def assert_expected(self): + self.run_dbt(['test']) + self.assert_case_tables_equal('snapshot_actual', 'snapshot_expected') + + +class TestSimpleSnapshotFiles(BaseSimpleSnapshotTest): + @property + def project_config(self): + return { + 'config-version': 2, + "data-paths": ['data'], + "snapshot-paths": ['test-snapshots-pg'], + 'macro-paths': ['macros'], + } + + +class TestSimpleColumnSnapshotFiles(DBTIntegrationTest): + + @property + def schema(self): + return "simple_snapshot" + + @property + def models(self): + return "models-checkall" + + @property + def project_config(self): + return { + 'config-version': 2, + 'data-paths': ['data'], + 'macro-paths': ['custom-snapshot-macros', 'macros'], + 'snapshot-paths': ['test-snapshots-checkall'], + 'seeds': { + 'quote_columns': False, + } + } + + def _run_snapshot_test(self): + self.run_dbt(['seed']) + self.run_dbt(['snapshot']) + database = self.default_database + database = self.adapter.quote(database) + results = self.run_sql( + 'select * from {}.{}.my_snapshot'.format(database, self.unique_schema()), + fetch='all' + ) + self.assertEqual(len(results), 3) + for result in results: + self.assertEqual(len(result), 6) + + self.run_dbt(['snapshot', '--vars', '{seed_name: seed_newcol}']) + results = self.run_sql( + 'select * from {}.{}.my_snapshot where last_name is not NULL'.format(database, self.unique_schema()), + fetch='all' + ) + self.assertEqual(len(results), 3) + + for result in results: + # new column + self.assertEqual(len(result), 7) + self.assertIsNotNone(result[-1]) + + results = self.run_sql( + 'select * from {}.{}.my_snapshot where last_name is NULL'.format(database, self.unique_schema()), + fetch='all' + ) + self.assertEqual(len(results), 3) + for result in results: + # new column + self.assertEqual(len(result), 7) + + @use_profile('bigquery') + def test_bigquery_renamed_source(self): + self._run_snapshot_test() + + +class TestSimpleSnapshotFileSelects(DBTIntegrationTest): + @property + def schema(self): + return "simple_snapshot" + + @property + def models(self): + return "models" + + @property + def project_config(self): + return { + 'config-version': 2, + "data-paths": ['data'], + "snapshot-paths": ['test-snapshots-select', + 'test-snapshots-pg'], + 'macro-paths': ['macros'], + } + + +class TestConfiguredSnapshotFileSelects(TestSimpleSnapshotFileSelects): + @property + def project_config(self): + return { + 'config-version': 2, + "data-paths": ['data'], + "snapshot-paths": ['test-snapshots-select-noconfig'], + "snapshots": { + "test": { + "target_schema": self.unique_schema(), + "unique_key": "id || '-' || first_name", + 'strategy': 'timestamp', + 'updated_at': 'updated_at', + }, + }, + 'macro-paths': ['macros'], + } + + +class TestSimpleSnapshotFilesBigquery(DBTIntegrationTest): + @property + def schema(self): + return "simple_snapshot" + + @property + def models(self): + return "models" + + @property + def project_config(self): + return { + 'config-version': 2, + "snapshot-paths": ['test-snapshots-bq'], + 'macro-paths': ['macros'], + } + + def assert_expected(self): + self.run_dbt(['test']) + self.assertTablesEqual('snapshot_actual', 'snapshot_expected') + + @use_profile('bigquery') + def test__bigquery__simple_snapshot(self): + self.run_sql_file("seed_bq.sql") + + self.run_dbt(["snapshot"]) + + self.assert_expected() + + self.run_sql_file("invalidate_bigquery.sql") + self.run_sql_file("update_bq.sql") + + self.run_dbt(["snapshot"]) + + self.assert_expected() + + @use_profile('bigquery') + def test__bigquery__snapshot_with_new_field(self): + + self.run_sql_file("seed_bq.sql") + + self.run_dbt(["snapshot"]) + + self.assertTablesEqual("snapshot_expected", "snapshot_actual") + + self.run_sql_file("invalidate_bigquery.sql") + self.run_sql_file("update_bq.sql") + + # This adds new fields to the source table, and updates the expected snapshot output accordingly + self.run_sql_file("add_column_to_source_bq.sql") + + self.run_dbt(["snapshot"]) + + # A more thorough test would assert that snapshotted == expected, but BigQuery does not support the + # "EXCEPT DISTINCT" operator on nested fields! Instead, just check that schemas are congruent. + + expected_cols = self.get_table_columns( + database=self.default_database, + schema=self.unique_schema(), + table='snapshot_expected' + ) + snapshotted_cols = self.get_table_columns( + database=self.default_database, + schema=self.unique_schema(), + table='snapshot_actual' + ) + + self.assertTrue(len(expected_cols) > 0, "source table does not exist -- bad test") + self.assertEqual(len(expected_cols), len(snapshotted_cols), "actual and expected column lengths are different") + + for (expected_col, actual_col) in zip(expected_cols, snapshotted_cols): + expected_name, expected_type, _ = expected_col + actual_name, actual_type, _ = actual_col + self.assertTrue(expected_name is not None) + self.assertTrue(expected_type is not None) + + self.assertEqual(expected_name, actual_name, "names are different") + self.assertEqual(expected_type, actual_type, "data types are different") + + +class TestCrossDBSnapshotFiles(DBTIntegrationTest): + setup_alternate_db = True + + @property + def schema(self): + return "simple_snapshot" + + @property + def models(self): + return "models" + + @property + def project_config(self): + paths = ['test-snapshots-bq'] + return { + 'config-version': 2, + 'snapshot-paths': paths, + 'macro-paths': ['macros'], + } + + def run_snapshot(self): + return self.run_dbt(['snapshot', '--vars', '{{"target_database": {}}}'.format(self.alternative_database)]) + + @use_profile('bigquery') + def test__bigquery__cross_snapshot(self): + self.run_sql_file("seed_bq.sql") + + self.run_snapshot() + + self.assertTablesEqual("snapshot_expected", "snapshot_actual", table_b_db=self.alternative_database) + + self.run_sql_file("invalidate_bigquery.sql") + self.run_sql_file("update_bq.sql") + + self.run_snapshot() + + self.assertTablesEqual("snapshot_expected", "snapshot_actual", table_b_db=self.alternative_database) + + +class TestCheckCols(TestSimpleSnapshotFiles): + NUM_SNAPSHOT_MODELS = 2 + + def _assertTablesEqualSql(self, relation_a, relation_b, columns=None): + # When building the equality tests, only test columns that don't start + # with 'dbt_', because those are time-sensitive + if columns is None: + columns = [c for c in self.get_relation_columns(relation_a) if not c[0].lower().startswith('dbt_')] + return super()._assertTablesEqualSql(relation_a, relation_b, columns=columns) + + def assert_expected(self): + super().assert_expected() + self.assert_case_tables_equal('snapshot_checkall', 'snapshot_expected') + + @property + def project_config(self): + return { + 'config-version': 2, + "data-paths": ['data'], + "snapshot-paths": ['test-check-col-snapshots'], + 'macro-paths': ['macros'], + } + + +class TestConfiguredCheckCols(TestCheckCols): + @property + def project_config(self): + return { + 'config-version': 2, + "data-paths": ['data'], + "snapshot-paths": ['test-check-col-snapshots-noconfig'], + "snapshots": { + "test": { + "target_schema": self.unique_schema(), + "unique_key": "id || '-' || first_name", + "strategy": "check", + "check_cols": ["email"], + }, + }, + 'macro-paths': ['macros'], + } + + +class TestUpdatedAtCheckCols(TestCheckCols): + def _assertTablesEqualSql(self, relation_a, relation_b, columns=None): + revived_records = self.run_sql( + ''' + select + id, + updated_at, + dbt_valid_from + from {} + '''.format(relation_b), + fetch='all' + ) + + for result in revived_records: + # result is a tuple, the updated_at is second and dbt_valid_from is latest + self.assertIsInstance(result[1], datetime) + self.assertIsInstance(result[2], datetime) + self.assertEqual(result[1].replace(tzinfo=pytz.UTC), result[2].replace(tzinfo=pytz.UTC)) + + if columns is None: + columns = [c for c in self.get_relation_columns(relation_a) if not c[0].lower().startswith('dbt_')] + return super()._assertTablesEqualSql(relation_a, relation_b, columns=columns) + + def assert_expected(self): + super().assert_expected() + self.assertTablesEqual('snapshot_checkall', 'snapshot_expected') + + + @property + def project_config(self): + return { + 'config-version': 2, + "data-paths": ['data'], + "snapshot-paths": ['test-check-col-snapshots-noconfig'], + "snapshots": { + "test": { + "target_schema": self.unique_schema(), + "unique_key": "id || '-' || first_name", + "strategy": "check", + "check_cols" : "all", + "updated_at": "updated_at", + }, + }, + 'macro-paths': ['macros'], + } + + +class TestCheckColsBigquery(TestSimpleSnapshotFilesBigquery): + def _assertTablesEqualSql(self, relation_a, relation_b, columns=None): + # When building the equality tests, only test columns that don't start + # with 'dbt_', because those are time-sensitive + if columns is None: + columns = [c for c in self.get_relation_columns(relation_a) if not c[0].lower().startswith('dbt_')] + return super()._assertTablesEqualSql(relation_a, relation_b, columns=columns) + + def assert_expected(self): + super().assert_expected() + self.assertTablesEqual('snapshot_checkall', 'snapshot_expected') + + @property + def project_config(self): + return { + 'config-version': 2, + "data-paths": ['data'], + "snapshot-paths": ['test-check-col-snapshots-bq'], + 'macro-paths': ['macros'], + } + + @use_profile('bigquery') + def test__bigquery__snapshot_with_new_field(self): + self.use_default_project() + self.use_profile('bigquery') + + self.run_sql_file("seed_bq.sql") + + self.run_dbt(["snapshot"]) + + self.assertTablesEqual("snapshot_expected", "snapshot_actual") + self.assertTablesEqual("snapshot_expected", "snapshot_checkall") + + self.run_sql_file("invalidate_bigquery.sql") + self.run_sql_file("update_bq.sql") + + # This adds new fields to the source table, and updates the expected snapshot output accordingly + self.run_sql_file("add_column_to_source_bq.sql") + + # check_cols='all' will replace the changed field + self.run_dbt(['snapshot']) + + # A more thorough test would assert that snapshotted == expected, but BigQuery does not support the + # "EXCEPT DISTINCT" operator on nested fields! Instead, just check that schemas are congruent. + + expected_cols = self.get_table_columns( + database=self.default_database, + schema=self.unique_schema(), + table='snapshot_expected' + ) + snapshotted_cols = self.get_table_columns( + database=self.default_database, + schema=self.unique_schema(), + table='snapshot_actual' + ) + snapshotted_all_cols = self.get_table_columns( + database=self.default_database, + schema=self.unique_schema(), + table='snapshot_checkall' + ) + + self.assertTrue(len(expected_cols) > 0, "source table does not exist -- bad test") + self.assertEqual(len(expected_cols), len(snapshotted_cols), "actual and expected column lengths are different") + self.assertEqual(len(expected_cols), len(snapshotted_all_cols)) + + for (expected_col, actual_col) in zip(expected_cols, snapshotted_cols): + expected_name, expected_type, _ = expected_col + actual_name, actual_type, _ = actual_col + self.assertTrue(expected_name is not None) + self.assertTrue(expected_type is not None) + + self.assertEqual(expected_name, actual_name, "names are different") + self.assertEqual(expected_type, actual_type, "data types are different") + + +class TestSnapshotHardDelete(DBTIntegrationTest): + # These tests uses the same seed data, containing 20 records of which we hard delete the last 10. + # These deleted records set the dbt_valid_to to time the snapshot was ran. + NUM_SNAPSHOT_MODELS = 1 + + @property + def schema(self): + return "simple_snapshot" + + @property + def models(self): + return "models" + + @property + def project_config(self): + paths = ['test-snapshots-bq'] + + return { + 'config-version': 2, + "data-paths": ['data'], + "snapshot-paths": paths, + 'macro-paths': ['macros'], + } + + @use_profile('bigquery') + def test__bigquery__snapshot_hard_delete(self): + self.run_sql_file('seed_bq.sql') + self._test_snapshot_hard_delete() + + def _test_snapshot_hard_delete(self): + self._snapshot() + + self.assertTablesEqual("snapshot_expected", "snapshot_actual") + + self._invalidated_snapshot_datetime = None + self._revived_snapshot_datetime = None + + self._delete_records() + self._snapshot_and_assert_invalidated() + self._revive_records() + self._snapshot_and_assert_revived() + + def _snapshot(self): + begin_snapshot_datetime = datetime.now(pytz.UTC) + results = self.run_dbt(['snapshot', '--vars', '{invalidate_hard_deletes: true}']) + self.assertEqual(len(results), self.NUM_SNAPSHOT_MODELS) + + return begin_snapshot_datetime + + def _delete_records(self): + database = self.adapter.quote(database) + + self.run_sql( + 'delete from {}.{}.seed where id >= 10;'.format(database, self.unique_schema()) + ) + + def _snapshot_and_assert_invalidated(self): + self._invalidated_snapshot_datetime = self._snapshot() + + database = self.adapter.quote(database) + + snapshotted = self.run_sql( + ''' + select + id, + dbt_valid_to + from {}.{}.snapshot_actual + order by id + '''.format(database, self.unique_schema()), + fetch='all' + ) + + self.assertEqual(len(snapshotted), 20) + for result in snapshotted[10:]: + # result is a tuple, the dbt_valid_to column is the latest + self.assertIsInstance(result[-1], datetime) + self.assertGreaterEqual(result[-1].astimezone(pytz.UTC), self._invalidated_snapshot_datetime) + + def _revive_records(self): + database = self.adapter.quote(database) + + revival_timestamp = datetime.now(pytz.UTC).strftime(r'%Y-%m-%d %H:%M:%S') + self.run_sql( + ''' + insert into {}.{}.seed (id, first_name, last_name, email, gender, ip_address, updated_at) values + (10, 'Rachel', 'Lopez', 'rlopez9@themeforest.net', 'Female', '237.165.82.71', '{}'), + (11, 'Donna', 'Welch', 'dwelcha@shutterfly.com', 'Female', '103.33.110.138', '{}') + '''.format(database, self.unique_schema(), revival_timestamp, revival_timestamp) + ) + + def _snapshot_and_assert_revived(self): + self._revived_snapshot_datetime = self._snapshot() + + database = self.adapter.quote(database) + + # records which weren't revived (id != 10, 11) + invalidated_records = self.run_sql( + ''' + select + id, + dbt_valid_to + from {}.{}.snapshot_actual + where dbt_valid_to is not null + order by id + '''.format(database, self.unique_schema()), + fetch='all' + ) + + self.assertEqual(len(invalidated_records), 11) + for result in invalidated_records: + # result is a tuple, the dbt_valid_to column is the latest + self.assertIsInstance(result[1], datetime) + self.assertGreaterEqual(result[1].astimezone(pytz.UTC), self._invalidated_snapshot_datetime) + + # records which weren't revived (id != 10, 11) + revived_records = self.run_sql( + ''' + select + id, + dbt_valid_from, + dbt_valid_to + from {}.{}.snapshot_actual + where dbt_valid_to is null + and id IN (10, 11) + '''.format(database, self.unique_schema()), + fetch='all' + ) + + self.assertEqual(len(revived_records), 2) + for result in revived_records: + # result is a tuple, the dbt_valid_from is second and dbt_valid_to is latest + self.assertIsInstance(result[1], datetime) + # there are milliseconds (part of microseconds in datetime objects) in the + # invalidated_snapshot_datetime and not in result datetime so set the microseconds to 0 + self.assertGreaterEqual(result[1].astimezone(pytz.UTC), self._invalidated_snapshot_datetime.replace(microsecond=0)) + self.assertIsNone(result[2]) diff --git a/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py b/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py new file mode 100644 index 000000000..bd1436c47 --- /dev/null +++ b/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py @@ -0,0 +1,40 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + + +class TestSimpleSnapshotFiles(DBTIntegrationTest): + NUM_SNAPSHOT_MODELS = 1 + + @property + def schema(self): + return "simple_snapshot" + + @property + def models(self): + return "models" + + @property + def project_config(self): + return { + 'config-version': 2, + "snapshot-paths": ['check-snapshots'], + "test-paths": ['check-snapshots-expected'], + "source-paths": [], + } + + def test_snapshot_check_cols_cycle(self): + results = self.run_dbt(["snapshot", '--vars', 'version: 1']) + self.assertEqual(len(results), 1) + + results = self.run_dbt(["snapshot", '--vars', 'version: 2']) + self.assertEqual(len(results), 1) + + results = self.run_dbt(["snapshot", '--vars', 'version: 3']) + self.assertEqual(len(results), 1) + + def assert_expected(self): + self.run_dbt(['test', '--data', '--vars', 'version: 3']) + + @use_profile('bigquery') + def test__bigquery__simple_snapshot(self): + self.test_snapshot_check_cols_cycle() + self.assert_expected() diff --git a/tests/integration/simple_snapshot_test/update_bq.sql b/tests/integration/simple_snapshot_test/update_bq.sql new file mode 100644 index 000000000..5c972d8af --- /dev/null +++ b/tests/integration/simple_snapshot_test/update_bq.sql @@ -0,0 +1,78 @@ +-- insert v2 of the 11 - 21 records + +insert {database}.{schema}.snapshot_expected ( + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + dbt_valid_from, + dbt_valid_to, + dbt_updated_at, + dbt_scd_id +) + +select + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + -- fields added by snapshotting + updated_at as dbt_valid_from, + cast(null as timestamp) as dbt_valid_to, + updated_at as dbt_updated_at, + to_hex(md5(concat(cast(id as string), '-', first_name, '|', cast(updated_at as string)))) as dbt_scd_id +from {database}.{schema}.seed +where id >= 10 and id <= 20; + + +-- insert 10 new records +insert into {database}.{schema}.seed (id, first_name, last_name, email, gender, ip_address, updated_at) values +(21, 'Judy', 'Robinson', 'jrobinsonk@blogs.com', 'Female', '208.21.192.232', '2016-09-18 08:27:38'), +(22, 'Kevin', 'Alvarez', 'kalvarezl@buzzfeed.com', 'Male', '228.106.146.9', '2016-07-29 03:07:37'), +(23, 'Barbara', 'Carr', 'bcarrm@pen.io', 'Female', '106.165.140.17', '2015-09-24 13:27:23'), +(24, 'William', 'Watkins', 'wwatkinsn@guardian.co.uk', 'Male', '78.155.84.6', '2016-03-08 19:13:08'), +(25, 'Judy', 'Cooper', 'jcoopero@google.com.au', 'Female', '24.149.123.184', '2016-10-05 20:49:33'), +(26, 'Shirley', 'Castillo', 'scastillop@samsung.com', 'Female', '129.252.181.12', '2016-06-20 21:12:21'), +(27, 'Justin', 'Harper', 'jharperq@opera.com', 'Male', '131.172.103.218', '2016-05-21 22:56:46'), +(28, 'Marie', 'Medina', 'mmedinar@nhs.uk', 'Female', '188.119.125.67', '2015-10-08 13:44:33'), +(29, 'Kelly', 'Edwards', 'kedwardss@phoca.cz', 'Female', '47.121.157.66', '2015-09-15 06:33:37'), +(30, 'Carl', 'Coleman', 'ccolemant@wikipedia.org', 'Male', '82.227.154.83', '2016-05-26 16:46:40'); + + +-- add these new records to the snapshot table +insert {database}.{schema}.snapshot_expected ( + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + dbt_valid_from, + dbt_valid_to, + dbt_updated_at, + dbt_scd_id +) + +select + id, + first_name, + last_name, + email, + gender, + ip_address, + updated_at, + -- fields added by snapshotting + updated_at as dbt_valid_from, + cast(null as timestamp) as dbt_valid_to, + updated_at as dbt_updated_at, + to_hex(md5(concat(cast(id as string), '-', first_name, '|', cast(updated_at as string)))) as dbt_scd_id +from {database}.{schema}.seed +where id > 20; + diff --git a/tests/integration/sources_test/data/expected_multi_source.csv b/tests/integration/sources_test/data/expected_multi_source.csv new file mode 100644 index 000000000..de9c1c01d --- /dev/null +++ b/tests/integration/sources_test/data/expected_multi_source.csv @@ -0,0 +1,4 @@ +id,first_name,color +1,Larry,blue +2,Curly,red +3,Moe,green diff --git a/tests/integration/sources_test/data/other_source_table.csv b/tests/integration/sources_test/data/other_source_table.csv new file mode 100644 index 000000000..a92b2cb8e --- /dev/null +++ b/tests/integration/sources_test/data/other_source_table.csv @@ -0,0 +1,4 @@ +id,color +1,blue +2,red +3,green diff --git a/tests/integration/sources_test/data/other_table.csv b/tests/integration/sources_test/data/other_table.csv new file mode 100644 index 000000000..56bdda92b --- /dev/null +++ b/tests/integration/sources_test/data/other_table.csv @@ -0,0 +1,4 @@ +id,first_name +1,Larry +2,Curly +3,Moe diff --git a/tests/integration/sources_test/data/source.csv b/tests/integration/sources_test/data/source.csv new file mode 100644 index 000000000..a8f87412e --- /dev/null +++ b/tests/integration/sources_test/data/source.csv @@ -0,0 +1,101 @@ +favorite_color,id,first_name,email,ip_address,updated_at +blue,1,Larry,lking0@miitbeian.gov.cn,'69.135.206.194',2008-09-12 19:08:31 +blue,2,Larry,lperkins1@toplist.cz,'64.210.133.162',1978-05-09 04:15:14 +blue,3,Anna,amontgomery2@miitbeian.gov.cn,'168.104.64.114',2011-10-16 04:07:57 +blue,4,Sandra,sgeorge3@livejournal.com,'229.235.252.98',1973-07-19 10:52:43 +blue,5,Fred,fwoods4@google.cn,'78.229.170.124',2012-09-30 16:38:29 +blue,6,Stephen,shanson5@livejournal.com,'182.227.157.105',1995-11-07 21:40:50 +blue,7,William,wmartinez6@upenn.edu,'135.139.249.50',1982-09-05 03:11:59 +blue,8,Jessica,jlong7@hao123.com,'203.62.178.210',1991-10-16 11:03:15 +blue,9,Douglas,dwhite8@tamu.edu,'178.187.247.1',1979-10-01 09:49:48 +blue,10,Lisa,lcoleman9@nydailynews.com,'168.234.128.249',2011-05-26 07:45:49 +blue,11,Ralph,rfieldsa@home.pl,'55.152.163.149',1972-11-18 19:06:11 +blue,12,Louise,lnicholsb@samsung.com,'141.116.153.154',2014-11-25 20:56:14 +blue,13,Clarence,cduncanc@sfgate.com,'81.171.31.133',2011-11-17 07:02:36 +blue,14,Daniel,dfranklind@omniture.com,'8.204.211.37',1980-09-13 00:09:04 +blue,15,Katherine,klanee@auda.org.au,'176.96.134.59',1997-08-22 19:36:56 +blue,16,Billy,bwardf@wikia.com,'214.108.78.85',2003-10-19 02:14:47 +blue,17,Annie,agarzag@ocn.ne.jp,'190.108.42.70',1988-10-28 15:12:35 +blue,18,Shirley,scolemanh@fastcompany.com,'109.251.164.84',1988-08-24 10:50:57 +blue,19,Roger,rfrazieri@scribd.com,'38.145.218.108',1985-12-31 15:17:15 +blue,20,Lillian,lstanleyj@goodreads.com,'47.57.236.17',1970-06-08 02:09:05 +blue,21,Aaron,arodriguezk@nps.gov,'205.245.118.221',1985-10-11 23:07:49 +blue,22,Patrick,pparkerl@techcrunch.com,'19.8.100.182',2006-03-29 12:53:56 +blue,23,Phillip,pmorenom@intel.com,'41.38.254.103',2011-11-07 15:35:43 +blue,24,Henry,hgarcian@newsvine.com,'1.191.216.252',2008-08-28 08:30:44 +blue,25,Irene,iturnero@opera.com,'50.17.60.190',1994-04-01 07:15:02 +blue,26,Andrew,adunnp@pen.io,'123.52.253.176',2000-11-01 06:03:25 +blue,27,David,dgutierrezq@wp.com,'238.23.203.42',1988-01-25 07:29:18 +blue,28,Henry,hsanchezr@cyberchimps.com,'248.102.2.185',1983-01-01 13:36:37 +blue,29,Evelyn,epetersons@gizmodo.com,'32.80.46.119',1979-07-16 17:24:12 +blue,30,Tammy,tmitchellt@purevolume.com,'249.246.167.88',2001-04-03 10:00:23 +blue,31,Jacqueline,jlittleu@domainmarket.com,'127.181.97.47',1986-02-11 21:35:50 +blue,32,Earl,eortizv@opera.com,'166.47.248.240',1996-07-06 08:16:27 +blue,33,Juan,jgordonw@sciencedirect.com,'71.77.2.200',1987-01-31 03:46:44 +blue,34,Diane,dhowellx@nyu.edu,'140.94.133.12',1994-06-11 02:30:05 +blue,35,Randy,rkennedyy@microsoft.com,'73.255.34.196',2005-05-26 20:28:39 +blue,36,Janice,jriveraz@time.com,'22.214.227.32',1990-02-09 04:16:52 +blue,37,Laura,lperry10@diigo.com,'159.148.145.73',2015-03-17 05:59:25 +blue,38,Gary,gray11@statcounter.com,'40.193.124.56',1970-01-27 10:04:51 +blue,39,Jesse,jmcdonald12@typepad.com,'31.7.86.103',2009-03-14 08:14:29 +blue,40,Sandra,sgonzalez13@goodreads.com,'223.80.168.239',1993-05-21 14:08:54 +blue,41,Scott,smoore14@archive.org,'38.238.46.83',1980-08-30 11:16:56 +blue,42,Phillip,pevans15@cisco.com,'158.234.59.34',2011-12-15 23:26:31 +blue,43,Steven,sriley16@google.ca,'90.247.57.68',2011-10-29 19:03:28 +blue,44,Deborah,dbrown17@hexun.com,'179.125.143.240',1995-04-10 14:36:07 +blue,45,Lori,lross18@ow.ly,'64.80.162.180',1980-12-27 16:49:15 +blue,46,Sean,sjackson19@tumblr.com,'240.116.183.69',1988-06-12 21:24:45 +blue,47,Terry,tbarnes1a@163.com,'118.38.213.137',1997-09-22 16:43:19 +blue,48,Dorothy,dross1b@ebay.com,'116.81.76.49',2005-02-28 13:33:24 +blue,49,Samuel,swashington1c@house.gov,'38.191.253.40',1989-01-19 21:15:48 +blue,50,Ralph,rcarter1d@tinyurl.com,'104.84.60.174',2007-08-11 10:21:49 +green,51,Wayne,whudson1e@princeton.edu,'90.61.24.102',1983-07-03 16:58:12 +green,52,Rose,rjames1f@plala.or.jp,'240.83.81.10',1995-06-08 11:46:23 +green,53,Louise,lcox1g@theglobeandmail.com,'105.11.82.145',2016-09-19 14:45:51 +green,54,Kenneth,kjohnson1h@independent.co.uk,'139.5.45.94',1976-08-17 11:26:19 +green,55,Donna,dbrown1i@amazon.co.uk,'19.45.169.45',2006-05-27 16:51:40 +green,56,Johnny,jvasquez1j@trellian.com,'118.202.238.23',1975-11-17 08:42:32 +green,57,Patrick,pramirez1k@tamu.edu,'231.25.153.198',1997-08-06 11:51:09 +green,58,Helen,hlarson1l@prweb.com,'8.40.21.39',1993-08-04 19:53:40 +green,59,Patricia,pspencer1m@gmpg.org,'212.198.40.15',1977-08-03 16:37:27 +green,60,Joseph,jspencer1n@marriott.com,'13.15.63.238',2005-07-23 20:22:06 +green,61,Phillip,pschmidt1o@blogtalkradio.com,'177.98.201.190',1976-05-19 21:47:44 +green,62,Joan,jwebb1p@google.ru,'105.229.170.71',1972-09-07 17:53:47 +green,63,Phyllis,pkennedy1q@imgur.com,'35.145.8.244',2000-01-01 22:33:37 +green,64,Katherine,khunter1r@smh.com.au,'248.168.205.32',1991-01-09 06:40:24 +green,65,Laura,lvasquez1s@wiley.com,'128.129.115.152',1997-10-23 12:04:56 +green,66,Juan,jdunn1t@state.gov,'44.228.124.51',2004-11-10 05:07:35 +green,67,Judith,jholmes1u@wiley.com,'40.227.179.115',1977-08-02 17:01:45 +green,68,Beverly,bbaker1v@wufoo.com,'208.34.84.59',2016-03-06 20:07:23 +green,69,Lawrence,lcarr1w@flickr.com,'59.158.212.223',1988-09-13 06:07:21 +green,70,Gloria,gwilliams1x@mtv.com,'245.231.88.33',1995-03-18 22:32:46 +green,71,Steven,ssims1y@cbslocal.com,'104.50.58.255',2001-08-05 21:26:20 +green,72,Betty,bmills1z@arstechnica.com,'103.177.214.220',1981-12-14 21:26:54 +green,73,Mildred,mfuller20@prnewswire.com,'151.158.8.130',2000-04-19 10:13:55 +green,74,Donald,dday21@icq.com,'9.178.102.255',1972-12-03 00:58:24 +green,75,Eric,ethomas22@addtoany.com,'85.2.241.227',1992-11-01 05:59:30 +green,76,Joyce,jarmstrong23@sitemeter.com,'169.224.20.36',1985-10-24 06:50:01 +green,77,Maria,mmartinez24@amazonaws.com,'143.189.167.135',2005-10-05 05:17:42 +green,78,Harry,hburton25@youtube.com,'156.47.176.237',1978-03-26 05:53:33 +green,79,Kevin,klawrence26@hao123.com,'79.136.183.83',1994-10-12 04:38:52 +green,80,David,dhall27@prweb.com,'133.149.172.153',1976-12-15 16:24:24 +green,81,Kathy,kperry28@twitter.com,'229.242.72.228',1979-03-04 02:58:56 +green,82,Adam,aprice29@elegantthemes.com,'13.145.21.10',1982-11-07 11:46:59 +green,83,Brandon,bgriffin2a@va.gov,'73.249.128.212',2013-10-30 05:30:36 +green,84,Henry,hnguyen2b@discovery.com,'211.36.214.242',1985-01-09 06:37:27 +green,85,Eric,esanchez2c@edublogs.org,'191.166.188.251',2004-05-01 23:21:42 +green,86,Jason,jlee2d@jimdo.com,'193.92.16.182',1973-01-08 09:05:39 +green,87,Diana,drichards2e@istockphoto.com,'19.130.175.245',1994-10-05 22:50:49 +green,88,Andrea,awelch2f@abc.net.au,'94.155.233.96',2002-04-26 08:41:44 +green,89,Louis,lwagner2g@miitbeian.gov.cn,'26.217.34.111',2003-08-25 07:56:39 +green,90,Jane,jsims2h@seesaa.net,'43.4.220.135',1987-03-20 20:39:04 +green,91,Larry,lgrant2i@si.edu,'97.126.79.34',2000-09-07 20:26:19 +green,92,Louis,ldean2j@prnewswire.com,'37.148.40.127',2011-09-16 20:12:14 +green,93,Jennifer,jcampbell2k@xing.com,'38.106.254.142',1988-07-15 05:06:49 +green,94,Wayne,wcunningham2l@google.com.hk,'223.28.26.187',2009-12-15 06:16:54 +green,95,Lori,lstevens2m@icq.com,'181.250.181.58',1984-10-28 03:29:19 +green,96,Judy,jsimpson2n@marriott.com,'180.121.239.219',1986-02-07 15:18:10 +green,97,Phillip,phoward2o@usa.gov,'255.247.0.175',2002-12-26 08:44:45 +green,98,Gloria,gwalker2p@usa.gov,'156.140.7.128',1997-10-04 07:58:58 +green,99,Paul,pjohnson2q@umn.edu,'183.59.198.197',1991-11-14 12:33:55 +green,100,Frank,fgreene2r@blogspot.com,'150.143.68.121',2010-06-12 23:55:39 diff --git a/tests/integration/sources_test/error_models/model.sql b/tests/integration/sources_test/error_models/model.sql new file mode 100644 index 000000000..55bbcba67 --- /dev/null +++ b/tests/integration/sources_test/error_models/model.sql @@ -0,0 +1 @@ +select * from {{ source('test_source', 'test_table') }} diff --git a/tests/integration/sources_test/error_models/schema.yml b/tests/integration/sources_test/error_models/schema.yml new file mode 100644 index 000000000..69cf1f304 --- /dev/null +++ b/tests/integration/sources_test/error_models/schema.yml @@ -0,0 +1,12 @@ +version: 2 +sources: + - name: test_source + loader: custom + freshness: + warn_after: {count: 10, period: hour} + error_after: {count: 1, period: day} + schema: invalid + tables: + - name: test_table + identifier: source + loaded_at_field: updated_at diff --git a/tests/integration/sources_test/filtered_models/schema.yml b/tests/integration/sources_test/filtered_models/schema.yml new file mode 100644 index 000000000..edad7f6ec --- /dev/null +++ b/tests/integration/sources_test/filtered_models/schema.yml @@ -0,0 +1,18 @@ +version: 2 +sources: + - name: test_source + loader: custom + freshness: + warn_after: {count: 10, period: hour} + error_after: {count: 1, period: day} + filter: id > 1 + schema: "{{ var(env_var('DBT_TEST_SCHEMA_NAME_VARIABLE')) }}" + quoting: + identifier: True + tables: + - name: test_table + identifier: source + loaded_at_field: updated_at + freshness: + error_after: {count: 18, period: hour} + filter: id > 101 diff --git a/tests/integration/sources_test/macros/macro.sql b/tests/integration/sources_test/macros/macro.sql new file mode 100644 index 000000000..a607a6e4c --- /dev/null +++ b/tests/integration/sources_test/macros/macro.sql @@ -0,0 +1,14 @@ +{% macro override_me() -%} + {{ exceptions.raise_compiler_error('this is a bad macro') }} +{%- endmacro %} + +{% macro happy_little_macro() -%} + {{ override_me() }} +{%- endmacro %} + + +{% macro vacuum_source(source_name, table_name) -%} + {% call statement('stmt', auto_begin=false, fetch_result=false) %} + vacuum {{ source(source_name, table_name) }} + {% endcall %} +{%- endmacro %} diff --git a/tests/integration/sources_test/malformed_models/descendant_model.sql b/tests/integration/sources_test/malformed_models/descendant_model.sql new file mode 100644 index 000000000..55bbcba67 --- /dev/null +++ b/tests/integration/sources_test/malformed_models/descendant_model.sql @@ -0,0 +1 @@ +select * from {{ source('test_source', 'test_table') }} diff --git a/tests/integration/sources_test/malformed_models/schema.yml b/tests/integration/sources_test/malformed_models/schema.yml new file mode 100644 index 000000000..544d18d65 --- /dev/null +++ b/tests/integration/sources_test/malformed_models/schema.yml @@ -0,0 +1,14 @@ +version: 2 +sources: + - name: test_source + loader: custom + schema: "{{ var('test_run_schema') }}" + tables: + - name: test_table + identifier: source + tests: + - relationships: + # this is invalid (list of 3 1-key dicts instead of a single 3-key dict) + - column_name: favorite_color + - to: ref('descendant_model') + - field: favorite_color diff --git a/tests/integration/sources_test/malformed_schema_tests/model.sql b/tests/integration/sources_test/malformed_schema_tests/model.sql new file mode 100644 index 000000000..55bbcba67 --- /dev/null +++ b/tests/integration/sources_test/malformed_schema_tests/model.sql @@ -0,0 +1 @@ +select * from {{ source('test_source', 'test_table') }} diff --git a/tests/integration/sources_test/malformed_schema_tests/schema.yml b/tests/integration/sources_test/malformed_schema_tests/schema.yml new file mode 100644 index 000000000..d72ab2eee --- /dev/null +++ b/tests/integration/sources_test/malformed_schema_tests/schema.yml @@ -0,0 +1,14 @@ +version: 2 +sources: + - name: test_source + schema: "{{ var('test_run_schema') }}" + tables: + - name: test_table + identifier: source + columns: + - name: favorite_color + tests: + - relationships: + to: ref('model') + # this will get rendered as its literal + field: "{{ 'favorite' ~ 'color' }}" diff --git a/tests/integration/sources_test/models/descendant_model.sql b/tests/integration/sources_test/models/descendant_model.sql new file mode 100644 index 000000000..55bbcba67 --- /dev/null +++ b/tests/integration/sources_test/models/descendant_model.sql @@ -0,0 +1 @@ +select * from {{ source('test_source', 'test_table') }} diff --git a/tests/integration/sources_test/models/ephemeral_model.sql b/tests/integration/sources_test/models/ephemeral_model.sql new file mode 100644 index 000000000..8de35cd3e --- /dev/null +++ b/tests/integration/sources_test/models/ephemeral_model.sql @@ -0,0 +1,3 @@ +{{ config(materialized='ephemeral') }} + +select 1 as id diff --git a/tests/integration/sources_test/models/multi_source_model.sql b/tests/integration/sources_test/models/multi_source_model.sql new file mode 100644 index 000000000..e310206b0 --- /dev/null +++ b/tests/integration/sources_test/models/multi_source_model.sql @@ -0,0 +1,2 @@ +select * from {{ source('test_source', 'other_test_table')}} + join {{ source('other_source', 'test_table')}} using (id) diff --git a/tests/integration/sources_test/models/nonsource_descendant.sql b/tests/integration/sources_test/models/nonsource_descendant.sql new file mode 100644 index 000000000..97f2151c7 --- /dev/null +++ b/tests/integration/sources_test/models/nonsource_descendant.sql @@ -0,0 +1 @@ +select * from {{ schema }}.source diff --git a/tests/integration/sources_test/models/schema.yml b/tests/integration/sources_test/models/schema.yml new file mode 100644 index 000000000..f02eb1345 --- /dev/null +++ b/tests/integration/sources_test/models/schema.yml @@ -0,0 +1,77 @@ +version: 2 +models: + - name: descendant_model + columns: + - name: favorite_color + tests: + - relationships: + to: source('test_source', 'test_table') + field: favorite_color + +sources: + - name: test_source + loader: custom + freshness: + warn_after: {count: 10, period: hour} + error_after: {count: 1, period: day} + schema: "{{ var(env_var('DBT_TEST_SCHEMA_NAME_VARIABLE')) }}" + quoting: + identifier: True + tags: + - my_test_source_tag + tables: + - name: test_table + identifier: source + loaded_at_field: "{{ var('test_loaded_at') | as_text }}" + freshness: + error_after: {count: 18, period: hour} + tags: + - my_test_source_table_tag + columns: + - name: favorite_color + description: The favorite color + - name: id + description: The user ID + tests: + - unique + - not_null + tags: + - id_column + - name: first_name + description: The first name of the user + tests: [] + - name: email + description: The email address of the user + - name: ip_address + description: The last IP address the user logged in from + - name: updated_at + description: The last update time for this user + tests: + - relationships: + # do this as a table-level test, just to test out that aspect + column_name: favorite_color + to: ref('descendant_model') + field: favorite_color + - name: other_test_table + identifier: other_table + columns: + - name: id + tests: + - not_null + - unique + tags: + - id_column + - name: disabled_test_table + freshness: null + loaded_at_field: "{{ var('test_loaded_at') | as_text }}" + - name: other_source + schema: "{{ var('test_run_schema') }}" + quoting: + identifier: True + tables: + - name: test_table + identifier: other_source_table + - name: external_source + schema: "{{ var('test_run_alt_schema', var('test_run_schema')) }}" + tables: + - name: table diff --git a/tests/integration/sources_test/models/view_model.sql b/tests/integration/sources_test/models/view_model.sql new file mode 100644 index 000000000..ecb330804 --- /dev/null +++ b/tests/integration/sources_test/models/view_model.sql @@ -0,0 +1,3 @@ +{# See here: https://github.com/dbt-labs/dbt/pull/1729 #} + +select * from {{ ref('ephemeral_model') }} diff --git a/tests/integration/sources_test/seed.sql b/tests/integration/sources_test/seed.sql new file mode 100644 index 000000000..40110b990 --- /dev/null +++ b/tests/integration/sources_test/seed.sql @@ -0,0 +1,113 @@ +create table {schema}.seed_expected ( + favorite_color TEXT, + id INTEGER, + first_name TEXT, + email TEXT, + ip_address TEXT, + updated_at TIMESTAMP WITHOUT TIME ZONE +); + + +INSERT INTO {schema}.seed_expected + ("favorite_color","id","first_name","email","ip_address","updated_at") +VALUES + ('blue',1,'Larry','lking0@miitbeian.gov.cn','''69.135.206.194''','2008-09-12 19:08:31'), + ('blue',2,'Larry','lperkins1@toplist.cz','''64.210.133.162''','1978-05-09 04:15:14'), + ('blue',3,'Anna','amontgomery2@miitbeian.gov.cn','''168.104.64.114''','2011-10-16 04:07:57'), + ('blue',4,'Sandra','sgeorge3@livejournal.com','''229.235.252.98''','1973-07-19 10:52:43'), + ('blue',5,'Fred','fwoods4@google.cn','''78.229.170.124''','2012-09-30 16:38:29'), + ('blue',6,'Stephen','shanson5@livejournal.com','''182.227.157.105''','1995-11-07 21:40:50'), + ('blue',7,'William','wmartinez6@upenn.edu','''135.139.249.50''','1982-09-05 03:11:59'), + ('blue',8,'Jessica','jlong7@hao123.com','''203.62.178.210''','1991-10-16 11:03:15'), + ('blue',9,'Douglas','dwhite8@tamu.edu','''178.187.247.1''','1979-10-01 09:49:48'), + ('blue',10,'Lisa','lcoleman9@nydailynews.com','''168.234.128.249''','2011-05-26 07:45:49'), + ('blue',11,'Ralph','rfieldsa@home.pl','''55.152.163.149''','1972-11-18 19:06:11'), + ('blue',12,'Louise','lnicholsb@samsung.com','''141.116.153.154''','2014-11-25 20:56:14'), + ('blue',13,'Clarence','cduncanc@sfgate.com','''81.171.31.133''','2011-11-17 07:02:36'), + ('blue',14,'Daniel','dfranklind@omniture.com','''8.204.211.37''','1980-09-13 00:09:04'), + ('blue',15,'Katherine','klanee@auda.org.au','''176.96.134.59''','1997-08-22 19:36:56'), + ('blue',16,'Billy','bwardf@wikia.com','''214.108.78.85''','2003-10-19 02:14:47'), + ('blue',17,'Annie','agarzag@ocn.ne.jp','''190.108.42.70''','1988-10-28 15:12:35'), + ('blue',18,'Shirley','scolemanh@fastcompany.com','''109.251.164.84''','1988-08-24 10:50:57'), + ('blue',19,'Roger','rfrazieri@scribd.com','''38.145.218.108''','1985-12-31 15:17:15'), + ('blue',20,'Lillian','lstanleyj@goodreads.com','''47.57.236.17''','1970-06-08 02:09:05'), + ('blue',21,'Aaron','arodriguezk@nps.gov','''205.245.118.221''','1985-10-11 23:07:49'), + ('blue',22,'Patrick','pparkerl@techcrunch.com','''19.8.100.182''','2006-03-29 12:53:56'), + ('blue',23,'Phillip','pmorenom@intel.com','''41.38.254.103''','2011-11-07 15:35:43'), + ('blue',24,'Henry','hgarcian@newsvine.com','''1.191.216.252''','2008-08-28 08:30:44'), + ('blue',25,'Irene','iturnero@opera.com','''50.17.60.190''','1994-04-01 07:15:02'), + ('blue',26,'Andrew','adunnp@pen.io','''123.52.253.176''','2000-11-01 06:03:25'), + ('blue',27,'David','dgutierrezq@wp.com','''238.23.203.42''','1988-01-25 07:29:18'), + ('blue',28,'Henry','hsanchezr@cyberchimps.com','''248.102.2.185''','1983-01-01 13:36:37'), + ('blue',29,'Evelyn','epetersons@gizmodo.com','''32.80.46.119''','1979-07-16 17:24:12'), + ('blue',30,'Tammy','tmitchellt@purevolume.com','''249.246.167.88''','2001-04-03 10:00:23'), + ('blue',31,'Jacqueline','jlittleu@domainmarket.com','''127.181.97.47''','1986-02-11 21:35:50'), + ('blue',32,'Earl','eortizv@opera.com','''166.47.248.240''','1996-07-06 08:16:27'), + ('blue',33,'Juan','jgordonw@sciencedirect.com','''71.77.2.200''','1987-01-31 03:46:44'), + ('blue',34,'Diane','dhowellx@nyu.edu','''140.94.133.12''','1994-06-11 02:30:05'), + ('blue',35,'Randy','rkennedyy@microsoft.com','''73.255.34.196''','2005-05-26 20:28:39'), + ('blue',36,'Janice','jriveraz@time.com','''22.214.227.32''','1990-02-09 04:16:52'), + ('blue',37,'Laura','lperry10@diigo.com','''159.148.145.73''','2015-03-17 05:59:25'), + ('blue',38,'Gary','gray11@statcounter.com','''40.193.124.56''','1970-01-27 10:04:51'), + ('blue',39,'Jesse','jmcdonald12@typepad.com','''31.7.86.103''','2009-03-14 08:14:29'), + ('blue',40,'Sandra','sgonzalez13@goodreads.com','''223.80.168.239''','1993-05-21 14:08:54'), + ('blue',41,'Scott','smoore14@archive.org','''38.238.46.83''','1980-08-30 11:16:56'), + ('blue',42,'Phillip','pevans15@cisco.com','''158.234.59.34''','2011-12-15 23:26:31'), + ('blue',43,'Steven','sriley16@google.ca','''90.247.57.68''','2011-10-29 19:03:28'), + ('blue',44,'Deborah','dbrown17@hexun.com','''179.125.143.240''','1995-04-10 14:36:07'), + ('blue',45,'Lori','lross18@ow.ly','''64.80.162.180''','1980-12-27 16:49:15'), + ('blue',46,'Sean','sjackson19@tumblr.com','''240.116.183.69''','1988-06-12 21:24:45'), + ('blue',47,'Terry','tbarnes1a@163.com','''118.38.213.137''','1997-09-22 16:43:19'), + ('blue',48,'Dorothy','dross1b@ebay.com','''116.81.76.49''','2005-02-28 13:33:24'), + ('blue',49,'Samuel','swashington1c@house.gov','''38.191.253.40''','1989-01-19 21:15:48'), + ('blue',50,'Ralph','rcarter1d@tinyurl.com','''104.84.60.174''','2007-08-11 10:21:49'), + ('green',51,'Wayne','whudson1e@princeton.edu','''90.61.24.102''','1983-07-03 16:58:12'), + ('green',52,'Rose','rjames1f@plala.or.jp','''240.83.81.10''','1995-06-08 11:46:23'), + ('green',53,'Louise','lcox1g@theglobeandmail.com','''105.11.82.145''','2016-09-19 14:45:51'), + ('green',54,'Kenneth','kjohnson1h@independent.co.uk','''139.5.45.94''','1976-08-17 11:26:19'), + ('green',55,'Donna','dbrown1i@amazon.co.uk','''19.45.169.45''','2006-05-27 16:51:40'), + ('green',56,'Johnny','jvasquez1j@trellian.com','''118.202.238.23''','1975-11-17 08:42:32'), + ('green',57,'Patrick','pramirez1k@tamu.edu','''231.25.153.198''','1997-08-06 11:51:09'), + ('green',58,'Helen','hlarson1l@prweb.com','''8.40.21.39''','1993-08-04 19:53:40'), + ('green',59,'Patricia','pspencer1m@gmpg.org','''212.198.40.15''','1977-08-03 16:37:27'), + ('green',60,'Joseph','jspencer1n@marriott.com','''13.15.63.238''','2005-07-23 20:22:06'), + ('green',61,'Phillip','pschmidt1o@blogtalkradio.com','''177.98.201.190''','1976-05-19 21:47:44'), + ('green',62,'Joan','jwebb1p@google.ru','''105.229.170.71''','1972-09-07 17:53:47'), + ('green',63,'Phyllis','pkennedy1q@imgur.com','''35.145.8.244''','2000-01-01 22:33:37'), + ('green',64,'Katherine','khunter1r@smh.com.au','''248.168.205.32''','1991-01-09 06:40:24'), + ('green',65,'Laura','lvasquez1s@wiley.com','''128.129.115.152''','1997-10-23 12:04:56'), + ('green',66,'Juan','jdunn1t@state.gov','''44.228.124.51''','2004-11-10 05:07:35'), + ('green',67,'Judith','jholmes1u@wiley.com','''40.227.179.115''','1977-08-02 17:01:45'), + ('green',68,'Beverly','bbaker1v@wufoo.com','''208.34.84.59''','2016-03-06 20:07:23'), + ('green',69,'Lawrence','lcarr1w@flickr.com','''59.158.212.223''','1988-09-13 06:07:21'), + ('green',70,'Gloria','gwilliams1x@mtv.com','''245.231.88.33''','1995-03-18 22:32:46'), + ('green',71,'Steven','ssims1y@cbslocal.com','''104.50.58.255''','2001-08-05 21:26:20'), + ('green',72,'Betty','bmills1z@arstechnica.com','''103.177.214.220''','1981-12-14 21:26:54'), + ('green',73,'Mildred','mfuller20@prnewswire.com','''151.158.8.130''','2000-04-19 10:13:55'), + ('green',74,'Donald','dday21@icq.com','''9.178.102.255''','1972-12-03 00:58:24'), + ('green',75,'Eric','ethomas22@addtoany.com','''85.2.241.227''','1992-11-01 05:59:30'), + ('green',76,'Joyce','jarmstrong23@sitemeter.com','''169.224.20.36''','1985-10-24 06:50:01'), + ('green',77,'Maria','mmartinez24@amazonaws.com','''143.189.167.135''','2005-10-05 05:17:42'), + ('green',78,'Harry','hburton25@youtube.com','''156.47.176.237''','1978-03-26 05:53:33'), + ('green',79,'Kevin','klawrence26@hao123.com','''79.136.183.83''','1994-10-12 04:38:52'), + ('green',80,'David','dhall27@prweb.com','''133.149.172.153''','1976-12-15 16:24:24'), + ('green',81,'Kathy','kperry28@twitter.com','''229.242.72.228''','1979-03-04 02:58:56'), + ('green',82,'Adam','aprice29@elegantthemes.com','''13.145.21.10''','1982-11-07 11:46:59'), + ('green',83,'Brandon','bgriffin2a@va.gov','''73.249.128.212''','2013-10-30 05:30:36'), + ('green',84,'Henry','hnguyen2b@discovery.com','''211.36.214.242''','1985-01-09 06:37:27'), + ('green',85,'Eric','esanchez2c@edublogs.org','''191.166.188.251''','2004-05-01 23:21:42'), + ('green',86,'Jason','jlee2d@jimdo.com','''193.92.16.182''','1973-01-08 09:05:39'), + ('green',87,'Diana','drichards2e@istockphoto.com','''19.130.175.245''','1994-10-05 22:50:49'), + ('green',88,'Andrea','awelch2f@abc.net.au','''94.155.233.96''','2002-04-26 08:41:44'), + ('green',89,'Louis','lwagner2g@miitbeian.gov.cn','''26.217.34.111''','2003-08-25 07:56:39'), + ('green',90,'Jane','jsims2h@seesaa.net','''43.4.220.135''','1987-03-20 20:39:04'), + ('green',91,'Larry','lgrant2i@si.edu','''97.126.79.34''','2000-09-07 20:26:19'), + ('green',92,'Louis','ldean2j@prnewswire.com','''37.148.40.127''','2011-09-16 20:12:14'), + ('green',93,'Jennifer','jcampbell2k@xing.com','''38.106.254.142''','1988-07-15 05:06:49'), + ('green',94,'Wayne','wcunningham2l@google.com.hk','''223.28.26.187''','2009-12-15 06:16:54'), + ('green',95,'Lori','lstevens2m@icq.com','''181.250.181.58''','1984-10-28 03:29:19'), + ('green',96,'Judy','jsimpson2n@marriott.com','''180.121.239.219''','1986-02-07 15:18:10'), + ('green',97,'Phillip','phoward2o@usa.gov','''255.247.0.175''','2002-12-26 08:44:45'), + ('green',98,'Gloria','gwalker2p@usa.gov','''156.140.7.128''','1997-10-04 07:58:58'), + ('green',99,'Paul','pjohnson2q@umn.edu','''183.59.198.197''','1991-11-14 12:33:55'), + ('green',100,'Frank','fgreene2r@blogspot.com','''150.143.68.121''','2010-06-12 23:55:39'); diff --git a/tests/integration/sources_test/test_sources.py b/tests/integration/sources_test/test_sources.py new file mode 100644 index 000000000..8a8cbd0b2 --- /dev/null +++ b/tests/integration/sources_test/test_sources.py @@ -0,0 +1,182 @@ +import json +import os +from datetime import datetime, timedelta + +import yaml + +from dbt.exceptions import CompilationException +import dbt.tracking +import dbt.version +from tests.integration.base import DBTIntegrationTest, use_profile, AnyFloat, \ + AnyStringWith + + +class BaseSourcesTest(DBTIntegrationTest): + @property + def schema(self): + return "sources" + + @property + def models(self): + return "models" + + @property + def project_config(self): + return { + 'config-version': 2, + 'data-paths': ['data'], + 'quoting': {'database': True, 'schema': True, 'identifier': True}, + 'seeds': { + 'quote_columns': True, + }, + } + + def setUp(self): + super().setUp() + os.environ['DBT_TEST_SCHEMA_NAME_VARIABLE'] = 'test_run_schema' + + def tearDown(self): + del os.environ['DBT_TEST_SCHEMA_NAME_VARIABLE'] + super().tearDown() + + def run_dbt_with_vars(self, cmd, *args, **kwargs): + vars_dict = { + 'test_run_schema': self.unique_schema(), + 'test_loaded_at': self.adapter.quote('updated_at'), + } + cmd.extend(['--vars', yaml.safe_dump(vars_dict)]) + return self.run_dbt(cmd, *args, **kwargs) + + +class SuccessfulSourcesTest(BaseSourcesTest): + def setUp(self): + super().setUp() + self.run_dbt_with_vars(['seed']) + self.maxDiff = None + self._id = 101 + # this is the db initial value + self.last_inserted_time = "2016-09-19T14:45:51+00:00" + os.environ['DBT_ENV_CUSTOM_ENV_key'] = 'value' + + def tearDown(self): + super().tearDown() + del os.environ['DBT_ENV_CUSTOM_ENV_key'] + + def _set_updated_at_to(self, delta): + insert_time = datetime.utcnow() + delta + timestr = insert_time.strftime("%Y-%m-%d %H:%M:%S") + # favorite_color,id,first_name,email,ip_address,updated_at + insert_id = self._id + self._id += 1 + raw_sql = """INSERT INTO {schema}.{source} + ({quoted_columns}) + VALUES ( + 'blue',{id},'Jake','abc@example.com','192.168.1.1','{time}' + )""" + quoted_columns = ','.join( + c for c in + ('favorite_color', 'id', 'first_name', + 'email', 'ip_address', 'updated_at') + ) + self.run_sql( + raw_sql, + kwargs={ + 'schema': self.unique_schema(), + 'time': timestr, + 'id': insert_id, + 'source': self.adapter.quote('source'), + 'quoted_columns': quoted_columns, + } + ) + self.last_inserted_time = insert_time.strftime( + "%Y-%m-%dT%H:%M:%S+00:00") + + +class TestSourceFreshness(SuccessfulSourcesTest): + + def _assert_freshness_results(self, path, state): + self.assertTrue(os.path.exists(path)) + with open(path) as fp: + data = json.load(fp) + + assert set(data) == {'metadata', 'results', 'elapsed_time'} + assert 'generated_at' in data['metadata'] + assert isinstance(data['elapsed_time'], float) + self.assertBetween(data['metadata']['generated_at'], + self.freshness_start_time) + assert data['metadata']['dbt_schema_version'] == 'https://schemas.getdbt.com/dbt/sources/v2.json' + assert data['metadata']['dbt_version'] == dbt.version.__version__ + assert data['metadata']['invocation_id'] == dbt.tracking.active_user.invocation_id + key = 'key' + if os.name == 'nt': + key = key.upper() + assert data['metadata']['env'] == {key: 'value'} + + last_inserted_time = self.last_inserted_time + + self.assertEqual(len(data['results']), 1) + + self.assertEqual(data['results'], [ + { + 'unique_id': 'source.test.test_source.test_table', + 'max_loaded_at': last_inserted_time, + 'snapshotted_at': AnyStringWith(), + 'max_loaded_at_time_ago_in_s': AnyFloat(), + 'status': state, + 'criteria': { + 'filter': None, + 'warn_after': {'count': 10, 'period': 'hour'}, + 'error_after': {'count': 18, 'period': 'hour'}, + }, + 'adapter_response': {}, + 'thread_id': AnyStringWith('Thread-'), + 'execution_time': AnyFloat(), + 'timing': [ + { + 'name': 'compile', + 'started_at': AnyStringWith(), + 'completed_at': AnyStringWith(), + }, + { + 'name': 'execute', + 'started_at': AnyStringWith(), + 'completed_at': AnyStringWith(), + } + ] + } + ]) + + def _run_source_freshness(self): + # test_source.test_table should have a loaded_at field of `updated_at` + # and a freshness of warn_after: 10 hours, error_after: 18 hours + # by default, our data set is way out of date! + self.freshness_start_time = datetime.utcnow() + results = self.run_dbt_with_vars( + ['source', 'freshness', '-o', 'target/error_source.json'], + expect_pass=False + ) + self.assertEqual(len(results), 1) + self.assertEqual(results[0].status, 'error') + self._assert_freshness_results('target/error_source.json', 'error') + + self._set_updated_at_to(timedelta(hours=-12)) + self.freshness_start_time = datetime.utcnow() + results = self.run_dbt_with_vars( + ['source', 'freshness', '-o', 'target/warn_source.json'], + ) + self.assertEqual(len(results), 1) + self.assertEqual(results[0].status, 'warn') + self._assert_freshness_results('target/warn_source.json', 'warn') + + self._set_updated_at_to(timedelta(hours=-2)) + self.freshness_start_time = datetime.utcnow() + results = self.run_dbt_with_vars( + ['source', 'freshness', '-o', 'target/pass_source.json'], + ) + self.assertEqual(len(results), 1) + self.assertEqual(results[0].status, 'pass') + self._assert_freshness_results('target/pass_source.json', 'pass') + + @use_profile('bigquery') + def test_bigquery_source_freshness(self): + self._run_source_freshness() diff --git a/tests/integration/statement_test/models-bq/statement_actual.sql b/tests/integration/statement_test/models-bq/statement_actual.sql new file mode 100644 index 000000000..92f9ab1ab --- /dev/null +++ b/tests/integration/statement_test/models-bq/statement_actual.sql @@ -0,0 +1,23 @@ + +-- {{ ref('seed') }} + +{%- call statement('test_statement', fetch_result=True) -%} + + select + count(*) as `num_records` + + from {{ ref('seed') }} + +{%- endcall -%} + +{% set result = load_result('test_statement') %} + +{% set res_table = result['table'] %} +{% set res_matrix = result['data'] %} + +{% set matrix_value = res_matrix[0][0] %} +{% set table_value = res_table[0]['num_records'] %} + +select 'matrix' as source, {{ matrix_value }} as value +union all +select 'table' as source, {{ table_value }} as value diff --git a/tests/integration/statement_test/seed/seed.csv b/tests/integration/statement_test/seed/seed.csv new file mode 100644 index 000000000..640af6c4e --- /dev/null +++ b/tests/integration/statement_test/seed/seed.csv @@ -0,0 +1,101 @@ +id,first_name,last_name,email,gender,ip_address +1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168 +2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35 +3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243 +4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175 +5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136 +6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220 +7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64 +8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13 +9,Gary,Day,gday8@nih.gov,Male,35.81.68.186 +10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100 +11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67 +12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193 +13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5 +14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250 +15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245 +16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54 +17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96 +18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72 +19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174 +20,Andrew,Davis,adavisj@patch.com,Male,9.255.67.25 +21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253 +22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153 +23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201 +24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122 +25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95 +26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52 +27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26 +28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118 +29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28 +30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177 +31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233 +32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203 +33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149 +34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167 +35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110 +36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68 +37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89 +38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81 +39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15 +40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255 +41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140 +42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24 +43,Sean,Mason,smason16@icq.com,Male,159.219.155.249 +44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218 +45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198 +46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18 +47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238 +48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61 +49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21 +50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209 +51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87 +52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142 +53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126 +54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212 +55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194 +56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22 +57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60 +58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50 +59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222 +60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115 +61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155 +62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94 +63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106 +64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68 +65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41 +66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109 +67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77 +68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194 +69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135 +70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87 +71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44 +72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182 +73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241 +74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24 +75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214 +76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199 +77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41 +78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255 +79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144 +80,Rose,King,rking27@ucoz.com,Female,212.123.168.231 +81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188 +82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61 +83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30 +84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192 +85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232 +86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109 +87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156 +88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84 +89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235 +90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53 +91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221 +92,Angela,Rogers,arogers2j@goodreads.com,Female,98.104.132.187 +93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57 +94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189 +95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180 +96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144 +97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117 +98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126 +99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244 +100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88 diff --git a/tests/integration/statement_test/seed/statement_expected.csv b/tests/integration/statement_test/seed/statement_expected.csv new file mode 100644 index 000000000..cf9d9af15 --- /dev/null +++ b/tests/integration/statement_test/seed/statement_expected.csv @@ -0,0 +1,3 @@ +source,value +matrix,100 +table,100 diff --git a/tests/integration/statement_test/test_statements.py b/tests/integration/statement_test/test_statements.py new file mode 100644 index 000000000..0c96fc78c --- /dev/null +++ b/tests/integration/statement_test/test_statements.py @@ -0,0 +1,36 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + + +class TestStatementsBigquery(DBTIntegrationTest): + + @property + def schema(self): + return "statements" + + @staticmethod + def dir(path): + return path.lstrip("/") + + @property + def models(self): + return self.dir("models-bq") + + @property + def project_config(self): + return { + 'config-version': 2, + 'seeds': { + 'quote_columns': False, + } + } + + @use_profile("bigquery") + def test_bigquery_statements(self): + self.use_default_project({"data-paths": [self.dir("seed")]}) + + results = self.run_dbt(["seed"]) + self.assertEqual(len(results), 2) + results = self.run_dbt() + self.assertEqual(len(results), 1) + + self.assertTablesEqual("statement_actual", "statement_expected") diff --git a/tests/integration/store_test_failures_tests/data/expected/expected_accepted_values.csv b/tests/integration/store_test_failures_tests/data/expected/expected_accepted_values.csv new file mode 100644 index 000000000..02f28435b --- /dev/null +++ b/tests/integration/store_test_failures_tests/data/expected/expected_accepted_values.csv @@ -0,0 +1,3 @@ +value_field,n_records +Gary,1 +Rose,1 diff --git a/tests/integration/store_test_failures_tests/data/expected/expected_failing_test.csv b/tests/integration/store_test_failures_tests/data/expected/expected_failing_test.csv new file mode 100644 index 000000000..d9e7257f1 --- /dev/null +++ b/tests/integration/store_test_failures_tests/data/expected/expected_failing_test.csv @@ -0,0 +1,11 @@ +id,first_name,last_name,email,gender,ip_address +1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168 +2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35 +3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243 +4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175 +5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136 +6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220 +7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64 +8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13 +9,Gary,Day,gday8@nih.gov,Male,35.81.68.186 +10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100 diff --git a/tests/integration/store_test_failures_tests/data/expected/expected_not_null_problematic_model_id.csv b/tests/integration/store_test_failures_tests/data/expected/expected_not_null_problematic_model_id.csv new file mode 100644 index 000000000..95fef8a25 --- /dev/null +++ b/tests/integration/store_test_failures_tests/data/expected/expected_not_null_problematic_model_id.csv @@ -0,0 +1,3 @@ +id,first_name,last_name,email,gender,ip_address +,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243 +,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175 diff --git a/tests/integration/store_test_failures_tests/data/expected/expected_unique_problematic_model_id.csv b/tests/integration/store_test_failures_tests/data/expected/expected_unique_problematic_model_id.csv new file mode 100644 index 000000000..431d54ef8 --- /dev/null +++ b/tests/integration/store_test_failures_tests/data/expected/expected_unique_problematic_model_id.csv @@ -0,0 +1,3 @@ +unique_field,n_records +2,2 +1,2 \ No newline at end of file diff --git a/tests/integration/store_test_failures_tests/data/people.csv b/tests/integration/store_test_failures_tests/data/people.csv new file mode 100644 index 000000000..d9e7257f1 --- /dev/null +++ b/tests/integration/store_test_failures_tests/data/people.csv @@ -0,0 +1,11 @@ +id,first_name,last_name,email,gender,ip_address +1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168 +2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35 +3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243 +4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175 +5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136 +6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220 +7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64 +8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13 +9,Gary,Day,gday8@nih.gov,Male,35.81.68.186 +10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100 diff --git a/tests/integration/store_test_failures_tests/models/fine_model.sql b/tests/integration/store_test_failures_tests/models/fine_model.sql new file mode 100644 index 000000000..94b923a17 --- /dev/null +++ b/tests/integration/store_test_failures_tests/models/fine_model.sql @@ -0,0 +1 @@ +select * from {{ ref('people') }} diff --git a/tests/integration/store_test_failures_tests/models/fine_model_but_with_a_no_good_very_long_name.sql b/tests/integration/store_test_failures_tests/models/fine_model_but_with_a_no_good_very_long_name.sql new file mode 100644 index 000000000..97536ffaf --- /dev/null +++ b/tests/integration/store_test_failures_tests/models/fine_model_but_with_a_no_good_very_long_name.sql @@ -0,0 +1 @@ +select 1 as quite_long_column_name diff --git a/tests/integration/store_test_failures_tests/models/problematic_model.sql b/tests/integration/store_test_failures_tests/models/problematic_model.sql new file mode 100644 index 000000000..e780d6b00 --- /dev/null +++ b/tests/integration/store_test_failures_tests/models/problematic_model.sql @@ -0,0 +1,11 @@ +select * from {{ ref('people') }} + +union all + +select * from {{ ref('people') }} +where id in (1,2) + +union all + +select null as id, first_name, last_name, email, gender, ip_address from {{ ref('people') }} +where id in (3,4) diff --git a/tests/integration/store_test_failures_tests/models/schema.yml b/tests/integration/store_test_failures_tests/models/schema.yml new file mode 100644 index 000000000..f01a9e350 --- /dev/null +++ b/tests/integration/store_test_failures_tests/models/schema.yml @@ -0,0 +1,40 @@ +version: 2 + +models: + + - name: fine_model + columns: + - name: id + tests: + - unique + - not_null + + - name: problematic_model + columns: + - name: id + tests: + - unique: + store_failures: true + - not_null + - name: first_name + tests: + # test truncation of really long test name + - accepted_values: + values: + - Jack + - Kathryn + - Gerald + - Bonnie + - Harold + - Jacqueline + - Wanda + - Craig + # - Gary + # - Rose + + - name: fine_model_but_with_a_no_good_very_long_name + columns: + - name: quite_long_column_name + tests: + # test truncation of really long test name with builtin + - unique diff --git a/tests/integration/store_test_failures_tests/test_store_test_failures.py b/tests/integration/store_test_failures_tests/test_store_test_failures.py new file mode 100644 index 000000000..e647c2de4 --- /dev/null +++ b/tests/integration/store_test_failures_tests/test_store_test_failures.py @@ -0,0 +1,60 @@ +from tests.integration.base import DBTIntegrationTest, use_profile + + +class TestStoreTestFailures(DBTIntegrationTest): + @property + def schema(self): + return "test_store_test_failures" + + def tearDown(self): + test_audit_schema = self.unique_schema() + "_dbt_test__audit" + with self.adapter.connection_named('__test'): + self._drop_schema_named(self.default_database, test_audit_schema) + + super().tearDown() + + @property + def models(self): + return "models" + + @property + def project_config(self): + return { + "config-version": 2, + "test-paths": ["tests"], + "seeds": { + "quote_columns": False, + "test": { + "expected": self.column_type_overrides() + }, + }, + } + + def column_type_overrides(self): + return {} + + def run_tests_store_failures_and_assert(self): + test_audit_schema = self.unique_schema() + "_dbt_test__audit" + + self.run_dbt(["seed"]) + self.run_dbt(["run"]) + # make sure this works idempotently for all tests + self.run_dbt(["test", "--store-failures"], expect_pass=False) + results = self.run_dbt(["test", "--store-failures"], expect_pass=False) + + # compare test results + actual = [(r.status, r.failures) for r in results] + expected = [('pass', 0), ('pass', 0), ('pass', 0), ('pass', 0), + ('fail', 2), ('fail', 2), ('fail', 2), ('fail', 10)] + self.assertEqual(sorted(actual), sorted(expected)) + + # compare test results stored in database + self.assertTablesEqual("failing_test", "expected_failing_test", test_audit_schema) + self.assertTablesEqual("not_null_problematic_model_id", "expected_not_null_problematic_model_id", test_audit_schema) + self.assertTablesEqual("unique_problematic_model_id", "expected_unique_problematic_model_id", test_audit_schema) + self.assertTablesEqual("accepted_values_problematic_mo_c533ab4ca65c1a9dbf14f79ded49b628", "expected_accepted_values", test_audit_schema) + +class BigQueryTestStoreTestFailures(TestStoreTestFailures): + @use_profile('bigquery') + def test__bigquery__store_and_assert(self): + self.run_tests_store_failures_and_assert() diff --git a/tests/integration/store_test_failures_tests/tests/failing_test.sql b/tests/integration/store_test_failures_tests/tests/failing_test.sql new file mode 100644 index 000000000..1bb5ae5ba --- /dev/null +++ b/tests/integration/store_test_failures_tests/tests/failing_test.sql @@ -0,0 +1 @@ +select * from {{ ref('fine_model') }} diff --git a/tests/integration/store_test_failures_tests/tests/passing_test.sql b/tests/integration/store_test_failures_tests/tests/passing_test.sql new file mode 100644 index 000000000..15c9a7a64 --- /dev/null +++ b/tests/integration/store_test_failures_tests/tests/passing_test.sql @@ -0,0 +1,2 @@ +select * from {{ ref('fine_model') }} +where false diff --git a/tests/unit/__init__.py b/tests/unit/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/tests/unit/mock_adapter.py b/tests/unit/mock_adapter.py new file mode 100644 index 000000000..4cd9a9717 --- /dev/null +++ b/tests/unit/mock_adapter.py @@ -0,0 +1,81 @@ +from unittest import mock + +from dbt.adapters.base import BaseAdapter +from contextlib import contextmanager + + +def adapter_factory(): + class MockAdapter(BaseAdapter): + ConnectionManager = mock.MagicMock(TYPE='mock') + responder = mock.MagicMock() + # some convenient defaults + responder.quote.side_effect = lambda identifier: '"{}"'.format(identifier) + responder.date_function.side_effect = lambda: 'unitdate()' + responder.is_cancelable.side_effect = lambda: False + + @contextmanager + def exception_handler(self, *args, **kwargs): + self.responder.exception_handler(*args, **kwargs) + yield + + def execute(self, *args, **kwargs): + return self.responder.execute(*args, **kwargs) + + def drop_relation(self, *args, **kwargs): + return self.responder.drop_relation(*args, **kwargs) + + def truncate_relation(self, *args, **kwargs): + return self.responder.truncate_relation(*args, **kwargs) + + def rename_relation(self, *args, **kwargs): + return self.responder.rename_relation(*args, **kwargs) + + def get_columns_in_relation(self, *args, **kwargs): + return self.responder.get_columns_in_relation(*args, **kwargs) + + def expand_column_types(self, *args, **kwargs): + return self.responder.expand_column_types(*args, **kwargs) + + def list_relations_without_caching(self, *args, **kwargs): + return self.responder.list_relations_without_caching(*args, **kwargs) + + def create_schema(self, *args, **kwargs): + return self.responder.create_schema(*args, **kwargs) + + def drop_schema(self, *args, **kwargs): + return self.responder.drop_schema(*args, **kwargs) + + @classmethod + def quote(cls, identifier): + return cls.responder.quote(identifier) + + def convert_text_type(self, *args, **kwargs): + return self.responder.convert_text_type(*args, **kwargs) + + def convert_number_type(self, *args, **kwargs): + return self.responder.convert_number_type(*args, **kwargs) + + def convert_boolean_type(self, *args, **kwargs): + return self.responder.convert_boolean_type(*args, **kwargs) + + def convert_datetime_type(self, *args, **kwargs): + return self.responder.convert_datetime_type(*args, **kwargs) + + def convert_date_type(self, *args, **kwargs): + return self.responder.convert_date_type(*args, **kwargs) + + def convert_time_type(self, *args, **kwargs): + return self.responder.convert_time_type(*args, **kwargs) + + def list_schemas(self, *args, **kwargs): + return self.responder.list_schemas(*args, **kwargs) + + @classmethod + def date_function(cls): + return cls.responder.date_function() + + @classmethod + def is_cancelable(cls): + return cls.responder.is_cancelable() + + return MockAdapter diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py new file mode 100644 index 000000000..46f38afde --- /dev/null +++ b/tests/unit/test_bigquery_adapter.py @@ -0,0 +1,1018 @@ +import agate +import decimal +import json +import string +import random +import re +import pytest +import unittest +from contextlib import contextmanager +from requests.exceptions import ConnectionError +from unittest.mock import patch, MagicMock, Mock, create_autospec, ANY + +import dbt.dataclass_schema + +import dbt.flags as flags + +from dbt.adapters.bigquery import BigQueryCredentials +from dbt.adapters.bigquery import BigQueryAdapter +from dbt.adapters.bigquery import BigQueryRelation +from dbt.adapters.bigquery import Plugin as BigQueryPlugin +from dbt.adapters.bigquery.connections import BigQueryConnectionManager +from dbt.adapters.bigquery.connections import _sanitize_label, _VALIDATE_LABEL_LENGTH_LIMIT +from dbt.adapters.base.query_headers import MacroQueryStringSetter +from dbt.clients import agate_helper +import dbt.exceptions +from dbt.logger import GLOBAL_LOGGER as logger # noqa +from dbt.context.providers import RuntimeConfigObject + +import google.cloud.bigquery + +from .utils import config_from_parts_or_dicts, inject_adapter, TestAdapterConversions + + +def _bq_conn(): + conn = MagicMock() + conn.get.side_effect = lambda x: 'bigquery' if x == 'type' else None + return conn + + +class BaseTestBigQueryAdapter(unittest.TestCase): + + def setUp(self): + self.raw_profile = { + 'outputs': { + 'oauth': { + 'type': 'bigquery', + 'method': 'oauth', + 'project': 'dbt-unit-000000', + 'schema': 'dummy_schema', + 'threads': 1, + }, + 'service_account': { + 'type': 'bigquery', + 'method': 'service-account', + 'project': 'dbt-unit-000000', + 'schema': 'dummy_schema', + 'keyfile': '/tmp/dummy-service-account.json', + 'threads': 1, + }, + 'loc': { + 'type': 'bigquery', + 'method': 'oauth', + 'project': 'dbt-unit-000000', + 'schema': 'dummy_schema', + 'threads': 1, + 'location': 'Luna Station', + 'priority': 'batch', + 'maximum_bytes_billed': 0, + }, + 'impersonate': { + 'type': 'bigquery', + 'method': 'oauth', + 'project': 'dbt-unit-000000', + 'schema': 'dummy_schema', + 'threads': 1, + 'impersonate_service_account': 'dummyaccount@dbt.iam.gserviceaccount.com' + }, + 'oauth-credentials-token': { + 'type': 'bigquery', + 'method': 'oauth-secrets', + 'token': 'abc', + 'project': 'dbt-unit-000000', + 'schema': 'dummy_schema', + 'threads': 1, + 'location': 'Luna Station', + 'priority': 'batch', + 'maximum_bytes_billed': 0, + }, + 'oauth-credentials': { + 'type': 'bigquery', + 'method': 'oauth-secrets', + 'client_id': 'abc', + 'client_secret': 'def', + 'refresh_token': 'ghi', + 'token_uri': 'jkl', + 'project': 'dbt-unit-000000', + 'schema': 'dummy_schema', + 'threads': 1, + 'location': 'Luna Station', + 'priority': 'batch', + 'maximum_bytes_billed': 0, + }, + 'oauth-no-project': { + 'type': 'bigquery', + 'method': 'oauth', + 'schema': 'dummy_schema', + 'threads': 1, + 'location': 'Solar Station', + }, + }, + 'target': 'oauth', + } + + self.project_cfg = { + 'name': 'X', + 'version': '0.1', + 'project-root': '/tmp/dbt/does-not-exist', + 'profile': 'default', + 'config-version': 2, + } + self.qh_patch = None + + def tearDown(self): + if self.qh_patch: + self.qh_patch.stop() + super().tearDown() + + def get_adapter(self, target): + project = self.project_cfg.copy() + profile = self.raw_profile.copy() + profile['target'] = target + + config = config_from_parts_or_dicts( + project=project, + profile=profile, + ) + adapter = BigQueryAdapter(config) + + adapter.connections.query_header = MacroQueryStringSetter(config, MagicMock(macros={})) + + self.qh_patch = patch.object(adapter.connections.query_header, 'add') + self.mock_query_header_add = self.qh_patch.start() + self.mock_query_header_add.side_effect = lambda q: '/* dbt */\n{}'.format(q) + + inject_adapter(adapter, BigQueryPlugin) + return adapter + + +class TestBigQueryAdapterAcquire(BaseTestBigQueryAdapter): + @patch('dbt.adapters.bigquery.connections.get_bigquery_defaults', return_value=('credentials', 'project_id')) + @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn()) + def test_acquire_connection_oauth_no_project_validations(self, mock_open_connection, mock_get_bigquery_defaults): + adapter = self.get_adapter('oauth-no-project') + mock_get_bigquery_defaults.assert_called_once() + try: + connection = adapter.acquire_connection('dummy') + self.assertEqual(connection.type, 'bigquery') + + except dbt.exceptions.ValidationException as e: + self.fail('got ValidationException: {}'.format(str(e))) + + except BaseException as e: + raise + + mock_open_connection.assert_not_called() + connection.handle + mock_open_connection.assert_called_once() + + @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn()) + def test_acquire_connection_oauth_validations(self, mock_open_connection): + adapter = self.get_adapter('oauth') + try: + connection = adapter.acquire_connection('dummy') + self.assertEqual(connection.type, 'bigquery') + + except dbt.exceptions.ValidationException as e: + self.fail('got ValidationException: {}'.format(str(e))) + + except BaseException as e: + raise + + mock_open_connection.assert_not_called() + connection.handle + mock_open_connection.assert_called_once() + + @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn()) + def test_acquire_connection_service_account_validations(self, mock_open_connection): + adapter = self.get_adapter('service_account') + try: + connection = adapter.acquire_connection('dummy') + self.assertEqual(connection.type, 'bigquery') + + except dbt.exceptions.ValidationException as e: + self.fail('got ValidationException: {}'.format(str(e))) + + except BaseException as e: + raise + + mock_open_connection.assert_not_called() + connection.handle + mock_open_connection.assert_called_once() + + @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn()) + def test_acquire_connection_oauth_token_validations(self, mock_open_connection): + adapter = self.get_adapter('oauth-credentials-token') + try: + connection = adapter.acquire_connection('dummy') + self.assertEqual(connection.type, 'bigquery') + + except dbt.exceptions.ValidationException as e: + self.fail('got ValidationException: {}'.format(str(e))) + + except BaseException as e: + raise + + mock_open_connection.assert_not_called() + connection.handle + mock_open_connection.assert_called_once() + + @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn()) + def test_acquire_connection_oauth_credentials_validations(self, mock_open_connection): + adapter = self.get_adapter('oauth-credentials') + try: + connection = adapter.acquire_connection('dummy') + self.assertEqual(connection.type, 'bigquery') + + except dbt.exceptions.ValidationException as e: + self.fail('got ValidationException: {}'.format(str(e))) + + except BaseException as e: + raise + + mock_open_connection.assert_not_called() + connection.handle + mock_open_connection.assert_called_once() + + @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn()) + def test_acquire_connection_impersonated_service_account_validations(self, mock_open_connection): + adapter = self.get_adapter('impersonate') + try: + connection = adapter.acquire_connection('dummy') + self.assertEqual(connection.type, 'bigquery') + + except dbt.exceptions.ValidationException as e: + self.fail('got ValidationException: {}'.format(str(e))) + + except BaseException as e: + raise + + mock_open_connection.assert_not_called() + connection.handle + mock_open_connection.assert_called_once() + + @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn()) + def test_acquire_connection_priority(self, mock_open_connection): + adapter = self.get_adapter('loc') + try: + connection = adapter.acquire_connection('dummy') + self.assertEqual(connection.type, 'bigquery') + self.assertEqual(connection.credentials.priority, 'batch') + + except dbt.exceptions.ValidationException as e: + self.fail('got ValidationException: {}'.format(str(e))) + + mock_open_connection.assert_not_called() + connection.handle + mock_open_connection.assert_called_once() + + @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn()) + def test_acquire_connection_maximum_bytes_billed(self, mock_open_connection): + adapter = self.get_adapter('loc') + try: + connection = adapter.acquire_connection('dummy') + self.assertEqual(connection.type, 'bigquery') + self.assertEqual(connection.credentials.maximum_bytes_billed, 0) + + except dbt.exceptions.ValidationException as e: + self.fail('got ValidationException: {}'.format(str(e))) + + mock_open_connection.assert_not_called() + connection.handle + mock_open_connection.assert_called_once() + + def test_cancel_open_connections_empty(self): + adapter = self.get_adapter('oauth') + self.assertEqual(adapter.cancel_open_connections(), None) + + def test_cancel_open_connections_master(self): + adapter = self.get_adapter('oauth') + adapter.connections.thread_connections[0] = object() + self.assertEqual(adapter.cancel_open_connections(), None) + + def test_cancel_open_connections_single(self): + adapter = self.get_adapter('oauth') + adapter.connections.thread_connections.update({ + 0: object(), + 1: object(), + }) + # actually does nothing + self.assertEqual(adapter.cancel_open_connections(), None) + + @patch('dbt.adapters.bigquery.impl.google.auth.default') + @patch('dbt.adapters.bigquery.impl.google.cloud.bigquery') + def test_location_user_agent(self, mock_bq, mock_auth_default): + creds = MagicMock() + mock_auth_default.return_value = (creds, MagicMock()) + adapter = self.get_adapter('loc') + + connection = adapter.acquire_connection('dummy') + mock_client = mock_bq.Client + + mock_client.assert_not_called() + connection.handle + mock_client.assert_called_once_with('dbt-unit-000000', creds, + location='Luna Station', + client_info=HasUserAgent()) + + +class HasUserAgent: + PAT = re.compile(r'dbt-\d+\.\d+\.\d+((a|b|rc)\d+)?') + + def __eq__(self, other): + compare = getattr(other, 'user_agent', '') + return bool(self.PAT.match(compare)) + + +class TestConnectionNamePassthrough(BaseTestBigQueryAdapter): + + def setUp(self): + super().setUp() + self._conn_patch = patch.object(BigQueryAdapter, 'ConnectionManager') + self.conn_manager_cls = self._conn_patch.start() + + self._relation_patch = patch.object(BigQueryAdapter, 'Relation') + self.relation_cls = self._relation_patch.start() + + self.mock_connection_manager = self.conn_manager_cls.return_value + self.conn_manager_cls.TYPE = 'bigquery' + self.relation_cls.get_default_quote_policy.side_effect = BigQueryRelation.get_default_quote_policy + + self.adapter = self.get_adapter('oauth') + + def tearDown(self): + super().tearDown() + self._conn_patch.stop() + self._relation_patch.stop() + + def test_get_relation(self): + self.adapter.get_relation('db', 'schema', 'my_model') + self.mock_connection_manager.get_bq_table.assert_called_once_with('db', 'schema', 'my_model') + + def test_create_schema(self): + relation = BigQueryRelation.create(database='db', schema='schema') + self.adapter.create_schema(relation) + self.mock_connection_manager.create_dataset.assert_called_once_with('db', 'schema') + + @patch.object(BigQueryAdapter, 'check_schema_exists') + def test_drop_schema(self, mock_check_schema): + mock_check_schema.return_value = True + relation = BigQueryRelation.create(database='db', schema='schema') + self.adapter.drop_schema(relation) + self.mock_connection_manager.drop_dataset.assert_called_once_with('db', 'schema') + + def test_get_columns_in_relation(self): + self.mock_connection_manager.get_bq_table.side_effect = ValueError + self.adapter.get_columns_in_relation( + MagicMock(database='db', schema='schema', identifier='ident'), + ) + self.mock_connection_manager.get_bq_table.assert_called_once_with( + database='db', schema='schema', identifier='ident' + ) + + +class TestBigQueryRelation(unittest.TestCase): + def setUp(self): + pass + + def test_view_temp_relation(self): + kwargs = { + 'type': None, + 'path': { + 'database': 'test-project', + 'schema': 'test_schema', + 'identifier': 'my_view' + }, + 'quote_policy': { + 'identifier': False + } + } + BigQueryRelation.validate(kwargs) + + def test_view_relation(self): + kwargs = { + 'type': 'view', + 'path': { + 'database': 'test-project', + 'schema': 'test_schema', + 'identifier': 'my_view' + }, + 'quote_policy': { + 'identifier': True, + 'schema': True + } + } + BigQueryRelation.validate(kwargs) + + def test_table_relation(self): + kwargs = { + 'type': 'table', + 'path': { + 'database': 'test-project', + 'schema': 'test_schema', + 'identifier': 'generic_table' + }, + 'quote_policy': { + 'identifier': True, + 'schema': True + } + } + BigQueryRelation.validate(kwargs) + + def test_external_source_relation(self): + kwargs = { + 'type': 'external', + 'path': { + 'database': 'test-project', + 'schema': 'test_schema', + 'identifier': 'sheet' + }, + 'quote_policy': { + 'identifier': True, + 'schema': True + } + } + BigQueryRelation.validate(kwargs) + + def test_invalid_relation(self): + kwargs = { + 'type': 'invalid-type', + 'path': { + 'database': 'test-project', + 'schema': 'test_schema', + 'identifier': 'my_invalid_id' + }, + 'quote_policy': { + 'identifier': False, + 'schema': True + } + } + with self.assertRaises(dbt.dataclass_schema.ValidationError): + BigQueryRelation.validate(kwargs) + + +class TestBigQueryInformationSchema(unittest.TestCase): + def setUp(self): + pass + + def test_replace(self): + + kwargs = { + 'type': None, + 'path': { + 'database': 'test-project', + 'schema': 'test_schema', + 'identifier': 'my_view' + }, + # test for #2188 + 'quote_policy': { + 'database': False + }, + 'include_policy': { + 'database': True, + 'schema': True, + 'identifier': True, + } + } + BigQueryRelation.validate(kwargs) + relation = BigQueryRelation.from_dict(kwargs) + info_schema = relation.information_schema() + + tables_schema = info_schema.replace(information_schema_view='__TABLES__') + assert tables_schema.information_schema_view == '__TABLES__' + assert tables_schema.include_policy.schema is True + assert tables_schema.include_policy.identifier is False + assert tables_schema.include_policy.database is True + assert tables_schema.quote_policy.schema is True + assert tables_schema.quote_policy.identifier is False + assert tables_schema.quote_policy.database is False + + schemata_schema = info_schema.replace(information_schema_view='SCHEMATA') + assert schemata_schema.information_schema_view == 'SCHEMATA' + assert schemata_schema.include_policy.schema is False + assert schemata_schema.include_policy.identifier is True + assert schemata_schema.include_policy.database is True + assert schemata_schema.quote_policy.schema is True + assert schemata_schema.quote_policy.identifier is False + assert schemata_schema.quote_policy.database is False + + other_schema = info_schema.replace(information_schema_view='SOMETHING_ELSE') + assert other_schema.information_schema_view == 'SOMETHING_ELSE' + assert other_schema.include_policy.schema is True + assert other_schema.include_policy.identifier is True + assert other_schema.include_policy.database is True + assert other_schema.quote_policy.schema is True + assert other_schema.quote_policy.identifier is False + assert other_schema.quote_policy.database is False + + +class TestBigQueryConnectionManager(unittest.TestCase): + + def setUp(self): + credentials = Mock(BigQueryCredentials) + profile = Mock(query_comment=None, credentials=credentials) + self.connections = BigQueryConnectionManager(profile=profile) + self.mock_client = Mock( + dbt.adapters.bigquery.impl.google.cloud.bigquery.Client) + self.mock_connection = MagicMock() + + self.mock_connection.handle = self.mock_client + + self.connections.get_thread_connection = lambda: self.mock_connection + + @patch( + 'dbt.adapters.bigquery.connections._is_retryable', return_value=True) + def test_retry_and_handle(self, is_retryable): + self.connections.DEFAULT_MAXIMUM_DELAY = 2.0 + + @contextmanager + def dummy_handler(msg): + yield + + self.connections.exception_handler = dummy_handler + + class DummyException(Exception): + """Count how many times this exception is raised""" + count = 0 + + def __init__(self): + DummyException.count += 1 + + def raiseDummyException(): + raise DummyException() + + with self.assertRaises(DummyException): + self.connections._retry_and_handle( + "some sql", Mock(credentials=Mock(retries=8)), + raiseDummyException) + self.assertEqual(DummyException.count, 9) + + @patch( + 'dbt.adapters.bigquery.connections._is_retryable', return_value=True) + def test_retry_connection_reset(self, is_retryable): + self.connections.open = MagicMock() + self.connections.close = MagicMock() + self.connections.DEFAULT_MAXIMUM_DELAY = 2.0 + + @contextmanager + def dummy_handler(msg): + yield + + self.connections.exception_handler = dummy_handler + + def raiseConnectionResetError(): + raise ConnectionResetError("Connection broke") + + mock_conn = Mock(credentials=Mock(retries=1)) + with self.assertRaises(ConnectionResetError): + self.connections._retry_and_handle( + "some sql", mock_conn, + raiseConnectionResetError) + self.connections.close.assert_called_once_with(mock_conn) + self.connections.open.assert_called_once_with(mock_conn) + + def test_is_retryable(self): + _is_retryable = dbt.adapters.bigquery.connections._is_retryable + exceptions = dbt.adapters.bigquery.impl.google.cloud.exceptions + internal_server_error = exceptions.InternalServerError('code broke') + bad_request_error = exceptions.BadRequest('code broke') + connection_error = ConnectionError('code broke') + client_error = exceptions.ClientError('bad code') + rate_limit_error = exceptions.Forbidden("code broke", errors=[{"reason": "rateLimitExceeded"}]) + + self.assertTrue(_is_retryable(internal_server_error)) + self.assertTrue(_is_retryable(bad_request_error)) + self.assertTrue(_is_retryable(connection_error)) + self.assertFalse(_is_retryable(client_error)) + self.assertTrue(_is_retryable(rate_limit_error)) + + def test_drop_dataset(self): + mock_table = Mock() + mock_table.reference = 'table1' + + self.mock_client.list_tables.return_value = [mock_table] + + self.connections.drop_dataset('project', 'dataset') + + self.mock_client.list_tables.assert_not_called() + self.mock_client.delete_table.assert_not_called() + self.mock_client.delete_dataset.assert_called_once() + + @patch('dbt.adapters.bigquery.impl.google.cloud.bigquery') + def test_query_and_results(self, mock_bq): + self.connections.get_timeout = lambda x: 100.0 + + self.connections._query_and_results( + self.mock_client, 'sql', self.mock_connection, + {'description': 'blah'}) + + mock_bq.QueryJobConfig.assert_called_once() + self.mock_client.query.assert_called_once_with( + 'sql', job_config=mock_bq.QueryJobConfig()) + + def test_copy_bq_table_appends(self): + self._copy_table( + write_disposition=dbt.adapters.bigquery.impl.WRITE_APPEND) + args, kwargs = self.mock_client.copy_table.call_args + self.mock_client.copy_table.assert_called_once_with( + [self._table_ref('project', 'dataset', 'table1', None)], + self._table_ref('project', 'dataset', 'table2', None), + job_config=ANY) + args, kwargs = self.mock_client.copy_table.call_args + self.assertEqual( + kwargs['job_config'].write_disposition, + dbt.adapters.bigquery.impl.WRITE_APPEND) + + def test_copy_bq_table_truncates(self): + self._copy_table( + write_disposition=dbt.adapters.bigquery.impl.WRITE_TRUNCATE) + args, kwargs = self.mock_client.copy_table.call_args + self.mock_client.copy_table.assert_called_once_with( + [self._table_ref('project', 'dataset', 'table1', None)], + self._table_ref('project', 'dataset', 'table2', None), + job_config=ANY) + args, kwargs = self.mock_client.copy_table.call_args + self.assertEqual( + kwargs['job_config'].write_disposition, + dbt.adapters.bigquery.impl.WRITE_TRUNCATE) + + def test_job_labels_valid_json(self): + expected = {"key": "value"} + labels = self.connections._labels_from_query_comment(json.dumps(expected)) + self.assertEqual(labels, expected) + + def test_job_labels_invalid_json(self): + labels = self.connections._labels_from_query_comment("not json") + self.assertEqual(labels, {"query_comment": "not_json"}) + + def _table_ref(self, proj, ds, table, conn): + return google.cloud.bigquery.table.TableReference.from_string( + '{}.{}.{}'.format(proj, ds, table)) + + def _copy_table(self, write_disposition): + self.connections.table_ref = self._table_ref + source = BigQueryRelation.create( + database='project', schema='dataset', identifier='table1') + destination = BigQueryRelation.create( + database='project', schema='dataset', identifier='table2') + self.connections.copy_bq_table(source, destination, write_disposition) + + +class TestBigQueryAdapter(BaseTestBigQueryAdapter): + + def test_copy_table_materialization_table(self): + adapter = self.get_adapter('oauth') + adapter.connections = MagicMock() + adapter.copy_table('source', 'destination', 'table') + adapter.connections.copy_bq_table.assert_called_once_with( + 'source', 'destination', + dbt.adapters.bigquery.impl.WRITE_TRUNCATE) + + def test_copy_table_materialization_incremental(self): + adapter = self.get_adapter('oauth') + adapter.connections = MagicMock() + adapter.copy_table('source', 'destination', 'incremental') + adapter.connections.copy_bq_table.assert_called_once_with( + 'source', 'destination', + dbt.adapters.bigquery.impl.WRITE_APPEND) + + def test_parse_partition_by(self): + adapter = self.get_adapter('oauth') + + with self.assertRaises(dbt.exceptions.CompilationException): + adapter.parse_partition_by("date(ts)") + + with self.assertRaises(dbt.exceptions.CompilationException): + adapter.parse_partition_by("ts") + + self.assertEqual( + adapter.parse_partition_by({ + "field": "ts", + }).to_dict(omit_none=True), { + "field": "ts", + "data_type": "date", + "granularity": "day" + } + ) + + self.assertEqual( + adapter.parse_partition_by({ + "field": "ts", + "data_type": "date", + }).to_dict(omit_none=True), { + "field": "ts", + "data_type": "date", + "granularity": "day" + } + ) + + self.assertEqual( + adapter.parse_partition_by({ + "field": "ts", + "data_type": "date", + "granularity": "MONTH" + + }).to_dict(omit_none=True), { + "field": "ts", + "data_type": "date", + "granularity": "MONTH" + } + ) + + self.assertEqual( + adapter.parse_partition_by({ + "field": "ts", + "data_type": "date", + "granularity": "YEAR" + + }).to_dict(omit_none=True), { + "field": "ts", + "data_type": "date", + "granularity": "YEAR" + } + ) + + self.assertEqual( + adapter.parse_partition_by({ + "field": "ts", + "data_type": "timestamp", + "granularity": "HOUR" + + }).to_dict(omit_none=True), { + "field": "ts", + "data_type": "timestamp", + "granularity": "HOUR" + } + ) + + self.assertEqual( + adapter.parse_partition_by({ + "field": "ts", + "data_type": "timestamp", + "granularity": "MONTH" + + }).to_dict(omit_none=True + ), { + "field": "ts", + "data_type": "timestamp", + "granularity": "MONTH" + } + ) + + self.assertEqual( + adapter.parse_partition_by({ + "field": "ts", + "data_type": "timestamp", + "granularity": "YEAR" + + }).to_dict(omit_none=True), { + "field": "ts", + "data_type": "timestamp", + "granularity": "YEAR" + } + ) + + self.assertEqual( + adapter.parse_partition_by({ + "field": "ts", + "data_type": "datetime", + "granularity": "HOUR" + + }).to_dict(omit_none=True), { + "field": "ts", + "data_type": "datetime", + "granularity": "HOUR" + } + ) + + self.assertEqual( + adapter.parse_partition_by({ + "field": "ts", + "data_type": "datetime", + "granularity": "MONTH" + + }).to_dict(omit_none=True), { + "field": "ts", + "data_type": "datetime", + "granularity": "MONTH" + } + ) + + self.assertEqual( + adapter.parse_partition_by({ + "field": "ts", + "data_type": "datetime", + "granularity": "YEAR" + + }).to_dict(omit_none=True), { + "field": "ts", + "data_type": "datetime", + "granularity": "YEAR" + } + ) + + # Invalid, should raise an error + with self.assertRaises(dbt.exceptions.CompilationException): + adapter.parse_partition_by({}) + + # passthrough + self.assertEqual( + adapter.parse_partition_by({ + "field": "id", + "data_type": "int64", + "range": { + "start": 1, + "end": 100, + "interval": 20 + } + }).to_dict(omit_none=True + ), { + "field": "id", + "data_type": "int64", + "granularity": "day", + "range": { + "start": 1, + "end": 100, + "interval": 20 + } + } + ) + + def test_hours_to_expiration(self): + adapter = self.get_adapter('oauth') + mock_config = create_autospec( + RuntimeConfigObject) + config = {'hours_to_expiration': 4} + mock_config.get.side_effect = lambda name: config.get(name) + + expected = { + 'expiration_timestamp': 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 4 hour)', + } + actual = adapter.get_table_options(mock_config, node={}, temporary=False) + self.assertEqual(expected, actual) + + + def test_hours_to_expiration_temporary(self): + adapter = self.get_adapter('oauth') + mock_config = create_autospec( + RuntimeConfigObject) + config={'hours_to_expiration': 4} + mock_config.get.side_effect = lambda name: config.get(name) + + expected = { + 'expiration_timestamp': ( + 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)'), + } + actual = adapter.get_table_options(mock_config, node={}, temporary=True) + self.assertEqual(expected, actual) + + def test_table_kms_key_name(self): + adapter = self.get_adapter('oauth') + mock_config = create_autospec( + RuntimeConfigObject) + config={'kms_key_name': 'some_key'} + mock_config.get.side_effect = lambda name: config.get(name) + + expected = { + 'kms_key_name': "'some_key'" + } + actual = adapter.get_table_options(mock_config, node={}, temporary=False) + self.assertEqual(expected, actual) + + + def test_view_kms_key_name(self): + adapter = self.get_adapter('oauth') + mock_config = create_autospec( + RuntimeConfigObject) + config={'kms_key_name': 'some_key'} + mock_config.get.side_effect = lambda name: config.get(name) + + expected = {} + actual = adapter.get_view_options(mock_config, node={}) + self.assertEqual(expected, actual) + + + +class TestBigQueryFilterCatalog(unittest.TestCase): + def test__catalog_filter_table(self): + manifest = MagicMock() + manifest.get_used_schemas.return_value = [['a', 'B'], ['a', '1234']] + column_names = ['table_name', 'table_database', 'table_schema', 'something'] + rows = [ + ['foo', 'a', 'b', '1234'], # include + ['foo', 'a', '1234', '1234'], # include, w/ table schema as str + ['foo', 'c', 'B', '1234'], # skip + ['1234', 'A', 'B', '1234'], # include, w/ table name as str + ] + table = agate.Table( + rows, column_names, agate_helper.DEFAULT_TYPE_TESTER + ) + + result = BigQueryAdapter._catalog_filter_table(table, manifest) + assert len(result) == 3 + for row in result.rows: + assert isinstance(row['table_schema'], str) + assert isinstance(row['table_database'], str) + assert isinstance(row['table_name'], str) + assert isinstance(row['something'], decimal.Decimal) + + +class TestBigQueryAdapterConversions(TestAdapterConversions): + def test_convert_text_type(self): + rows = [ + ['', 'a1', 'stringval1'], + ['', 'a2', 'stringvalasdfasdfasdfa'], + ['', 'a3', 'stringval3'], + ] + agate_table = self._make_table_of(rows, agate.Text) + expected = ['string', 'string', 'string'] + for col_idx, expect in enumerate(expected): + assert BigQueryAdapter.convert_text_type(agate_table, col_idx) == expect + + def test_convert_number_type(self): + rows = [ + ['', '23.98', '-1'], + ['', '12.78', '-2'], + ['', '79.41', '-3'], + ] + agate_table = self._make_table_of(rows, agate.Number) + expected = ['int64', 'float64', 'int64'] + for col_idx, expect in enumerate(expected): + assert BigQueryAdapter.convert_number_type(agate_table, col_idx) == expect + + def test_convert_boolean_type(self): + rows = [ + ['', 'false', 'true'], + ['', 'false', 'false'], + ['', 'false', 'true'], + ] + agate_table = self._make_table_of(rows, agate.Boolean) + expected = ['bool', 'bool', 'bool'] + for col_idx, expect in enumerate(expected): + assert BigQueryAdapter.convert_boolean_type(agate_table, col_idx) == expect + + def test_convert_datetime_type(self): + rows = [ + ['', '20190101T01:01:01Z', '2019-01-01 01:01:01'], + ['', '20190102T01:01:01Z', '2019-01-01 01:01:01'], + ['', '20190103T01:01:01Z', '2019-01-01 01:01:01'], + ] + agate_table = self._make_table_of(rows, [agate.DateTime, agate_helper.ISODateTime, agate.DateTime]) + expected = ['datetime', 'datetime', 'datetime'] + for col_idx, expect in enumerate(expected): + assert BigQueryAdapter.convert_datetime_type(agate_table, col_idx) == expect + + def test_convert_date_type(self): + rows = [ + ['', '2019-01-01', '2019-01-04'], + ['', '2019-01-02', '2019-01-04'], + ['', '2019-01-03', '2019-01-04'], + ] + agate_table = self._make_table_of(rows, agate.Date) + expected = ['date', 'date', 'date'] + for col_idx, expect in enumerate(expected): + assert BigQueryAdapter.convert_date_type(agate_table, col_idx) == expect + + def test_convert_time_type(self): + # dbt's default type testers actually don't have a TimeDelta at all. + agate.TimeDelta + rows = [ + ['', '120s', '10s'], + ['', '3m', '11s'], + ['', '1h', '12s'], + ] + agate_table = self._make_table_of(rows, agate.TimeDelta) + expected = ['time', 'time', 'time'] + for col_idx, expect in enumerate(expected): + assert BigQueryAdapter.convert_time_type(agate_table, col_idx) == expect + + +@pytest.mark.parametrize( + ["input", "output"], + [ + ("ABC", "abc"), + ("a c", "a_c"), + ("a ", "a"), + ], +) +def test_sanitize_label(input, output): + assert _sanitize_label(input) == output + + +@pytest.mark.parametrize( + "label_length", + [64, 65, 100], +) +def test_sanitize_label_length(label_length): + random_string = "".join( + random.choice(string.ascii_uppercase + string.digits) + for i in range(label_length) + ) + test_error_msg = ( + f"Job label length {label_length} is greater than length limit: " + f"{_VALIDATE_LABEL_LENGTH_LIMIT}\n" + f"Current sanitized label: {random_string.lower()}" + ) + with pytest.raises(dbt.exceptions.RuntimeException) as error_info: + _sanitize_label(random_string) + assert error_info.value.args[0] == test_error_msg diff --git a/tests/unit/utils.py b/tests/unit/utils.py new file mode 100644 index 000000000..01c737e1b --- /dev/null +++ b/tests/unit/utils.py @@ -0,0 +1,371 @@ +"""Unit test utility functions. + +Note that all imports should be inside the functions to avoid import/mocking +issues. +""" +import string +import os +from unittest import mock +from unittest import TestCase + +import agate +import pytest +from dbt.dataclass_schema import ValidationError + + +def normalize(path): + """On windows, neither is enough on its own: + + >>> normcase('C:\\documents/ALL CAPS/subdir\\..') + 'c:\\documents\\all caps\\subdir\\..' + >>> normpath('C:\\documents/ALL CAPS/subdir\\..') + 'C:\\documents\\ALL CAPS' + >>> normpath(normcase('C:\\documents/ALL CAPS/subdir\\..')) + 'c:\\documents\\all caps' + """ + return os.path.normcase(os.path.normpath(path)) + + +class Obj: + which = 'blah' + single_threaded = False + + +def mock_connection(name, state='open'): + conn = mock.MagicMock() + conn.name = name + conn.state = state + return conn + + +def profile_from_dict(profile, profile_name, cli_vars='{}'): + from dbt.config import Profile + from dbt.config.renderer import ProfileRenderer + from dbt.context.base import generate_base_context + from dbt.config.utils import parse_cli_vars + if not isinstance(cli_vars, dict): + cli_vars = parse_cli_vars(cli_vars) + + renderer = ProfileRenderer(generate_base_context(cli_vars)) + return Profile.from_raw_profile_info( + profile, + profile_name, + renderer, + ) + + +def project_from_dict(project, profile, packages=None, selectors=None, cli_vars='{}'): + from dbt.context.target import generate_target_context + from dbt.config import Project + from dbt.config.renderer import DbtProjectYamlRenderer + from dbt.config.utils import parse_cli_vars + if not isinstance(cli_vars, dict): + cli_vars = parse_cli_vars(cli_vars) + + renderer = DbtProjectYamlRenderer(generate_target_context(profile, cli_vars)) + + project_root = project.pop('project-root', os.getcwd()) + + return Project.render_from_dict( + project_root, project, packages, selectors, renderer + ) + + +def config_from_parts_or_dicts(project, profile, packages=None, selectors=None, cli_vars='{}'): + from dbt.config import Project, Profile, RuntimeConfig + from copy import deepcopy + + if isinstance(project, Project): + profile_name = project.profile_name + else: + profile_name = project.get('profile') + + if not isinstance(profile, Profile): + profile = profile_from_dict( + deepcopy(profile), + profile_name, + cli_vars, + ) + + if not isinstance(project, Project): + project = project_from_dict( + deepcopy(project), + profile, + packages, + selectors, + cli_vars, + ) + + args = Obj() + args.vars = cli_vars + args.profile_dir = '/dev/null' + return RuntimeConfig.from_parts( + project=project, + profile=profile, + args=args + ) + + +def inject_plugin(plugin): + from dbt.adapters.factory import FACTORY + key = plugin.adapter.type() + FACTORY.plugins[key] = plugin + + +def inject_plugin_for(config): + # from dbt.adapters.postgres import Plugin, PostgresAdapter + from dbt.adapters.factory import FACTORY + FACTORY.load_plugin(config.credentials.type) + adapter = FACTORY.get_adapter(config) + return adapter + + +def inject_adapter(value, plugin): + """Inject the given adapter into the adapter factory, so your hand-crafted + artisanal adapter will be available from get_adapter() as if dbt loaded it. + """ + inject_plugin(plugin) + from dbt.adapters.factory import FACTORY + key = value.type() + FACTORY.adapters[key] = value + + +def clear_plugin(plugin): + from dbt.adapters.factory import FACTORY + key = plugin.adapter.type() + FACTORY.plugins.pop(key, None) + FACTORY.adapters.pop(key, None) + + +class ContractTestCase(TestCase): + ContractType = None + + def setUp(self): + self.maxDiff = None + super().setUp() + + def assert_to_dict(self, obj, dct): + self.assertEqual(obj.to_dict(omit_none=True), dct) + + def assert_from_dict(self, obj, dct, cls=None): + if cls is None: + cls = self.ContractType + cls.validate(dct) + self.assertEqual(cls.from_dict(dct), obj) + + def assert_symmetric(self, obj, dct, cls=None): + self.assert_to_dict(obj, dct) + self.assert_from_dict(obj, dct, cls) + + def assert_fails_validation(self, dct, cls=None): + if cls is None: + cls = self.ContractType + + with self.assertRaises(ValidationError): + cls.validate(dct) + cls.from_dict(dct) + + +def compare_dicts(dict1, dict2): + first_set = set(dict1.keys()) + second_set = set(dict2.keys()) + print(f"--- Difference between first and second keys: {first_set.difference(second_set)}") + print(f"--- Difference between second and first keys: {second_set.difference(first_set)}") + common_keys = set(first_set).intersection(set(second_set)) + found_differences = False + for key in common_keys: + if dict1[key] != dict2[key] : + print(f"--- --- first dict: {key}: {str(dict1[key])}") + print(f"--- --- second dict: {key}: {str(dict2[key])}") + found_differences = True + if found_differences: + print("--- Found differences in dictionaries") + else: + print("--- Found no differences in dictionaries") + + +def assert_from_dict(obj, dct, cls=None): + if cls is None: + cls = obj.__class__ + cls.validate(dct) + obj_from_dict = cls.from_dict(dct) + if hasattr(obj, 'created_at'): + obj_from_dict.created_at = 1 + obj.created_at = 1 + assert obj_from_dict == obj + + +def assert_to_dict(obj, dct): + obj_to_dict = obj.to_dict(omit_none=True) + if 'created_at' in obj_to_dict: + obj_to_dict['created_at'] = 1 + if 'created_at' in dct: + dct['created_at'] = 1 + assert obj_to_dict == dct + + +def assert_symmetric(obj, dct, cls=None): + assert_to_dict(obj, dct) + assert_from_dict(obj, dct, cls) + + +def assert_fails_validation(dct, cls): + with pytest.raises(ValidationError): + cls.validate(dct) + cls.from_dict(dct) + + +def generate_name_macros(package): + from dbt.contracts.graph.parsed import ParsedMacro + from dbt.node_types import NodeType + name_sql = {} + for component in ('database', 'schema', 'alias'): + if component == 'alias': + source = 'node.name' + else: + source = f'target.{component}' + name = f'generate_{component}_name' + sql = f'{{% macro {name}(value, node) %}} {{% if value %}} {{{{ value }}}} {{% else %}} {{{{ {source} }}}} {{% endif %}} {{% endmacro %}}' + name_sql[name] = sql + + for name, sql in name_sql.items(): + pm = ParsedMacro( + name=name, + resource_type=NodeType.Macro, + unique_id=f'macro.{package}.{name}', + package_name=package, + original_file_path=normalize('macros/macro.sql'), + root_path='./dbt_modules/root', + path=normalize('macros/macro.sql'), + macro_sql=sql, + ) + yield pm + + +class TestAdapterConversions(TestCase): + def _get_tester_for(self, column_type): + from dbt.clients import agate_helper + if column_type is agate.TimeDelta: # dbt never makes this! + return agate.TimeDelta() + + for instance in agate_helper.DEFAULT_TYPE_TESTER._possible_types: + if type(instance) is column_type: + return instance + + raise ValueError(f'no tester for {column_type}') + + def _make_table_of(self, rows, column_types): + column_names = list(string.ascii_letters[:len(rows[0])]) + if isinstance(column_types, type): + column_types = [self._get_tester_for(column_types) for _ in column_names] + else: + column_types = [self._get_tester_for(typ) for typ in column_types] + table = agate.Table(rows, column_names=column_names, column_types=column_types) + return table + + +def MockMacro(package, name='my_macro', **kwargs): + from dbt.contracts.graph.parsed import ParsedMacro + from dbt.node_types import NodeType + + mock_kwargs = dict( + resource_type=NodeType.Macro, + package_name=package, + unique_id=f'macro.{package}.{name}', + original_file_path='/dev/null', + ) + + mock_kwargs.update(kwargs) + + macro = mock.MagicMock( + spec=ParsedMacro, + **mock_kwargs + ) + macro.name = name + return macro + + +def MockMaterialization(package, name='my_materialization', adapter_type=None, **kwargs): + if adapter_type is None: + adapter_type = 'default' + kwargs['adapter_type'] = adapter_type + return MockMacro(package, f'materialization_{name}_{adapter_type}', **kwargs) + + +def MockGenerateMacro(package, component='some_component', **kwargs): + name = f'generate_{component}_name' + return MockMacro(package, name=name, **kwargs) + + +def MockSource(package, source_name, name, **kwargs): + from dbt.node_types import NodeType + from dbt.contracts.graph.parsed import ParsedSourceDefinition + src = mock.MagicMock( + __class__=ParsedSourceDefinition, + resource_type=NodeType.Source, + source_name=source_name, + package_name=package, + unique_id=f'source.{package}.{source_name}.{name}', + search_name=f'{source_name}.{name}', + **kwargs + ) + src.name = name + return src + + +def MockNode(package, name, resource_type=None, **kwargs): + from dbt.node_types import NodeType + from dbt.contracts.graph.parsed import ParsedModelNode, ParsedSeedNode + if resource_type is None: + resource_type = NodeType.Model + if resource_type == NodeType.Model: + cls = ParsedModelNode + elif resource_type == NodeType.Seed: + cls = ParsedSeedNode + else: + raise ValueError(f'I do not know how to handle {resource_type}') + node = mock.MagicMock( + __class__=cls, + resource_type=resource_type, + package_name=package, + unique_id=f'{str(resource_type)}.{package}.{name}', + search_name=name, + **kwargs + ) + node.name = name + return node + + +def MockDocumentation(package, name, **kwargs): + from dbt.node_types import NodeType + from dbt.contracts.graph.parsed import ParsedDocumentation + doc = mock.MagicMock( + __class__=ParsedDocumentation, + resource_type=NodeType.Documentation, + package_name=package, + search_name=name, + unique_id=f'{package}.{name}', + **kwargs + ) + doc.name = name + return doc + + +def load_internal_manifest_macros(config, macro_hook = lambda m: None): + from dbt.parser.manifest import ManifestLoader + return ManifestLoader.load_macros(config, macro_hook) + + + +def dict_replace(dct, **kwargs): + dct = dct.copy() + dct.update(kwargs) + return dct + + +def replace_config(n, **kwargs): + return n.replace( + config=n.config.replace(**kwargs), + unrendered_config=dict_replace(n.unrendered_config, **kwargs), + ) + From 8be1ef7d096fd6e693c257e6540cc33a7ad5d39b Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Wed, 29 Sep 2021 12:01:19 +0200 Subject: [PATCH 373/860] Update README image, copy --- README.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 7a1b5f509..9c459bac2 100644 --- a/README.md +++ b/README.md @@ -1,8 +1,8 @@

- dbt logo + dbt logo

-**[dbt](https://www.getdbt.com/)** (data build tool) enables data analysts and engineers to transform their data using the same practices that software engineers use to build applications. +**[dbt](https://www.getdbt.com/)** enables data analysts and engineers to transform their data using the same practices that software engineers use to build applications. dbt is the T in ELT. Organize, cleanse, denormalize, filter, rename, and pre-aggregate the raw data in your warehouse so that it's ready for analysis. @@ -24,7 +24,7 @@ more information on using dbt with BigQuery, consult [the docs](https://docs.get ## Reporting bugs and contributing code -- Want to report a bug or request a feature? Let us know on [Slack](http://community.getdbt.com/), or open [an issue](https://github.com/dbt-labs/dbt/issues/new). +- Want to report a bug or request a feature? Let us know on [Slack](http://community.getdbt.com/), or open [an issue](https://github.com/dbt-labs/dbt-bigquery/issues/new). - Want to help us build dbt? Check out the [Contributing Getting Started Guide](https://github.com/dbt-labs/dbt/blob/HEAD/CONTRIBUTING.md) ## Code of Conduct From ada859a99f050277255a3139482a2bf307ebcf0b Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Wed, 29 Sep 2021 12:02:24 +0200 Subject: [PATCH 374/860] Switch back hard-coded test paths --- tests/integration/base.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/base.py b/tests/integration/base.py index 2f4c0fd35..4017bf177 100644 --- a/tests/integration/base.py +++ b/tests/integration/base.py @@ -114,7 +114,7 @@ def _pytest_get_test_root(): path_parts.append(tail) path_parts.reverse() # dbt tests are all of the form 'tests/integration/suite_name' - target = os.path.join(*path_parts[:5]) # TODO: try to not hard code this + target = os.path.join(*path_parts[:3]) # TODO: try to not hard code this return os.path.join(relative_to, target) From ce42f889e4136dd71a5bb01ae98bc871d96602d6 Mon Sep 17 00:00:00 2001 From: Ian Knox Date: Thu, 30 Sep 2021 13:50:23 -0500 Subject: [PATCH 375/860] issue and PR templates --- .github/ISSUE_TEMPLATE/bug_report.md | 33 +++++++++++++++++++ .github/ISSUE_TEMPLATE/feature_request.md | 23 +++++++++++++ .github/ISSUE_TEMPLATE/release.md | 10 ++++++ .../pull_request_template.md | 21 ++++++++++++ 4 files changed, 87 insertions(+) create mode 100644 .github/ISSUE_TEMPLATE/bug_report.md create mode 100644 .github/ISSUE_TEMPLATE/feature_request.md create mode 100644 .github/ISSUE_TEMPLATE/release.md create mode 100644 .github/PULL_REQUEST_TEMPLATE/pull_request_template.md diff --git a/.github/ISSUE_TEMPLATE/bug_report.md b/.github/ISSUE_TEMPLATE/bug_report.md new file mode 100644 index 000000000..c2cb2a061 --- /dev/null +++ b/.github/ISSUE_TEMPLATE/bug_report.md @@ -0,0 +1,33 @@ +--- +name: Bug report +about: Report a bug or an issue you've found with dbt-bigquery +title: '' +labels: bug, triage +assignees: '' + +--- + +### Describe the bug +A clear and concise description of what the bug is. What command did you run? What happened? + +### 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 +A clear and concise description of what you expected to happen. + +### Screenshots and log output +If applicable, add screenshots or log output to help explain your problem. + +### System information +**The output of `dbt --version`:** +``` + +``` + +**The operating system you're using:** + +**The output of `python --version`:** + +### Additional context +Add any other context about the problem here. diff --git a/.github/ISSUE_TEMPLATE/feature_request.md b/.github/ISSUE_TEMPLATE/feature_request.md new file mode 100644 index 000000000..5c0a9545f --- /dev/null +++ b/.github/ISSUE_TEMPLATE/feature_request.md @@ -0,0 +1,23 @@ +--- +name: Feature request +about: Suggest an idea for dbt-bigquery +title: '' +labels: enhancement, triage +assignees: '' + +--- + +### Describe the feature +A clear and concise description of what you want to happen. + +### Describe alternatives you've considered +A clear and concise description of any alternative solutions or features you've considered. + +### Additional context +Please include any other relevant context here. + +### Who will this benefit? +What kind of use case will this feature be useful for? Please be specific and provide examples, this will help us prioritize properly. + +### Are you interested in contributing this feature? +Let us know if you want to write some code, and how we can help. diff --git a/.github/ISSUE_TEMPLATE/release.md b/.github/ISSUE_TEMPLATE/release.md new file mode 100644 index 000000000..c3cb05c3c --- /dev/null +++ b/.github/ISSUE_TEMPLATE/release.md @@ -0,0 +1,10 @@ +--- +name: Release +about: Release a new version of dbt-bigquery +title: '' +labels: release +assignees: '' + +--- + +### TBD diff --git a/.github/PULL_REQUEST_TEMPLATE/pull_request_template.md b/.github/PULL_REQUEST_TEMPLATE/pull_request_template.md new file mode 100644 index 000000000..5e7cc23d3 --- /dev/null +++ b/.github/PULL_REQUEST_TEMPLATE/pull_request_template.md @@ -0,0 +1,21 @@ +resolves # + + + +### Description + + + +### Checklist + +- [ ] I have signed the [CLA](https://docs.getdbt.com/docs/contributor-license-agreements) +- [ ] I have run this code in development and it appears to resolve the stated issue +- [ ] This PR includes tests, or tests are not required/relevant for this PR +- [ ] I have updated the `CHANGELOG.md` and added information about my change to the "dbt-redshift next" section. \ No newline at end of file From d2da0f40ae997506074952cf58b2f0e2def4d308 Mon Sep 17 00:00:00 2001 From: Ian Knox Date: Thu, 30 Sep 2021 13:52:27 -0500 Subject: [PATCH 376/860] changelog, license and setup --- CHANGELOG.md | 10 +++ LICENSE.md | 201 +++++++++++++++++++++++++++++++++++++++++++++++++++ setup.py | 2 +- 3 files changed, 212 insertions(+), 1 deletion(-) create mode 100644 CHANGELOG.md create mode 100644 LICENSE.md diff --git a/CHANGELOG.md b/CHANGELOG.md new file mode 100644 index 000000000..e78891e0a --- /dev/null +++ b/CHANGELOG.md @@ -0,0 +1,10 @@ +## dbt-bigquery 1.0.0 (Release TBD) + +### Features +N/A + +### Fixes +N/A + +### Under the hood +First version as a seperate repo. \ No newline at end of file diff --git a/LICENSE.md b/LICENSE.md new file mode 100644 index 000000000..536bebee0 --- /dev/null +++ b/LICENSE.md @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright 2021 dbt Labs, Inc. + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/setup.py b/setup.py index 038033f5d..a60a9a69c 100644 --- a/setup.py +++ b/setup.py @@ -35,7 +35,7 @@ long_description_content_type='text/markdown', author="dbt Labs", author_email="info@dbtlabs.com", - url="https://github.com/dbt-labs/dbt", + url="https://github.com/dbt-labs/dbt-bigquery", packages=find_namespace_packages(include=['dbt', 'dbt.*']), package_data={ 'dbt': [ From c0e044f657036035cf538324aaec46cd98448506 Mon Sep 17 00:00:00 2001 From: Ian Knox <81931810+iknox-fa@users.noreply.github.com> Date: Fri, 1 Oct 2021 09:28:46 -0500 Subject: [PATCH 377/860] Update .github/PULL_REQUEST_TEMPLATE/pull_request_template.md Co-authored-by: Jeremy Cohen --- .github/PULL_REQUEST_TEMPLATE/pull_request_template.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/PULL_REQUEST_TEMPLATE/pull_request_template.md b/.github/PULL_REQUEST_TEMPLATE/pull_request_template.md index 5e7cc23d3..0f793f232 100644 --- a/.github/PULL_REQUEST_TEMPLATE/pull_request_template.md +++ b/.github/PULL_REQUEST_TEMPLATE/pull_request_template.md @@ -18,4 +18,4 @@ resolves # - [ ] I have signed the [CLA](https://docs.getdbt.com/docs/contributor-license-agreements) - [ ] I have run this code in development and it appears to resolve the stated issue - [ ] This PR includes tests, or tests are not required/relevant for this PR -- [ ] I have updated the `CHANGELOG.md` and added information about my change to the "dbt-redshift next" section. \ No newline at end of file +- [ ] I have updated the `CHANGELOG.md` and added information about my change to the "dbt-bigquery next" section. \ No newline at end of file From f123101245421173cc60d609fec7f2b09118e39f Mon Sep 17 00:00:00 2001 From: Jeremy Cohen Date: Sat, 2 Oct 2021 10:24:03 -0700 Subject: [PATCH 378/860] Plugin setup (#7) * setup.py, tox, GHA * Pull changes from main repo * Switch workflow trigger temporarily * Switch trigger back --- .bumpversion.cfg | 29 +++ .github/dependabot.yml | 8 + .../pull_request_template.md | 0 .github/scripts/integration-test-matrix.js | 95 ++++++++ .github/workflows/integration.yml | 202 +++++++++++++++++ .github/workflows/main.yml | 206 ++++++++++++++++++ .gitignore | 96 ++++++++ MANIFEST.in | 1 + README.md | 27 ++- dev_requirements.txt | 20 ++ scripts/build-dist.sh | 20 ++ setup.py | 53 +++-- tests/integration/base.py | 2 - .../integration/dbt_debug_test/test_debug.py | 4 + .../adapter-macro-macros/macros.sql | 17 -- .../adapter-macro-models-package/model.sql | 4 - .../adapter-macro-models/model.sql | 4 - .../deprecation_tests/test_deprecations.py | 62 ------ .../docs_generate_tests/test_docs_generate.py | 39 ++-- .../test_incremental_schema.py | 8 +- .../query_comments_test/models/x.sql | 3 +- .../test_simple_snapshot.py | 17 +- .../test_snapshot_check_cols.py | 2 +- tox.ini | 38 ++++ 24 files changed, 807 insertions(+), 150 deletions(-) create mode 100644 .bumpversion.cfg create mode 100644 .github/dependabot.yml rename .github/{PULL_REQUEST_TEMPLATE => }/pull_request_template.md (100%) create mode 100644 .github/scripts/integration-test-matrix.js create mode 100644 .github/workflows/integration.yml create mode 100644 .github/workflows/main.yml create mode 100644 .gitignore create mode 100644 MANIFEST.in create mode 100644 dev_requirements.txt create mode 100755 scripts/build-dist.sh delete mode 100644 tests/integration/deprecation_tests/adapter-macro-macros/macros.sql delete mode 100644 tests/integration/deprecation_tests/adapter-macro-models-package/model.sql delete mode 100644 tests/integration/deprecation_tests/adapter-macro-models/model.sql delete mode 100644 tests/integration/deprecation_tests/test_deprecations.py create mode 100644 tox.ini diff --git a/.bumpversion.cfg b/.bumpversion.cfg new file mode 100644 index 000000000..816a1a7ba --- /dev/null +++ b/.bumpversion.cfg @@ -0,0 +1,29 @@ +[bumpversion] +current_version = 0.21.0rc2 +parse = (?P\d+) +\.(?P\d+) +\.(?P\d+) +(\.(?P\d+))? +((?P[a-z]+)(?P\d+))? +serialize = +{major}.{minor}.{patch}.{pluginpatch}{prerelease}{num} +{major}.{minor}.{patch}{prerelease}{num} +{major}.{minor}.{patch}.{pluginpatch} +{major}.{minor}.{patch} +commit = False +tag = False + +[bumpversion:part:prerelease] +first_value = a +values = +a +b +rc + +[bumpversion:part:num] +first_value = 1 + +[bumpversion:part:pluginpatch] +first_value = 1 + +[bumpversion:file:dbt/adapters/bigquery/__version__.py] diff --git a/.github/dependabot.yml b/.github/dependabot.yml new file mode 100644 index 000000000..dea04a2fa --- /dev/null +++ b/.github/dependabot.yml @@ -0,0 +1,8 @@ +version: 2 +updates: + # python dependencies + - package-ecosystem: "pip" + directory: "/dbt" + schedule: + interval: "daily" + rebase-strategy: "disabled" diff --git a/.github/PULL_REQUEST_TEMPLATE/pull_request_template.md b/.github/pull_request_template.md similarity index 100% rename from .github/PULL_REQUEST_TEMPLATE/pull_request_template.md rename to .github/pull_request_template.md diff --git a/.github/scripts/integration-test-matrix.js b/.github/scripts/integration-test-matrix.js new file mode 100644 index 000000000..ae1768bf7 --- /dev/null +++ b/.github/scripts/integration-test-matrix.js @@ -0,0 +1,95 @@ +module.exports = ({ context }) => { + const defaultPythonVersion = "3.8"; + const supportedPythonVersions = ["3.6", "3.7", "3.8", "3.9"]; + const supportedAdapters = ["bigquery"]; + + // if PR, generate matrix based on files changed and PR labels + if (context.eventName.includes("pull_request")) { + // `changes` is a list of adapter names that have related + // file changes in the PR + // ex: ['postgres', 'snowflake'] + const changes = JSON.parse(process.env.CHANGES); + const labels = context.payload.pull_request.labels.map(({ name }) => name); + console.log("labels", labels); + console.log("changes", changes); + const testAllLabel = labels.includes("test all"); + const include = []; + + for (const adapter of supportedAdapters) { + if ( + changes.includes(adapter) || + testAllLabel || + labels.includes(`test ${adapter}`) + ) { + for (const pythonVersion of supportedPythonVersions) { + if ( + pythonVersion === defaultPythonVersion || + labels.includes(`test python${pythonVersion}`) || + testAllLabel + ) { + // always run tests on ubuntu by default + include.push({ + os: "ubuntu-latest", + adapter, + "python-version": pythonVersion, + }); + + if (labels.includes("test windows") || testAllLabel) { + include.push({ + os: "windows-latest", + adapter, + "python-version": pythonVersion, + }); + } + + if (labels.includes("test macos") || testAllLabel) { + include.push({ + os: "macos-latest", + adapter, + "python-version": pythonVersion, + }); + } + } + } + } + } + + console.log("matrix", { include }); + + return { + include, + }; + } + // if not PR, generate matrix of python version, adapter, and operating + // system to run integration tests on + + const include = []; + // run for all adapters and python versions on ubuntu + for (const adapter of supportedAdapters) { + for (const pythonVersion of supportedPythonVersions) { + include.push({ + os: 'ubuntu-latest', + adapter: adapter, + "python-version": pythonVersion, + }); + } + } + + // additionally include runs for all adapters, on macos and windows, + // but only for the default python version + for (const adapter of supportedAdapters) { + for (const operatingSystem of ["windows-latest", "macos-latest"]) { + include.push({ + os: operatingSystem, + adapter: adapter, + "python-version": defaultPythonVersion, + }); + } + } + + console.log("matrix", { include }); + + return { + include, + }; +}; diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml new file mode 100644 index 000000000..a7b68ff15 --- /dev/null +++ b/.github/workflows/integration.yml @@ -0,0 +1,202 @@ +# **what?** +# This workflow runs all integration tests for supported OS +# and python versions and core adapters. If triggered by PR, +# the workflow will only run tests for adapters related +# to code changes. Use the `test all` and `test ${adapter}` +# label to run all or additional tests. Use `ok to test` +# label to mark PRs from forked repositories that are safe +# to run integration tests for. Requires secrets to run +# against different warehouses. + +# **why?** +# This checks the functionality of dbt from a user's perspective +# and attempts to catch functional regressions. + +# **when?** +# This workflow will run on every push to a protected branch +# and when manually triggered. It will also run for all PRs, including +# PRs from forks. The workflow will be skipped until there is a label +# to mark the PR as safe to run. + +name: Adapter Integration Tests + +on: + # pushes to release branches + push: + branches: + - "main" + - "develop" + - "*.latest" + - "releases/*" + # all PRs, important to note that `pull_request_target` workflows + # will run in the context of the target branch of a PR + pull_request_target: + # manual trigger + workflow_dispatch: + +# explicitly turn off permissions for `GITHUB_TOKEN` +permissions: read-all + +# will cancel previous workflows triggered by the same event and for the same ref for PRs or same SHA otherwise +concurrency: + group: ${{ github.workflow }}-${{ github.event_name }}-${{ contains(github.event_name, 'pull_request') && github.event.pull_request.head.ref || github.sha }} + cancel-in-progress: true + +# sets default shell to bash, for all operating systems +defaults: + run: + shell: bash + +jobs: + # generate test metadata about what files changed and the testing matrix to use + test-metadata: + # run if not a PR from a forked repository or has a label to mark as safe to test + if: >- + github.event_name != 'pull_request_target' || + github.event.pull_request.head.repo.full_name == github.repository || + contains(github.event.pull_request.labels.*.name, 'ok to test') + runs-on: ubuntu-latest + + outputs: + matrix: ${{ steps.generate-matrix.outputs.result }} + + steps: + - name: Check out the repository (non-PR) + if: github.event_name != 'pull_request_target' + uses: actions/checkout@v2 + with: + persist-credentials: false + + - name: Check out the repository (PR) + if: github.event_name == 'pull_request_target' + uses: actions/checkout@v2 + with: + persist-credentials: false + ref: ${{ github.event.pull_request.head.sha }} + + - name: Check if relevant files changed + # https://github.com/marketplace/actions/paths-changes-filter + # For each filter, it sets output variable named by the filter to the text: + # 'true' - if any of changed files matches any of filter rules + # 'false' - if none of changed files matches any of filter rules + # also, returns: + # `changes` - JSON array with names of all filters matching any of the changed files + uses: dorny/paths-filter@v2 + id: get-changes + with: + token: ${{ secrets.GITHUB_TOKEN }} + filters: | + bigquery: + - 'dbt/**' + - 'tests/**' + - name: Generate integration test matrix + id: generate-matrix + uses: actions/github-script@v4 + env: + CHANGES: ${{ steps.get-changes.outputs.changes }} + with: + script: | + const script = require('./.github/scripts/integration-test-matrix.js') + const matrix = script({ context }) + console.log(matrix) + return matrix + test: + name: ${{ matrix.adapter }} / python ${{ matrix.python-version }} / ${{ matrix.os }} + + # run if not a PR from a forked repository or has a label to mark as safe to test + # also checks that the matrix generated is not empty + if: >- + needs.test-metadata.outputs.matrix && + fromJSON( needs.test-metadata.outputs.matrix ).include[0] && + ( + github.event_name != 'pull_request_target' || + github.event.pull_request.head.repo.full_name == github.repository || + contains(github.event.pull_request.labels.*.name, 'ok to test') + ) + runs-on: ${{ matrix.os }} + + needs: test-metadata + + strategy: + fail-fast: false + matrix: ${{ fromJSON(needs.test-metadata.outputs.matrix) }} + + env: + TOXENV: integration-${{ matrix.adapter }} + PYTEST_ADDOPTS: "-v --color=yes -n4 --csv integration_results.csv" + DBT_INVOCATION_ENV: github-actions + + steps: + - name: Check out the repository + if: github.event_name != 'pull_request_target' + uses: actions/checkout@v2 + with: + persist-credentials: false + + # explicity checkout the branch for the PR, + # this is necessary for the `pull_request_target` event + - name: Check out the repository (PR) + if: github.event_name == 'pull_request_target' + uses: actions/checkout@v2 + with: + persist-credentials: false + ref: ${{ github.event.pull_request.head.sha }} + + - name: Set up Python ${{ matrix.python-version }} + uses: actions/setup-python@v2 + with: + python-version: ${{ matrix.python-version }} + + - name: Install python dependencies + run: | + pip install --upgrade pip + pip install tox + pip --version + tox --version + + - name: Run tox (bigquery) + if: matrix.adapter == 'bigquery' + env: + BIGQUERY_TEST_SERVICE_ACCOUNT_JSON: ${{ secrets.BIGQUERY_TEST_SERVICE_ACCOUNT_JSON }} + BIGQUERY_TEST_ALT_DATABASE: ${{ secrets.BIGQUERY_TEST_ALT_DATABASE }} + run: tox + + - uses: actions/upload-artifact@v2 + if: always() + with: + name: logs + path: ./logs + + - name: Get current date + if: always() + id: date + run: echo "::set-output name=date::$(date +'%Y-%m-%dT%H_%M_%S')" #no colons allowed for artifacts + + - uses: actions/upload-artifact@v2 + if: always() + with: + name: integration_results_${{ matrix.python-version }}_${{ matrix.os }}_${{ matrix.adapter }}-${{ steps.date.outputs.date }}.csv + path: integration_results.csv + + require-label-comment: + runs-on: ubuntu-latest + + needs: test + + permissions: + pull-requests: write + + steps: + - name: Needs permission PR comment + if: >- + needs.test.result == 'skipped' && + github.event_name == 'pull_request_target' && + github.event.pull_request.head.repo.full_name != github.repository + uses: unsplash/comment-on-pr@master + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + with: + msg: | + "You do not have permissions to run integration tests, @dbt-labs/core "\ + "needs to label this PR with `ok to test` in order to run integration tests!" + check_for_duplicate_msg: true diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml new file mode 100644 index 000000000..84ff1eabd --- /dev/null +++ b/.github/workflows/main.yml @@ -0,0 +1,206 @@ +# **what?** +# Runs code quality checks, unit tests, and verifies python build on +# all code commited to the repository. This workflow should not +# require any secrets since it runs for PRs from forked repos. +# By default, secrets are not passed to workflows running from +# a forked repo. + +# **why?** +# Ensure code for dbt meets a certain quality standard. + +# **when?** +# This will run for all PRs, when code is pushed to a release +# branch, and when manually triggered. + +name: Tests and Code Checks + +on: + push: + branches: + - "main" + - "develop" + - "*.latest" + - "releases/*" + pull_request: + workflow_dispatch: + +permissions: read-all + +# will cancel previous workflows triggered by the same event and for the same ref for PRs or same SHA otherwise +concurrency: + group: ${{ github.workflow }}-${{ github.event_name }}-${{ contains(github.event_name, 'pull_request') && github.event.pull_request.head.ref || github.sha }} + cancel-in-progress: true + +defaults: + run: + shell: bash + +jobs: + code-quality: + name: ${{ matrix.toxenv }} + + runs-on: ubuntu-latest + + strategy: + fail-fast: false + matrix: + toxenv: [flake8] + + env: + TOXENV: ${{ matrix.toxenv }} + PYTEST_ADDOPTS: "-v --color=yes" + + steps: + - name: Check out the repository + uses: actions/checkout@v2 + with: + persist-credentials: false + + - name: Set up Python + uses: actions/setup-python@v2 + + - name: Install python dependencies + run: | + pip install --upgrade pip + pip install tox + pip --version + tox --version + + - name: Run tox + run: tox + + unit: + name: unit test / python ${{ matrix.python-version }} + + runs-on: ubuntu-latest + + strategy: + fail-fast: false + matrix: + python-version: [3.6, 3.7, 3.8] # TODO: support unit testing for python 3.9 (https://github.com/dbt-labs/dbt/issues/3689) + + env: + TOXENV: "unit" + PYTEST_ADDOPTS: "-v --color=yes --csv unit_results.csv" + + steps: + - name: Check out the repository + uses: actions/checkout@v2 + with: + persist-credentials: false + + - name: Set up Python ${{ matrix.python-version }} + uses: actions/setup-python@v2 + with: + python-version: ${{ matrix.python-version }} + + - name: Install python dependencies + run: | + pip install --upgrade pip + pip install tox + pip --version + tox --version + + - name: Run tox + run: tox + + - name: Get current date + if: always() + id: date + run: echo "::set-output name=date::$(date +'%Y-%m-%dT%H_%M_%S')" #no colons allowed for artifacts + + - uses: actions/upload-artifact@v2 + if: always() + with: + name: unit_results_${{ matrix.python-version }}-${{ steps.date.outputs.date }}.csv + path: unit_results.csv + + build: + name: build packages + + runs-on: ubuntu-latest + + steps: + - name: Check out the repository + uses: actions/checkout@v2 + with: + persist-credentials: false + + - name: Set up Python + uses: actions/setup-python@v2 + with: + python-version: 3.8 + + - name: Install python dependencies + run: | + pip install --upgrade pip + pip install --upgrade setuptools wheel twine check-wheel-contents + pip --version + + - name: Build distributions + run: ./scripts/build-dist.sh + + - name: Show distributions + run: ls -lh dist/ + + - name: Check distribution descriptions + run: | + twine check dist/* + + - name: Check wheel contents + run: | + check-wheel-contents dist/*.whl --ignore W007,W008 + + - uses: actions/upload-artifact@v2 + with: + name: dist + path: dist/ + + test-build: + name: verify packages / python ${{ matrix.python-version }} / ${{ matrix.os }} + + needs: build + + runs-on: ${{ matrix.os }} + + strategy: + fail-fast: false + matrix: + os: [ubuntu-latest, macos-latest, windows-latest] + python-version: [3.6, 3.7, 3.8, 3.9] + + steps: + - name: Set up Python ${{ matrix.python-version }} + uses: actions/setup-python@v2 + with: + python-version: ${{ matrix.python-version }} + + - name: Install python dependencies + run: | + pip install --upgrade pip + pip install --upgrade wheel + pip --version + + - uses: actions/download-artifact@v2 + with: + name: dist + path: dist/ + + - name: Show distributions + run: ls -lh dist/ + + - name: Install wheel distributions + run: | + find ./dist/*.whl -maxdepth 1 -type f | xargs pip install --force-reinstall --find-links=dist/ + + - name: Check wheel distributions + run: | + dbt --version + + - name: Install source distributions + run: | + find ./dist/*.gz -maxdepth 1 -type f | xargs pip install --force-reinstall --find-links=dist/ + + - name: Check source distributions + run: | + dbt --version diff --git a/.gitignore b/.gitignore new file mode 100644 index 000000000..43724b61e --- /dev/null +++ b/.gitignore @@ -0,0 +1,96 @@ +# Byte-compiled / optimized / DLL files +__pycache__/ +*.py[cod] +*$py.class + +# C extensions +*.so + +# Distribution / packaging +.Python +env*/ +dbt_env/ +build/ +develop-eggs/ +dist/ +downloads/ +eggs/ +.eggs/ +lib/ +lib64/ +parts/ +sdist/ +var/ +*.egg-info/ +.installed.cfg +*.egg +*.mypy_cache/ +logs/ + +# PyInstaller +# Usually these files are written by a python script from a template +# before PyInstaller builds the exe, so as to inject date/other infos into it. +*.manifest +*.spec + +# Installer logs +pip-log.txt +pip-delete-this-directory.txt + +# Unit test / coverage reports +htmlcov/ +.tox/ +.coverage +.coverage.* +.cache +.env +nosetests.xml +coverage.xml +*,cover +.hypothesis/ +test.env + +# Mypy +.mypy_cache/ + +# Translations +*.mo +*.pot + +# Django stuff: +*.log + +# Sphinx documentation +docs/_build/ + +# PyBuilder +target/ + +#Ipython Notebook +.ipynb_checkpoints + +#Emacs +*~ + +# Sublime Text +*.sublime-* + +# Vim +*.sw* + +.python-version + +# Vim +*.sw* + +# pycharm +.idea/ +venv/ + +# AWS credentials +.aws/ + +.DS_Store + +# vscode +.vscode/ diff --git a/MANIFEST.in b/MANIFEST.in new file mode 100644 index 000000000..78412d5b8 --- /dev/null +++ b/MANIFEST.in @@ -0,0 +1 @@ +recursive-include dbt/include *.sql *.yml *.md \ No newline at end of file diff --git a/README.md b/README.md index 9c459bac2..976bdf918 100644 --- a/README.md +++ b/README.md @@ -1,6 +1,14 @@

dbt logo

+

+ + Unit Tests Badge + + + Integration Tests Badge + +

**[dbt](https://www.getdbt.com/)** enables data analysts and engineers to transform their data using the same practices that software engineers use to build applications. @@ -8,24 +16,23 @@ dbt is the T in ELT. Organize, cleanse, denormalize, filter, rename, and pre-agg ## dbt-bigquery -The `dbt-bigquery` package contains all of the code required to make dbt operate on a BigQuery database. For +The `dbt-bigquery` package contains all of the code enabling dbt to work with Google BigQuery. For more information on using dbt with BigQuery, consult [the docs](https://docs.getdbt.com/docs/profile-bigquery). +## Getting started -## Find out more - -- Check out the [Introduction to dbt](https://docs.getdbt.com/docs/introduction/). -- Read the [dbt Viewpoint](https://docs.getdbt.com/docs/about/viewpoint/). +- [Install dbt](https://docs.getdbt.com/docs/installation) +- Read the [introduction](https://docs.getdbt.com/docs/introduction/) and [viewpoint](https://docs.getdbt.com/docs/about/viewpoint/) -## Join thousands of analysts in the dbt community +## Join the dbt Community -- Join the [chat](http://community.getdbt.com/) on Slack. -- Find community posts on [dbt Discourse](https://discourse.getdbt.com). +- Be part of the conversation in the [dbt Community Slack](http://community.getdbt.com/) +- Read more on the [dbt Community Discourse](https://discourse.getdbt.com) ## Reporting bugs and contributing code -- Want to report a bug or request a feature? Let us know on [Slack](http://community.getdbt.com/), or open [an issue](https://github.com/dbt-labs/dbt-bigquery/issues/new). -- Want to help us build dbt? Check out the [Contributing Getting Started Guide](https://github.com/dbt-labs/dbt/blob/HEAD/CONTRIBUTING.md) +- Want to report a bug or request a feature? Let us know on [Slack](http://community.getdbt.com/), or open [an issue](https://github.com/dbt-labs/dbt-bigquery/issues/new) +- Want to help us build dbt? Check out the [Contributing Guide](https://github.com/dbt-labs/dbt/blob/HEAD/CONTRIBUTING.md) ## Code of Conduct diff --git a/dev_requirements.txt b/dev_requirements.txt new file mode 100644 index 000000000..b4ba00597 --- /dev/null +++ b/dev_requirements.txt @@ -0,0 +1,20 @@ +# install latest changes in dbt-core +# TODO: how to automate switching from develop to version branches? +git+https://github.com/dbt-labs/dbt.git@develop#egg=dbt-core&subdirectory=core + +bumpversion +flake8 +flaky +freezegun==0.3.12 +ipdb +mypy==0.782 +pip-tools +pytest +pytest-dotenv +pytest-logbook +pytest-csv +pytest-xdist +pytz +tox>=3.13 +twine +wheel \ No newline at end of file diff --git a/scripts/build-dist.sh b/scripts/build-dist.sh new file mode 100755 index 000000000..65e6dbc97 --- /dev/null +++ b/scripts/build-dist.sh @@ -0,0 +1,20 @@ +#!/bin/bash + +set -eo pipefail + +DBT_PATH="$( cd "$(dirname "$0")/.." ; pwd -P )" + +PYTHON_BIN=${PYTHON_BIN:-python} + +echo "$PYTHON_BIN" + +set -x + +rm -rf "$DBT_PATH"/dist +rm -rf "$DBT_PATH"/build +mkdir -p "$DBT_PATH"/dist + +cd "$DBT_PATH" +$PYTHON_BIN setup.py sdist bdist_wheel + +set +x diff --git a/setup.py b/setup.py index a60a9a69c..c7b7aeef0 100644 --- a/setup.py +++ b/setup.py @@ -1,13 +1,16 @@ #!/usr/bin/env python import os import sys +import re +# require python 3.6 or newer if sys.version_info < (3, 6): print('Error: dbt does not support this version of Python.') print('Please upgrade to Python 3.6 or higher.') sys.exit(1) +# require version of setuptools that supports find_namespace_packages from setuptools import setup try: from setuptools import find_namespace_packages @@ -19,14 +22,45 @@ sys.exit(1) -package_name = "dbt-bigquery" -package_version = "0.21.0rc1" -description = """The bigquery adapter plugin for dbt (data build tool)""" - +# pull long description from README this_directory = os.path.abspath(os.path.dirname(__file__)) with open(os.path.join(this_directory, 'README.md')) as f: long_description = f.read() + +# get this package's version from dbt/adapters//__version__.py +def _get_plugin_version_dict(): + _version_path = os.path.join( + this_directory, 'dbt', 'adapters', 'bigquery', '__version__.py' + ) + _semver = r'''(?P\d+)\.(?P\d+)\.(?P\d+)''' + _pre = r'''((?Pa|b|rc)(?P
\d+))?'''
+    _version_pattern = fr'''version\s*=\s*["']{_semver}{_pre}["']'''
+    with open(_version_path) as f:
+        match = re.search(_version_pattern, f.read().strip())
+        if match is None:
+            raise ValueError(f'invalid version at {_version_path}')
+        return match.groupdict()
+
+
+def _get_plugin_version():
+    parts = _get_plugin_version_dict()
+    return "{major}.{minor}.{patch}{prekind}{pre}".format(**parts)
+
+
+# require a compatible minor version (~=), prerelease if this is a prerelease
+def _get_dbt_core_version():
+    parts = _get_plugin_version_dict()
+    minor = "{major}.{minor}.0".format(**parts)
+    pre = (parts["prekind"]+"1" if parts["prekind"] else "")
+    return f"{minor}{pre}"
+
+
+package_name = "dbt-bigquery"
+package_version = _get_plugin_version()
+dbt_core_version = _get_dbt_core_version()
+description = """The BigQuery adapter plugin for dbt"""
+
 setup(
     name=package_name,
     version=package_version,
@@ -37,16 +71,9 @@
     author_email="info@dbtlabs.com",
     url="https://github.com/dbt-labs/dbt-bigquery",
     packages=find_namespace_packages(include=['dbt', 'dbt.*']),
-    package_data={
-        'dbt': [
-            'include/bigquery/dbt_project.yml',
-            'include/bigquery/sample_profiles.yml',
-            'include/bigquery/macros/*.sql',
-            'include/bigquery/macros/**/*.sql',
-        ]
-    },
+    include_package_data=True,
     install_requires=[
-        'dbt-core=={}'.format(package_version),
+        'dbt-core~={}'.format(dbt_core_version),
         'protobuf>=3.13.0,<4',
         'google-cloud-core>=1.3.0,<2',
         'google-cloud-bigquery>=1.25.0,<3',
diff --git a/tests/integration/base.py b/tests/integration/base.py
index 4017bf177..acb0eaec0 100644
--- a/tests/integration/base.py
+++ b/tests/integration/base.py
@@ -59,9 +59,7 @@ def __eq__(self, other):
 class FakeArgs:
     def __init__(self):
         self.threads = 1
-        self.data = False
         self.defer = False
-        self.schema = True
         self.full_refresh = False
         self.models = None
         self.select = None
diff --git a/tests/integration/dbt_debug_test/test_debug.py b/tests/integration/dbt_debug_test/test_debug.py
index 7e346f56e..f057c72a7 100644
--- a/tests/integration/dbt_debug_test/test_debug.py
+++ b/tests/integration/dbt_debug_test/test_debug.py
@@ -19,6 +19,10 @@ def dir(value):
     def models(self):
         return self.dir('models')
 
+    @pytest.fixture(autouse=True)
+    def capsys(self, capsys):
+        self.capsys = capsys
+
     @use_profile('bigquery')
     def test_bigquery_ok(self):
         self.run_dbt(['debug'])
diff --git a/tests/integration/deprecation_tests/adapter-macro-macros/macros.sql b/tests/integration/deprecation_tests/adapter-macro-macros/macros.sql
deleted file mode 100644
index b03ca28f1..000000000
--- a/tests/integration/deprecation_tests/adapter-macro-macros/macros.sql
+++ /dev/null
@@ -1,17 +0,0 @@
-{% macro some_macro(arg1, arg2) -%}
-    {{ adapter_macro('some_macro', arg1, arg2) }}
-{%- endmacro %}
-
-
-{% macro default__some_macro(arg1, arg2) %}
-    {% do exceptions.raise_compiler_error('not allowed') %}
-{% endmacro %}
-
-{% macro postgres__some_macro(arg1, arg2) -%}
-    {{ arg1 }}{{ arg2 }}
-{%- endmacro %}
-
-
-{% macro some_other_macro(arg1, arg2) -%}
-	{{ adapter_macro('test.some_macro', arg1, arg2) }}
-{%- endmacro %}
diff --git a/tests/integration/deprecation_tests/adapter-macro-models-package/model.sql b/tests/integration/deprecation_tests/adapter-macro-models-package/model.sql
deleted file mode 100644
index 7b86017b4..000000000
--- a/tests/integration/deprecation_tests/adapter-macro-models-package/model.sql
+++ /dev/null
@@ -1,4 +0,0 @@
-{% if some_other_macro('foo', 'bar') != 'foobar' %}
-  {% do exceptions.raise_compiler_error('invalid foobar') %}
-{% endif %}
-select 1 as id
diff --git a/tests/integration/deprecation_tests/adapter-macro-models/model.sql b/tests/integration/deprecation_tests/adapter-macro-models/model.sql
deleted file mode 100644
index 37621d3d2..000000000
--- a/tests/integration/deprecation_tests/adapter-macro-models/model.sql
+++ /dev/null
@@ -1,4 +0,0 @@
-{% if some_macro('foo', 'bar') != 'foobar' %}
-  {% do exceptions.raise_compiler_error('invalid foobar') %}
-{% endif %}
-select 1 as id
diff --git a/tests/integration/deprecation_tests/test_deprecations.py b/tests/integration/deprecation_tests/test_deprecations.py
deleted file mode 100644
index cf394a34a..000000000
--- a/tests/integration/deprecation_tests/test_deprecations.py
+++ /dev/null
@@ -1,62 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-from dbt import deprecations
-import dbt.exceptions
-
-
-class BaseTestDeprecations(DBTIntegrationTest):
-    def setUp(self):
-        super().setUp()
-        deprecations.reset_deprecations()
-
-    @property
-    def schema(self):
-        return "deprecation_test"
-
-    @staticmethod
-    def dir(path):
-        return path.lstrip("/")
-
-
-class TestAdapterMacroDeprecation(BaseTestDeprecations):
-    @property
-    def models(self):
-        return self.dir('adapter-macro-models')
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'macro-paths': [self.dir('adapter-macro-macros')]
-        }
-
-    @use_profile('bigquery')
-    def test_bigquery_adapter_macro(self):
-        self.assertEqual(deprecations.active_deprecations, set())
-        # picked up the default -> error
-        with self.assertRaises(dbt.exceptions.CompilationException) as exc:
-            self.run_dbt(expect_pass=False)
-        exc_str = ' '.join(str(exc.exception).split())  # flatten all whitespace
-        assert 'not allowed' in exc_str  # we saw the default macro
-
-
-class TestAdapterMacroDeprecationPackages(BaseTestDeprecations):
-    @property
-    def models(self):
-        return self.dir('adapter-macro-models-package')
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'macro-paths': [self.dir('adapter-macro-macros')]
-        }
-
-    @use_profile('bigquery')
-    def test_bigquery_adapter_macro_pkg(self):
-        self.assertEqual(deprecations.active_deprecations, set())
-        # picked up the default -> error
-        with self.assertRaises(dbt.exceptions.CompilationException) as exc:
-            self.run_dbt(expect_pass=False)
-        exc_str = ' '.join(str(exc.exception).split())  # flatten all whitespace
-        assert 'not allowed' in exc_str  # we saw the default macro
diff --git a/tests/integration/docs_generate_tests/test_docs_generate.py b/tests/integration/docs_generate_tests/test_docs_generate.py
index a791897ad..c153c6bc3 100644
--- a/tests/integration/docs_generate_tests/test_docs_generate.py
+++ b/tests/integration/docs_generate_tests/test_docs_generate.py
@@ -67,7 +67,7 @@ class TestDocsGenerateEscapes(DBTIntegrationTest):
 
     @property
     def schema(self):
-        return 'docs_generate_029'
+        return 'docs_generate'
 
     @staticmethod
     def dir(path):
@@ -94,7 +94,6 @@ def setUp(self):
         super().setUp()
         self.maxDiff = None
         self.alternate_schema = self.unique_schema() + '_test'
-        self.alternate_schema = self.alternate_schema.upper()
 
         self._created_schemas.add(self.alternate_schema)
         os.environ['DBT_ENV_CUSTOM_ENV_env_key'] = 'env_value'
@@ -105,7 +104,7 @@ def tearDown(self):
 
     @property
     def schema(self):
-        return 'docs_generate_029'
+        return 'docs_generate'
 
     @staticmethod
     def dir(path):
@@ -958,7 +957,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
                 },
                 'test.test.not_null_model_id.d01cc630e6': {
                     'alias': 'not_null_model_id',
-                    'compiled_path': Normalized('target/compiled/test/models/schema.yml/schema_test/not_null_model_id.sql'),
+                    'compiled_path': Normalized('target/compiled/test/models/schema.yml/not_null_model_id.sql'),
                     'build_path': None,
                     'created_at': ANY,
                     'column_name': 'id',
@@ -971,20 +970,20 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
                     },
                     'deferred': False,
                     'description': '',
-                    'fqn': ['test', 'schema_test', 'not_null_model_id'],
+                    'fqn': ['test', 'not_null_model_id'],
                     'name': 'not_null_model_id',
                     'original_file_path': model_schema_yml_path,
                     'package_name': 'test',
                     'patch_path': None,
-                    'path': Normalized('schema_test/not_null_model_id.sql'),
-                    'raw_sql': "{{ test_not_null(**_dbt_schema_test_kwargs) }}",
+                    'path': Normalized('not_null_model_id.sql'),
+                    'raw_sql': "{{ test_not_null(**_dbt_generic_test_kwargs) }}",
                     'refs': [['model']],
                     'relation_name': None,
                     'resource_type': 'test',
                     'root_path': self.test_root_realpath,
                     'schema': test_audit_schema,
                     'database': self.default_database,
-                    'tags': ['schema'],
+                    'tags': [],
                     'meta': {},
                     'unique_id': 'test.test.not_null_model_id.d01cc630e6',
                     'docs': {'show': True},
@@ -1049,7 +1048,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
                 },
                 'test.test.test_nothing_model_.5d38568946': {
                     'alias': 'test_nothing_model_',
-                    'compiled_path': Normalized('target/compiled/test/models/schema.yml/schema_test/test_nothing_model_.sql'),
+                    'compiled_path': Normalized('target/compiled/test/models/schema.yml/test_nothing_model_.sql'),
                     'build_path': None,
                     'created_at': ANY,
                     'column_name': None,
@@ -1062,20 +1061,20 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
                     },
                     'deferred': False,
                     'description': '',
-                    'fqn': ['test', 'schema_test', 'test_nothing_model_'],
+                    'fqn': ['test', 'test_nothing_model_'],
                     'name': 'test_nothing_model_',
                     'original_file_path': model_schema_yml_path,
                     'package_name': 'test',
                     'patch_path': None,
-                    'path': normalize('schema_test/test_nothing_model_.sql'),
-                    'raw_sql': "{{ test.test_nothing(**_dbt_schema_test_kwargs) }}",
+                    'path': normalize('test_nothing_model_.sql'),
+                    'raw_sql': "{{ test.test_nothing(**_dbt_generic_test_kwargs) }}",
                     'refs': [['model']],
                     'relation_name': None,
                     'resource_type': 'test',
                     'root_path': self.test_root_realpath,
                     'schema': test_audit_schema,
                     'database': self.default_database,
-                    'tags': ['schema'],
+                    'tags': [],
                     'meta': {},
                     'unique_id': 'test.test.test_nothing_model_.5d38568946',
                     'docs': {'show': True},
@@ -1095,7 +1094,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
                 },
                 'test.test.unique_model_id.67b76558ff': {
                     'alias': 'unique_model_id',
-                    'compiled_path': Normalized('target/compiled/test/models/schema.yml/schema_test/unique_model_id.sql'),
+                    'compiled_path': Normalized('target/compiled/test/models/schema.yml/unique_model_id.sql'),
                     'build_path': None,
                     'created_at': ANY,
                     'column_name': 'id',
@@ -1108,20 +1107,20 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
                     },
                     'deferred': False,
                     'description': '',
-                    'fqn': ['test', 'schema_test', 'unique_model_id'],
+                    'fqn': ['test', 'unique_model_id'],
                     'name': 'unique_model_id',
                     'original_file_path': model_schema_yml_path,
                     'package_name': 'test',
                     'patch_path': None,
-                    'path': normalize('schema_test/unique_model_id.sql'),
-                    'raw_sql': "{{ test_unique(**_dbt_schema_test_kwargs) }}",
+                    'path': normalize('unique_model_id.sql'),
+                    'raw_sql': "{{ test_unique(**_dbt_generic_test_kwargs) }}",
                     'refs': [['model']],
                     'relation_name': None,
                     'resource_type': 'test',
                     'root_path': self.test_root_realpath,
                     'schema': test_audit_schema,
                     'database': self.default_database,
-                    'tags': ['schema'],
+                    'tags': [],
                     'meta': {},
                     'unique_id': 'test.test.unique_model_id.67b76558ff',
                     'docs': {'show': True},
@@ -1288,7 +1287,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
                 'test.macro_info': ANY,
                 'test.macro_arg_info': ANY,
             },
-            'disabled': [],
+            'disabled': {},
         }
 
     def expected_bigquery_complex_manifest(self):
@@ -1750,7 +1749,7 @@ def expected_bigquery_complex_manifest(self):
                 'test.macro_info': ANY,
                 'test.macro_arg_info': ANY,
             },
-            'disabled': [],
+            'disabled': {},
         }
 
     def _checksum_file(self, path):
diff --git a/tests/integration/incremental_schema_tests/test_incremental_schema.py b/tests/integration/incremental_schema_tests/test_incremental_schema.py
index a991fd568..a25b2484d 100644
--- a/tests/integration/incremental_schema_tests/test_incremental_schema.py
+++ b/tests/integration/incremental_schema_tests/test_incremental_schema.py
@@ -26,11 +26,11 @@ def list_tests_and_assert(self, include, exclude, expected_tests):
         listed = self.run_dbt(list_args)
         print(listed)
         assert len(listed) == len(expected_tests)
-        test_names = [name.split('.')[2] for name in listed]
+        test_names = [name.split('.')[-1] for name in listed]
         assert sorted(test_names) == sorted(expected_tests)
 
     def run_tests_and_assert(
-        self, include, exclude, expected_tests, compare_source, compare_target, schema = False, data = False
+        self, include, exclude, expected_tests, compare_source, compare_target
     ):
 
         run_args = ['run']
@@ -47,10 +47,6 @@ def run_tests_and_assert(
             test_args.extend(('--models', include))
         if exclude:
             test_args.extend(('--exclude', exclude))
-        if schema:
-            test_args.append('--schema')
-        if data:
-            test_args.append('--data')
 
         results = self.run_dbt(test_args)
         tests_run = [r.node.name for r in results]
diff --git a/tests/integration/query_comments_test/models/x.sql b/tests/integration/query_comments_test/models/x.sql
index 03d6c0abb..e719a30b5 100644
--- a/tests/integration/query_comments_test/models/x.sql
+++ b/tests/integration/query_comments_test/models/x.sql
@@ -12,8 +12,7 @@
 {% set required = ['name', 'schema', 'type', 'threads'] %}
 
 {# Require what we document at https://docs.getdbt.com/docs/target #}
-	{% do required.extend(['project']) %}
-{% endif %}
+{% do required.extend(['project']) %}
 
 {% for value in required %}
 	{% if value not in target %}
diff --git a/tests/integration/simple_snapshot_test/test_simple_snapshot.py b/tests/integration/simple_snapshot_test/test_simple_snapshot.py
index 5db2d02e6..009fd7ad5 100644
--- a/tests/integration/simple_snapshot_test/test_simple_snapshot.py
+++ b/tests/integration/simple_snapshot_test/test_simple_snapshot.py
@@ -69,8 +69,7 @@ def project_config(self):
     def _run_snapshot_test(self):
         self.run_dbt(['seed'])
         self.run_dbt(['snapshot'])
-        database = self.default_database
-        database = self.adapter.quote(database)
+        database = self.adapter.quote(self.default_database)
         results = self.run_sql(
             'select * from {}.{}.my_snapshot'.format(database, self.unique_schema()),
             fetch='all'
@@ -478,7 +477,7 @@ def _snapshot(self):
         return begin_snapshot_datetime
 
     def _delete_records(self):
-        database = self.adapter.quote(database)
+        database = self.adapter.quote(self.default_database)
 
         self.run_sql(
             'delete from {}.{}.seed where id >= 10;'.format(database, self.unique_schema())
@@ -487,7 +486,7 @@ def _delete_records(self):
     def _snapshot_and_assert_invalidated(self):
         self._invalidated_snapshot_datetime = self._snapshot()
 
-        database = self.adapter.quote(database)
+        database = self.adapter.quote(self.default_database)
 
         snapshotted = self.run_sql(
             '''
@@ -504,10 +503,10 @@ def _snapshot_and_assert_invalidated(self):
         for result in snapshotted[10:]:
             # result is a tuple, the dbt_valid_to column is the latest
             self.assertIsInstance(result[-1], datetime)
-            self.assertGreaterEqual(result[-1].astimezone(pytz.UTC), self._invalidated_snapshot_datetime)
+            self.assertGreaterEqual(result[-1].replace(tzinfo=pytz.UTC), self._invalidated_snapshot_datetime)
 
     def _revive_records(self):
-        database = self.adapter.quote(database)
+        database = self.adapter.quote(self.default_database)
 
         revival_timestamp = datetime.now(pytz.UTC).strftime(r'%Y-%m-%d %H:%M:%S')
         self.run_sql(
@@ -521,7 +520,7 @@ def _revive_records(self):
     def _snapshot_and_assert_revived(self):
         self._revived_snapshot_datetime = self._snapshot()
 
-        database = self.adapter.quote(database)
+        database = self.adapter.quote(self.default_database)
 
         # records which weren't revived (id != 10, 11)
         invalidated_records = self.run_sql(
@@ -540,7 +539,7 @@ def _snapshot_and_assert_revived(self):
         for result in invalidated_records:
             # result is a tuple, the dbt_valid_to column is the latest
             self.assertIsInstance(result[1], datetime)
-            self.assertGreaterEqual(result[1].astimezone(pytz.UTC), self._invalidated_snapshot_datetime)
+            self.assertGreaterEqual(result[1].replace(tzinfo=pytz.UTC), self._invalidated_snapshot_datetime)
 
         # records which weren't revived (id != 10, 11)
         revived_records = self.run_sql(
@@ -562,5 +561,5 @@ def _snapshot_and_assert_revived(self):
             self.assertIsInstance(result[1], datetime)
             # there are milliseconds (part of microseconds in datetime objects) in the
             # invalidated_snapshot_datetime and not in result datetime so set the microseconds to 0
-            self.assertGreaterEqual(result[1].astimezone(pytz.UTC), self._invalidated_snapshot_datetime.replace(microsecond=0))
+            self.assertGreaterEqual(result[1].replace(tzinfo=pytz.UTC), self._invalidated_snapshot_datetime.replace(microsecond=0))
             self.assertIsNone(result[2])
diff --git a/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py b/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py
index bd1436c47..811ef2169 100644
--- a/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py
+++ b/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py
@@ -32,7 +32,7 @@ def test_snapshot_check_cols_cycle(self):
         self.assertEqual(len(results), 1)
 
     def assert_expected(self):
-        self.run_dbt(['test', '--data', '--vars', 'version: 3'])
+        self.run_dbt(['test', '--select', 'test_type:singular', '--vars', 'version: 3'])
 
     @use_profile('bigquery')
     def test__bigquery__simple_snapshot(self):
diff --git a/tox.ini b/tox.ini
new file mode 100644
index 000000000..53971de00
--- /dev/null
+++ b/tox.ini
@@ -0,0 +1,38 @@
+[tox]
+skipsdist = True
+envlist = py36,py37,py38,py39,flake8
+
+[testenv:flake8]
+description = flake8 code checks
+basepython = python3.8
+skip_install = true
+commands = flake8 --select=E,W,F --ignore=W504,E741 --max-line-length 99 \
+  dbt
+deps =
+  -rdev_requirements.txt
+
+[testenv:{unit,py36,py37,py38,py39,py}]
+description = unit testing
+skip_install = true
+passenv = DBT_* PYTEST_ADDOPTS
+commands = {envpython} -m pytest {posargs} tests/unit
+deps =
+  -rdev_requirements.txt
+  -e.
+
+[testenv:{integration,py36,py37,py38,py39,py}-{bigquery}]
+description = adapter plugin integration testing
+skip_install = true
+passenv = DBT_* BIGQUERY_TEST_* PYTEST_ADDOPTS
+commands =
+  bigquery: {envpython} -m pytest {posargs} -m profile_bigquery tests/integration
+deps =
+  -rdev_requirements.txt
+  -e.
+
+[pytest]
+env_files =
+    test.env
+testpaths =
+    tests/unit
+    tests/integration

From ce69292ccfea21bb6aa1a0fc37d167a161359f75 Mon Sep 17 00:00:00 2001
From: Kyle Wigley 
Date: Mon, 4 Oct 2021 19:09:15 -0400
Subject: [PATCH 379/860] Moved gcloud file into dbt-bigquery codebase (#9)

Co-authored-by: Matthew McKnight 
---
 dbt/adapters/bigquery/connections.py |  3 ++-
 dbt/adapters/bigquery/gcloud.py      | 26 ++++++++++++++++++++++++++
 dbt/adapters/bigquery/impl.py        |  1 -
 3 files changed, 28 insertions(+), 2 deletions(-)
 create mode 100644 dbt/adapters/bigquery/gcloud.py

diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 9a9fed58c..7a831ff96 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -18,8 +18,9 @@
     service_account as GoogleServiceAccountCredentials
 )
 
+from dbt.adapters.bigquery import gcloud
 from dbt.utils import format_bytes, format_rows_number
-from dbt.clients import agate_helper, gcloud
+from dbt.clients import agate_helper
 from dbt.tracking import active_user
 from dbt.contracts.connection import ConnectionState, AdapterResponse
 from dbt.exceptions import (
diff --git a/dbt/adapters/bigquery/gcloud.py b/dbt/adapters/bigquery/gcloud.py
new file mode 100644
index 000000000..77ed74fdc
--- /dev/null
+++ b/dbt/adapters/bigquery/gcloud.py
@@ -0,0 +1,26 @@
+from dbt.logger import GLOBAL_LOGGER as logger
+import dbt.exceptions
+from dbt.clients.system import run_cmd
+
+NOT_INSTALLED_MSG = """
+dbt requires the gcloud SDK to be installed to authenticate with BigQuery.
+Please download and install the SDK, or use a Service Account instead.
+
+https://cloud.google.com/sdk/
+"""
+
+
+def gcloud_installed():
+    try:
+        run_cmd('.', ['gcloud', '--version'])
+        return True
+    except OSError as e:
+        logger.debug(e)
+        return False
+
+
+def setup_default_credentials():
+    if gcloud_installed():
+        run_cmd('.', ["gcloud", "auth", "application-default", "login"])
+    else:
+        raise dbt.exceptions.RuntimeException(NOT_INSTALLED_MSG)
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 24aa1cb36..33ec96443 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -4,7 +4,6 @@
 
 import dbt.deprecations
 import dbt.exceptions
-import dbt.clients.gcloud
 import dbt.clients.agate_helper
 
 from dbt import ui

From 373f19ca25a9fa9957e829b86eb3de6651a0016c Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Mon, 11 Oct 2021 09:50:11 -0400
Subject: [PATCH 380/860] Fixing bumpversion configuration

---
 .bumpversion.cfg | 26 +++++++++++---------------
 1 file changed, 11 insertions(+), 15 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 816a1a7ba..4923bea1b 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,29 +1,25 @@
 [bumpversion]
-current_version = 0.21.0rc2
+current_version = 0.21.0rc1
 parse = (?P\d+)
-\.(?P\d+)
-\.(?P\d+)
-(\.(?P\d+))?
-((?P[a-z]+)(?P\d+))?
+	\.(?P\d+)
+	\.(?P\d+)
+	((?Pa|b|rc)(?P\d+))?
 serialize = 
-{major}.{minor}.{patch}.{pluginpatch}{prerelease}{num}
-{major}.{minor}.{patch}{prerelease}{num}
-{major}.{minor}.{patch}.{pluginpatch}
-{major}.{minor}.{patch}
+	{major}.{minor}.{patch}{prerelease}{num}
+	{major}.{minor}.{patch}
 commit = False
 tag = False
 
 [bumpversion:part:prerelease]
 first_value = a
+optional_value = final
 values = 
-a
-b
-rc
+	a
+	b
+	rc
+	final
 
 [bumpversion:part:num]
 first_value = 1
 
-[bumpversion:part:pluginpatch]
-first_value = 1
-
 [bumpversion:file:dbt/adapters/bigquery/__version__.py]

From cf794aaa44ad7b426fe0dec125c3d15df741ad73 Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Mon, 11 Oct 2021 09:54:21 -0400
Subject: [PATCH 381/860] Adding version bump Action

---
 .github/workflows/version-bump.yml | 102 +++++++++++++++++++++++++++++
 1 file changed, 102 insertions(+)
 create mode 100644 .github/workflows/version-bump.yml

diff --git a/.github/workflows/version-bump.yml b/.github/workflows/version-bump.yml
new file mode 100644
index 000000000..4913a6e84
--- /dev/null
+++ b/.github/workflows/version-bump.yml
@@ -0,0 +1,102 @@
+# **what?**
+# This workflow will take a version number and a dry run flag. With that
+# it will run versionbump to update the version number everywhere in the 
+# code base and then generate an update Docker requirements file. If this
+# is a dry run, a draft PR will open with the changes. If this isn't a dry
+# run, the changes will be committed to the branch this is run on.
+
+# **why?**
+# This is to aid in releasing dbt and making sure we have updated 
+# the versions and Docker requirements in all places.
+
+# **when?**
+# This is triggered either manually OR 
+# from the repository_dispatch event "version-bump" which is sent from
+# the dbt-release repo Action
+
+name: Version Bump
+
+on:
+  workflow_dispatch:
+    inputs:
+      version_number:
+       description: 'The version number to bump to'
+       required: true
+      is_dry_run:
+       description: 'Creates a draft PR to allow testing instead of committing to a branch'
+       required: true
+       default: 'true'  
+  repository_dispatch:
+    types: [version-bump]
+
+jobs:          
+  bump:
+    runs-on: ubuntu-latest
+    steps:
+      - name: Check out the repository
+        uses: actions/checkout@v2
+
+      - name: Set version and dry run values
+        id: variables
+        env:
+          VERSION_NUMBER: "${{ github.event.client_payload.version_number == '' && github.event.inputs.version_number || github.event.client_payload.version_number }}"
+          IS_DRY_RUN: "${{ github.event.client_payload.is_dry_run == '' && github.event.inputs.is_dry_run || github.event.client_payload.is_dry_run }}"
+        run: |
+          echo Repository dispatch event version: ${{ github.event.client_payload.version_number }}
+          echo Repository dispatch event dry run: ${{ github.event.client_payload.is_dry_run }}
+          echo Workflow dispatch event version: ${{ github.event.inputs.version_number }}
+          echo Workflow dispatch event dry run: ${{ github.event.inputs.is_dry_run }}
+          echo ::set-output name=VERSION_NUMBER::$VERSION_NUMBER
+          echo ::set-output name=IS_DRY_RUN::$IS_DRY_RUN
+
+      - uses: actions/setup-python@v2
+        with:
+          python-version: "3.8"
+
+      - name: Install python dependencies
+        run: |
+          python3 -m venv env
+          source env/bin/activate
+          pip install --upgrade pip     
+          
+      - name: Create PR branch
+        if: ${{ steps.variables.outputs.IS_DRY_RUN  == 'true' }}
+        run: |
+          git checkout -b bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_$GITHUB_RUN_ID
+          git push origin bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_$GITHUB_RUN_ID
+          git branch --set-upstream-to=origin/bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_$GITHUB_RUN_ID bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_$GITHUB_RUN_ID
+   
+      - name: Bumping version
+        run: |
+          source env/bin/activate
+          pip install -r dev_requirements.txt 
+          env/bin/bumpversion --allow-dirty --new-version ${{steps.variables.outputs.VERSION_NUMBER}} major
+          git status
+
+      - name: Commit version bump directly
+        uses: EndBug/add-and-commit@v7
+        if: ${{ steps.variables.outputs.IS_DRY_RUN == 'false' }}
+        with:
+          author_name: 'Github Build Bot'
+          author_email: 'buildbot@fishtownanalytics.com'
+          message: 'Bumping version to ${{steps.variables.outputs.VERSION_NUMBER}}'
+
+      - name: Commit version bump to branch
+        uses: EndBug/add-and-commit@v7
+        if: ${{ steps.variables.outputs.IS_DRY_RUN == 'true' }}
+        with:
+          author_name: 'Github Build Bot'
+          author_email: 'buildbot@fishtownanalytics.com'
+          message: 'Bumping version to ${{steps.variables.outputs.VERSION_NUMBER}}'
+          branch: 'bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_${{GITHUB.RUN_ID}}'
+          push: 'origin origin/bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_${{GITHUB.RUN_ID}}'
+
+      - name: Create Pull Request
+        uses: peter-evans/create-pull-request@v3
+        if: ${{ steps.variables.outputs.IS_DRY_RUN == 'true' }}
+        with:
+          author: 'Github Build Bot '
+          draft: true
+          base: ${{github.ref}}
+          title: 'Bumping version to ${{steps.variables.outputs.VERSION_NUMBER}}'
+          branch: 'bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_${{GITHUB.RUN_ID}}'    

From d50357ffa1052b90dd756b4d10838a7285f41255 Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Mon, 11 Oct 2021 14:02:06 -0400
Subject: [PATCH 382/860] Bumping version to 1.0.0b1 (#11)

Co-authored-by: Github Build Bot 
---
 .bumpversion.cfg                     | 2 +-
 dbt/adapters/bigquery/__version__.py | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 4923bea1b..39b23539a 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 0.21.0rc1
+current_version = 1.0.0b1
 parse = (?P\d+)
 	\.(?P\d+)
 	\.(?P\d+)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index cebffb393..6f075105d 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = '0.21.0rc1'
+version = '1.0.0b1'

From be2d31de0b0593c5d52fb235b49294c3fe20ce74 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Tue, 12 Oct 2021 05:57:33 -0700
Subject: [PATCH 383/860] Run integration workflow nightly (#13)

---
 .github/workflows/integration.yml | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index a7b68ff15..cd4fdb4f5 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -33,6 +33,9 @@ on:
   pull_request_target:
   # manual trigger
   workflow_dispatch:
+  # run this once per night to ensure no regressions from latest dbt-core changes
+  schedule:
+    - cron: '0 5 * * *' # 5 UTC
 
 # explicitly turn off permissions for `GITHUB_TOKEN`
 permissions: read-all

From 125463f6f221c25cc878f6c7b9bf1ac3a3a835cd Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Tue, 12 Oct 2021 06:26:00 -0700
Subject: [PATCH 384/860] HEAD instead of develop

---
 dev_requirements.txt | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/dev_requirements.txt b/dev_requirements.txt
index b4ba00597..e0641f9c9 100644
--- a/dev_requirements.txt
+++ b/dev_requirements.txt
@@ -1,6 +1,6 @@
 # install latest changes in dbt-core
 # TODO: how to automate switching from develop to version branches?
-git+https://github.com/dbt-labs/dbt.git@develop#egg=dbt-core&subdirectory=core
+git+https://github.com/dbt-labs/dbt.git#egg=dbt-core&subdirectory=core
 
 bumpversion
 flake8
@@ -17,4 +17,4 @@ pytest-xdist
 pytz
 tox>=3.13
 twine
-wheel
\ No newline at end of file
+wheel

From 50bcd3e54b3d36ef50faed2f66ddaf3e266cda02 Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Tue, 12 Oct 2021 14:20:27 -0400
Subject: [PATCH 385/860] Fixing integration tests for pip

---
 .github/workflows/integration.yml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index cd4fdb4f5..0ab4066d9 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -152,7 +152,7 @@ jobs:
 
       - name: Install python dependencies
         run: |
-          pip install --upgrade pip
+          pip install --user --upgrade pip
           pip install tox
           pip --version
           tox --version

From 7340369abe8b48b55c7f3764aae3efd99a702c8b Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Tue, 12 Oct 2021 14:21:13 -0400
Subject: [PATCH 386/860] Fixing unit tests for pip

---
 .github/workflows/main.yml | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 84ff1eabd..88d9e3653 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -61,7 +61,7 @@ jobs:
 
       - name: Install python dependencies
         run: |
-          pip install --upgrade pip
+          pip install --user --upgrade pip
           pip install tox
           pip --version
           tox --version
@@ -96,7 +96,7 @@ jobs:
 
       - name: Install python dependencies
         run: |
-          pip install --upgrade pip
+          pip install --user --upgrade pip
           pip install tox
           pip --version
           tox --version
@@ -133,7 +133,7 @@ jobs:
 
       - name: Install python dependencies
         run: |
-          pip install --upgrade pip
+          pip install --user --upgrade pip
           pip install --upgrade setuptools wheel twine check-wheel-contents
           pip --version
 
@@ -177,7 +177,7 @@ jobs:
 
       - name: Install python dependencies
         run: |
-          pip install --upgrade pip
+          pip install --user --upgrade pip
           pip install --upgrade wheel
           pip --version
 

From 5486d047b272d22fa4c1a434ccd5e0acf82d031a Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Wed, 13 Oct 2021 01:03:28 +0200
Subject: [PATCH 387/860] Fix dependabot directory

---
 .github/dependabot.yml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/.github/dependabot.yml b/.github/dependabot.yml
index dea04a2fa..2a6f34492 100644
--- a/.github/dependabot.yml
+++ b/.github/dependabot.yml
@@ -2,7 +2,7 @@ version: 2
 updates:
   # python dependencies
   - package-ecosystem: "pip"
-    directory: "/dbt"
+    directory: "/"
     schedule:
       interval: "daily"
     rebase-strategy: "disabled"

From 16ab8e6ac689aa7d2088bcbfe1c75a1f5cdac4e7 Mon Sep 17 00:00:00 2001
From: Emily Rockman 
Date: Thu, 14 Oct 2021 07:34:49 -0500
Subject: [PATCH 388/860] Update tests for path deprecations and split out in
 dbt-core (#38)

* updates tests for path deprecations and split out

updated data -> seeds
source-paths -> model-paths
data-paths -> seeds-path

* renamed typo folder - added s

* another path update
---
 .../test_adapter_methods.py                    |  4 ++--
 tests/integration/base.py                      |  2 +-
 .../{data => seeds}/data_seed.csv              |  0
 .../incremental_overwrite_date_expected.csv    |  0
 .../incremental_overwrite_day_expected.csv     |  0
 .../incremental_overwrite_range_expected.csv   |  0
 .../incremental_overwrite_time_expected.csv    |  0
 .../{data => seeds}/merge_expected.csv         |  0
 .../bigquery_test/test_simple_bigquery_view.py |  2 +-
 .../column_quoting/{data => seeds}/seed.csv    |  0
 .../defer_state_test/{data => seeds}/seed.csv  |  0
 .../docs_generate_tests/rs_models/model.sql    |  7 -------
 .../docs_generate_tests/rs_models/schema.yml   | 16 ----------------
 .../docs_generate_tests/test_docs_generate.py  |  4 ++--
 .../{data => seeds}/example_seed.csv           |  0
 .../hook_tests/test_model_hooks_bq.py          |  2 +-
 .../hook_tests/test_run_hooks_bq.py            |  2 +-
 .../{data => seeds}/seed.csv                   |  0
 .../test_override_database.py                  |  6 +++---
 .../{data => seeds}/seed.csv                   |  0
 .../schema_tests_test/test_schema_v2_tests.py  |  2 +-
 .../simple_copy_test/test_simple_copy.py       |  8 ++++----
 .../seed_disabled.csv                          |  0
 .../seed_enabled.csv                           |  0
 .../seed_tricky.csv                            |  0
 .../test_seed_type_override.py                 |  2 +-
 .../{data => seeds}/seed.csv                   |  0
 .../{data => seeds}/seed_newcol.csv            |  0
 .../test_simple_snapshot.py                    | 18 +++++++++---------
 .../test_snapshot_check_cols.py                |  2 +-
 .../{data => seeds}/expected_multi_source.csv  |  0
 .../{data => seeds}/other_source_table.csv     |  0
 .../{data => seeds}/other_table.csv            |  0
 .../sources_test/{data => seeds}/source.csv    |  0
 tests/integration/sources_test/test_sources.py |  2 +-
 .../statement_test/test_statements.py          |  2 +-
 .../expected/expected_accepted_values.csv      |  0
 .../expected/expected_failing_test.csv         |  0
 .../expected_not_null_problematic_model_id.csv |  0
 .../expected_unique_problematic_model_id.csv   |  0
 .../{data => seeds}/people.csv                 |  0
 tests/unit/utils.py                            |  2 +-
 42 files changed, 30 insertions(+), 53 deletions(-)
 rename tests/integration/bigquery_test/{data => seeds}/data_seed.csv (100%)
 rename tests/integration/bigquery_test/{data => seeds}/incremental_overwrite_date_expected.csv (100%)
 rename tests/integration/bigquery_test/{data => seeds}/incremental_overwrite_day_expected.csv (100%)
 rename tests/integration/bigquery_test/{data => seeds}/incremental_overwrite_range_expected.csv (100%)
 rename tests/integration/bigquery_test/{data => seeds}/incremental_overwrite_time_expected.csv (100%)
 rename tests/integration/bigquery_test/{data => seeds}/merge_expected.csv (100%)
 rename tests/integration/column_quoting/{data => seeds}/seed.csv (100%)
 rename tests/integration/defer_state_test/{data => seeds}/seed.csv (100%)
 delete mode 100644 tests/integration/docs_generate_tests/rs_models/model.sql
 delete mode 100644 tests/integration/docs_generate_tests/rs_models/schema.yml
 rename tests/integration/hook_tests/{data => seeds}/example_seed.csv (100%)
 rename tests/integration/override_database_test/{data => seeds}/seed.csv (100%)
 rename tests/integration/persist_docs_tests/{data => seeds}/seed.csv (100%)
 rename tests/integration/simple_seed_test/{data-config => seeds-config}/seed_disabled.csv (100%)
 rename tests/integration/simple_seed_test/{data-config => seeds-config}/seed_enabled.csv (100%)
 rename tests/integration/simple_seed_test/{data-config => seeds-config}/seed_tricky.csv (100%)
 rename tests/integration/simple_snapshot_test/{data => seeds}/seed.csv (100%)
 rename tests/integration/simple_snapshot_test/{data => seeds}/seed_newcol.csv (100%)
 rename tests/integration/sources_test/{data => seeds}/expected_multi_source.csv (100%)
 rename tests/integration/sources_test/{data => seeds}/other_source_table.csv (100%)
 rename tests/integration/sources_test/{data => seeds}/other_table.csv (100%)
 rename tests/integration/sources_test/{data => seeds}/source.csv (100%)
 rename tests/integration/store_test_failures_tests/{data => seeds}/expected/expected_accepted_values.csv (100%)
 rename tests/integration/store_test_failures_tests/{data => seeds}/expected/expected_failing_test.csv (100%)
 rename tests/integration/store_test_failures_tests/{data => seeds}/expected/expected_not_null_problematic_model_id.csv (100%)
 rename tests/integration/store_test_failures_tests/{data => seeds}/expected/expected_unique_problematic_model_id.csv (100%)
 rename tests/integration/store_test_failures_tests/{data => seeds}/people.csv (100%)

diff --git a/tests/integration/adapter_methods_test/test_adapter_methods.py b/tests/integration/adapter_methods_test/test_adapter_methods.py
index a7613ca1e..fd6595fcf 100644
--- a/tests/integration/adapter_methods_test/test_adapter_methods.py
+++ b/tests/integration/adapter_methods_test/test_adapter_methods.py
@@ -38,7 +38,7 @@ def models(self):
     def project_config(self):
         return {
             'config-version': 2,
-            'source-paths': ['models']
+            'model-paths': ['models']
         }
 
     @use_profile('bigquery')
@@ -67,7 +67,7 @@ def models(self):
     def project_config(self):
         return {
             'config-version': 2,
-            'source-paths': ['models']
+            'model-paths': ['models']
         }
 
     @use_profile('bigquery')
diff --git a/tests/integration/base.py b/tests/integration/base.py
index acb0eaec0..6e31d2d71 100644
--- a/tests/integration/base.py
+++ b/tests/integration/base.py
@@ -270,7 +270,7 @@ def use_default_project(self, overrides=None):
             'version': '1.0',
             'config-version': 2,
             'test-paths': [],
-            'source-paths': [self.models],
+            'model-paths': [self.models],
             'profile': 'test',
         }
 
diff --git a/tests/integration/bigquery_test/data/data_seed.csv b/tests/integration/bigquery_test/seeds/data_seed.csv
similarity index 100%
rename from tests/integration/bigquery_test/data/data_seed.csv
rename to tests/integration/bigquery_test/seeds/data_seed.csv
diff --git a/tests/integration/bigquery_test/data/incremental_overwrite_date_expected.csv b/tests/integration/bigquery_test/seeds/incremental_overwrite_date_expected.csv
similarity index 100%
rename from tests/integration/bigquery_test/data/incremental_overwrite_date_expected.csv
rename to tests/integration/bigquery_test/seeds/incremental_overwrite_date_expected.csv
diff --git a/tests/integration/bigquery_test/data/incremental_overwrite_day_expected.csv b/tests/integration/bigquery_test/seeds/incremental_overwrite_day_expected.csv
similarity index 100%
rename from tests/integration/bigquery_test/data/incremental_overwrite_day_expected.csv
rename to tests/integration/bigquery_test/seeds/incremental_overwrite_day_expected.csv
diff --git a/tests/integration/bigquery_test/data/incremental_overwrite_range_expected.csv b/tests/integration/bigquery_test/seeds/incremental_overwrite_range_expected.csv
similarity index 100%
rename from tests/integration/bigquery_test/data/incremental_overwrite_range_expected.csv
rename to tests/integration/bigquery_test/seeds/incremental_overwrite_range_expected.csv
diff --git a/tests/integration/bigquery_test/data/incremental_overwrite_time_expected.csv b/tests/integration/bigquery_test/seeds/incremental_overwrite_time_expected.csv
similarity index 100%
rename from tests/integration/bigquery_test/data/incremental_overwrite_time_expected.csv
rename to tests/integration/bigquery_test/seeds/incremental_overwrite_time_expected.csv
diff --git a/tests/integration/bigquery_test/data/merge_expected.csv b/tests/integration/bigquery_test/seeds/merge_expected.csv
similarity index 100%
rename from tests/integration/bigquery_test/data/merge_expected.csv
rename to tests/integration/bigquery_test/seeds/merge_expected.csv
diff --git a/tests/integration/bigquery_test/test_simple_bigquery_view.py b/tests/integration/bigquery_test/test_simple_bigquery_view.py
index 44df6c099..0418dc386 100644
--- a/tests/integration/bigquery_test/test_simple_bigquery_view.py
+++ b/tests/integration/bigquery_test/test_simple_bigquery_view.py
@@ -17,7 +17,7 @@ def models(self):
     def project_config(self):
         return {
             'config-version': 2,
-            'data-paths': ['data'],
+            'seed-paths': ['seeds'],
             'macro-paths': ['macros'],
             'seeds': {
                 'quote_columns': False,
diff --git a/tests/integration/column_quoting/data/seed.csv b/tests/integration/column_quoting/seeds/seed.csv
similarity index 100%
rename from tests/integration/column_quoting/data/seed.csv
rename to tests/integration/column_quoting/seeds/seed.csv
diff --git a/tests/integration/defer_state_test/data/seed.csv b/tests/integration/defer_state_test/seeds/seed.csv
similarity index 100%
rename from tests/integration/defer_state_test/data/seed.csv
rename to tests/integration/defer_state_test/seeds/seed.csv
diff --git a/tests/integration/docs_generate_tests/rs_models/model.sql b/tests/integration/docs_generate_tests/rs_models/model.sql
deleted file mode 100644
index 8ca73c7a0..000000000
--- a/tests/integration/docs_generate_tests/rs_models/model.sql
+++ /dev/null
@@ -1,7 +0,0 @@
-{{
-    config(
-        materialized='view', bind=False
-    )
-}}
-
-select * from {{ ref('seed') }}
diff --git a/tests/integration/docs_generate_tests/rs_models/schema.yml b/tests/integration/docs_generate_tests/rs_models/schema.yml
deleted file mode 100644
index 37dd397c8..000000000
--- a/tests/integration/docs_generate_tests/rs_models/schema.yml
+++ /dev/null
@@ -1,16 +0,0 @@
-version: 2
-
-models:
-  - name: model
-    description: "The test model"
-    columns:
-      - name: id
-        description: The user ID number
-      - name: first_name
-        description: The user's first name
-      - name: email
-        description: The user's email
-      - name: ip_address
-        description: The user's IP address
-      - name: updated_at
-        description: The last time this user's email was updated
diff --git a/tests/integration/docs_generate_tests/test_docs_generate.py b/tests/integration/docs_generate_tests/test_docs_generate.py
index c153c6bc3..e05424489 100644
--- a/tests/integration/docs_generate_tests/test_docs_generate.py
+++ b/tests/integration/docs_generate_tests/test_docs_generate.py
@@ -127,7 +127,7 @@ def run_and_generate(self, extra=None, seed_count=1, model_count=2, alternate_db
         if alternate_db is None:
             alternate_db = self.alternative_database
         project = {
-            "data-paths": [self.dir("seed")],
+            "seed-paths": [self.dir("seed")],
             'macro-paths': [self.dir('macros')],
             'snapshot-paths': [self.dir('snapshot')],
             'vars': {
@@ -1942,7 +1942,7 @@ def test__bigquery__run_and_generate(self):
     @use_profile('bigquery')
     def test__bigquery__complex_models(self):
         self.run_and_generate(
-            extra={'source-paths': [self.dir('bq_models')]},
+            extra={'model-paths': [self.dir('bq_models')]},
             model_count=4
         )
 
diff --git a/tests/integration/hook_tests/data/example_seed.csv b/tests/integration/hook_tests/seeds/example_seed.csv
similarity index 100%
rename from tests/integration/hook_tests/data/example_seed.csv
rename to tests/integration/hook_tests/seeds/example_seed.csv
diff --git a/tests/integration/hook_tests/test_model_hooks_bq.py b/tests/integration/hook_tests/test_model_hooks_bq.py
index 1cd3326cd..8b5c8d5bd 100644
--- a/tests/integration/hook_tests/test_model_hooks_bq.py
+++ b/tests/integration/hook_tests/test_model_hooks_bq.py
@@ -126,7 +126,7 @@ def models(self):
     def project_config(self):
         return {
             'config-version': 2,
-            'data-paths': ['data'],
+            'seed-paths': ['seeds'],
             'models': {},
             'seeds': {
                 'post-hook': [
diff --git a/tests/integration/hook_tests/test_run_hooks_bq.py b/tests/integration/hook_tests/test_run_hooks_bq.py
index 37b82735f..ef1b7cae0 100644
--- a/tests/integration/hook_tests/test_run_hooks_bq.py
+++ b/tests/integration/hook_tests/test_run_hooks_bq.py
@@ -34,7 +34,7 @@ def project_config(self):
         return {
             'config-version': 2,
             'macro-paths': ['macros'],
-            'data-paths': ['data'],
+            'seed-paths': ['seeds'],
 
             # The create and drop table statements here validate that these hooks run
             # in the same order that they are defined. Drop before create is an error.
diff --git a/tests/integration/override_database_test/data/seed.csv b/tests/integration/override_database_test/seeds/seed.csv
similarity index 100%
rename from tests/integration/override_database_test/data/seed.csv
rename to tests/integration/override_database_test/seeds/seed.csv
diff --git a/tests/integration/override_database_test/test_override_database.py b/tests/integration/override_database_test/test_override_database.py
index 7e0a85fb2..9b12b3e79 100644
--- a/tests/integration/override_database_test/test_override_database.py
+++ b/tests/integration/override_database_test/test_override_database.py
@@ -21,7 +21,7 @@ def alternative_database(self):
     def project_config(self):
         return {
             'config-version': 2,
-            'data-paths': ['data'],
+            'seed-paths': ['seeds'],
             'vars': {
                 'alternate_db': self.alternative_database,
             },
@@ -100,7 +100,7 @@ def project_config(self):
                     }
                 }
             },
-            'data-paths': ['data'],
+            'seed-paths': ['seeds'],
             'vars': {
                 'alternate_db': self.alternative_database,
             },
@@ -133,7 +133,7 @@ def project_config(self):
                     }
                 }
             },
-            'data-paths': ['data'],
+            'seed-paths': ['seeds'],
             'vars': {
                 'alternate_db': self.alternative_database,
             },
diff --git a/tests/integration/persist_docs_tests/data/seed.csv b/tests/integration/persist_docs_tests/seeds/seed.csv
similarity index 100%
rename from tests/integration/persist_docs_tests/data/seed.csv
rename to tests/integration/persist_docs_tests/seeds/seed.csv
diff --git a/tests/integration/schema_tests_test/test_schema_v2_tests.py b/tests/integration/schema_tests_test/test_schema_v2_tests.py
index a66f79664..7beb5ce07 100644
--- a/tests/integration/schema_tests_test/test_schema_v2_tests.py
+++ b/tests/integration/schema_tests_test/test_schema_v2_tests.py
@@ -27,7 +27,7 @@ def run_schema_validations(self):
 
     @use_profile('bigquery')
     def test_schema_tests_bigquery(self):
-        self.use_default_project({'data-paths': [self.dir('seed')]})
+        self.use_default_project({'seed-paths': [self.dir('seed')]})
         self.assertEqual(len(self.run_dbt(['seed'])), 1)
         results = self.run_dbt()
         self.assertEqual(len(results), 1)
diff --git a/tests/integration/simple_copy_test/test_simple_copy.py b/tests/integration/simple_copy_test/test_simple_copy.py
index 39733c4e6..e388dede7 100644
--- a/tests/integration/simple_copy_test/test_simple_copy.py
+++ b/tests/integration/simple_copy_test/test_simple_copy.py
@@ -39,7 +39,7 @@ class TestSimpleCopy(BaseTestSimpleCopy):
 
     @property
     def project_config(self):
-        return self.seed_quote_cfg_with({"data-paths": [self.dir("seed-initial")]})
+        return self.seed_quote_cfg_with({"seed-paths": [self.dir("seed-initial")]})
 
     @use_profile("bigquery")
     def test__bigquery__simple_copy(self):
@@ -53,7 +53,7 @@ def test__bigquery__simple_copy(self):
         self.assertTablesEqual("seed", "materialized")
         self.assertTablesEqual("seed", "get_and_ref")
 
-        self.use_default_project({"data-paths": [self.dir("seed-update")]})
+        self.use_default_project({"seed-paths": [self.dir("seed-update")]})
 
         results = self.run_dbt(["seed"])
         self.assertEqual(len(results),  1)
@@ -86,11 +86,11 @@ def seed_and_run(self):
     @use_profile("bigquery")
     def test__bigquery__incremental_merge_columns(self):
         self.use_default_project({
-            "data-paths": ["seeds-merge-cols-initial"]
+            "seed-paths": ["seeds-merge-cols-initial"]
         })
         self.seed_and_run()
         self.use_default_project({
-            "data-paths": ["seeds-merge-cols-update"]
+            "seed-paths": ["seeds-merge-cols-update"]
         })
         self.seed_and_run()
         self.assertTablesEqual("incremental_update_cols", "expected_result")
diff --git a/tests/integration/simple_seed_test/data-config/seed_disabled.csv b/tests/integration/simple_seed_test/seeds-config/seed_disabled.csv
similarity index 100%
rename from tests/integration/simple_seed_test/data-config/seed_disabled.csv
rename to tests/integration/simple_seed_test/seeds-config/seed_disabled.csv
diff --git a/tests/integration/simple_seed_test/data-config/seed_enabled.csv b/tests/integration/simple_seed_test/seeds-config/seed_enabled.csv
similarity index 100%
rename from tests/integration/simple_seed_test/data-config/seed_enabled.csv
rename to tests/integration/simple_seed_test/seeds-config/seed_enabled.csv
diff --git a/tests/integration/simple_seed_test/data-config/seed_tricky.csv b/tests/integration/simple_seed_test/seeds-config/seed_tricky.csv
similarity index 100%
rename from tests/integration/simple_seed_test/data-config/seed_tricky.csv
rename to tests/integration/simple_seed_test/seeds-config/seed_tricky.csv
diff --git a/tests/integration/simple_seed_test/test_seed_type_override.py b/tests/integration/simple_seed_test/test_seed_type_override.py
index 824226816..4f97880d8 100644
--- a/tests/integration/simple_seed_test/test_seed_type_override.py
+++ b/tests/integration/simple_seed_test/test_seed_type_override.py
@@ -11,7 +11,7 @@ def schema(self):
     def project_config(self):
         return {
             'config-version': 2,
-            'data-paths': ['data-config'],
+            'seed-paths': ['seeds-config'],
             'macro-paths': ['macros'],
             'seeds': {
                 'test': {
diff --git a/tests/integration/simple_snapshot_test/data/seed.csv b/tests/integration/simple_snapshot_test/seeds/seed.csv
similarity index 100%
rename from tests/integration/simple_snapshot_test/data/seed.csv
rename to tests/integration/simple_snapshot_test/seeds/seed.csv
diff --git a/tests/integration/simple_snapshot_test/data/seed_newcol.csv b/tests/integration/simple_snapshot_test/seeds/seed_newcol.csv
similarity index 100%
rename from tests/integration/simple_snapshot_test/data/seed_newcol.csv
rename to tests/integration/simple_snapshot_test/seeds/seed_newcol.csv
diff --git a/tests/integration/simple_snapshot_test/test_simple_snapshot.py b/tests/integration/simple_snapshot_test/test_simple_snapshot.py
index 009fd7ad5..7f6c134c3 100644
--- a/tests/integration/simple_snapshot_test/test_simple_snapshot.py
+++ b/tests/integration/simple_snapshot_test/test_simple_snapshot.py
@@ -38,7 +38,7 @@ class TestSimpleSnapshotFiles(BaseSimpleSnapshotTest):
     def project_config(self):
         return {
             'config-version': 2,
-            "data-paths": ['data'],
+            'seed-paths': ['seeds'],
             "snapshot-paths": ['test-snapshots-pg'],
             'macro-paths': ['macros'],
         }
@@ -58,7 +58,7 @@ def models(self):
     def project_config(self):
         return {
             'config-version': 2,
-            'data-paths': ['data'],
+            'seed-paths': ['seeds'],
             'macro-paths': ['custom-snapshot-macros', 'macros'],
             'snapshot-paths': ['test-snapshots-checkall'],
             'seeds': {
@@ -117,7 +117,7 @@ def models(self):
     def project_config(self):
         return {
             'config-version': 2,
-            "data-paths": ['data'],
+            'seed-paths': ['seeds'],
             "snapshot-paths": ['test-snapshots-select',
                                'test-snapshots-pg'],
             'macro-paths': ['macros'],
@@ -129,7 +129,7 @@ class TestConfiguredSnapshotFileSelects(TestSimpleSnapshotFileSelects):
     def project_config(self):
         return {
             'config-version': 2,
-            "data-paths": ['data'],
+            'seed-paths': ['seeds'],
             "snapshot-paths": ['test-snapshots-select-noconfig'],
             "snapshots": {
                 "test": {
@@ -280,7 +280,7 @@ def assert_expected(self):
     def project_config(self):
         return {
             'config-version': 2,
-            "data-paths": ['data'],
+            'seed-paths': ['seeds'],
             "snapshot-paths": ['test-check-col-snapshots'],
             'macro-paths': ['macros'],
         }
@@ -291,7 +291,7 @@ class TestConfiguredCheckCols(TestCheckCols):
     def project_config(self):
         return {
             'config-version': 2,
-            "data-paths": ['data'],
+            'seed-paths': ['seeds'],
             "snapshot-paths": ['test-check-col-snapshots-noconfig'],
             "snapshots": {
                 "test": {
@@ -337,7 +337,7 @@ def assert_expected(self):
      def project_config(self):
          return {
              'config-version': 2,
-             "data-paths": ['data'],
+            'seed-paths': ['seeds'],
              "snapshot-paths": ['test-check-col-snapshots-noconfig'],
              "snapshots": {
                  "test": {
@@ -368,7 +368,7 @@ def assert_expected(self):
     def project_config(self):
         return {
             'config-version': 2,
-            "data-paths": ['data'],
+            'seed-paths': ['seeds'],
             "snapshot-paths": ['test-check-col-snapshots-bq'],
             'macro-paths': ['macros'],
         }
@@ -446,7 +446,7 @@ def project_config(self):
 
         return {
             'config-version': 2,
-            "data-paths": ['data'],
+            'seed-paths': ['seeds'],
             "snapshot-paths": paths,
             'macro-paths': ['macros'],
         }
diff --git a/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py b/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py
index 811ef2169..ad027e0d8 100644
--- a/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py
+++ b/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py
@@ -18,7 +18,7 @@ def project_config(self):
             'config-version': 2,
             "snapshot-paths": ['check-snapshots'],
             "test-paths": ['check-snapshots-expected'],
-            "source-paths": [],
+            "model-paths": [],
         }
 
     def test_snapshot_check_cols_cycle(self):
diff --git a/tests/integration/sources_test/data/expected_multi_source.csv b/tests/integration/sources_test/seeds/expected_multi_source.csv
similarity index 100%
rename from tests/integration/sources_test/data/expected_multi_source.csv
rename to tests/integration/sources_test/seeds/expected_multi_source.csv
diff --git a/tests/integration/sources_test/data/other_source_table.csv b/tests/integration/sources_test/seeds/other_source_table.csv
similarity index 100%
rename from tests/integration/sources_test/data/other_source_table.csv
rename to tests/integration/sources_test/seeds/other_source_table.csv
diff --git a/tests/integration/sources_test/data/other_table.csv b/tests/integration/sources_test/seeds/other_table.csv
similarity index 100%
rename from tests/integration/sources_test/data/other_table.csv
rename to tests/integration/sources_test/seeds/other_table.csv
diff --git a/tests/integration/sources_test/data/source.csv b/tests/integration/sources_test/seeds/source.csv
similarity index 100%
rename from tests/integration/sources_test/data/source.csv
rename to tests/integration/sources_test/seeds/source.csv
diff --git a/tests/integration/sources_test/test_sources.py b/tests/integration/sources_test/test_sources.py
index 8a8cbd0b2..553e91287 100644
--- a/tests/integration/sources_test/test_sources.py
+++ b/tests/integration/sources_test/test_sources.py
@@ -24,7 +24,7 @@ def models(self):
     def project_config(self):
         return {
             'config-version': 2,
-            'data-paths': ['data'],
+            'seed-paths': ['seeds'],
             'quoting': {'database': True, 'schema': True, 'identifier': True},
             'seeds': {
                 'quote_columns': True,
diff --git a/tests/integration/statement_test/test_statements.py b/tests/integration/statement_test/test_statements.py
index 0c96fc78c..620ca239b 100644
--- a/tests/integration/statement_test/test_statements.py
+++ b/tests/integration/statement_test/test_statements.py
@@ -26,7 +26,7 @@ def project_config(self):
 
     @use_profile("bigquery")
     def test_bigquery_statements(self):
-        self.use_default_project({"data-paths": [self.dir("seed")]})
+        self.use_default_project({"seed-paths": [self.dir("seed")]})
 
         results = self.run_dbt(["seed"])
         self.assertEqual(len(results), 2)
diff --git a/tests/integration/store_test_failures_tests/data/expected/expected_accepted_values.csv b/tests/integration/store_test_failures_tests/seeds/expected/expected_accepted_values.csv
similarity index 100%
rename from tests/integration/store_test_failures_tests/data/expected/expected_accepted_values.csv
rename to tests/integration/store_test_failures_tests/seeds/expected/expected_accepted_values.csv
diff --git a/tests/integration/store_test_failures_tests/data/expected/expected_failing_test.csv b/tests/integration/store_test_failures_tests/seeds/expected/expected_failing_test.csv
similarity index 100%
rename from tests/integration/store_test_failures_tests/data/expected/expected_failing_test.csv
rename to tests/integration/store_test_failures_tests/seeds/expected/expected_failing_test.csv
diff --git a/tests/integration/store_test_failures_tests/data/expected/expected_not_null_problematic_model_id.csv b/tests/integration/store_test_failures_tests/seeds/expected/expected_not_null_problematic_model_id.csv
similarity index 100%
rename from tests/integration/store_test_failures_tests/data/expected/expected_not_null_problematic_model_id.csv
rename to tests/integration/store_test_failures_tests/seeds/expected/expected_not_null_problematic_model_id.csv
diff --git a/tests/integration/store_test_failures_tests/data/expected/expected_unique_problematic_model_id.csv b/tests/integration/store_test_failures_tests/seeds/expected/expected_unique_problematic_model_id.csv
similarity index 100%
rename from tests/integration/store_test_failures_tests/data/expected/expected_unique_problematic_model_id.csv
rename to tests/integration/store_test_failures_tests/seeds/expected/expected_unique_problematic_model_id.csv
diff --git a/tests/integration/store_test_failures_tests/data/people.csv b/tests/integration/store_test_failures_tests/seeds/people.csv
similarity index 100%
rename from tests/integration/store_test_failures_tests/data/people.csv
rename to tests/integration/store_test_failures_tests/seeds/people.csv
diff --git a/tests/unit/utils.py b/tests/unit/utils.py
index 01c737e1b..051f4a4e3 100644
--- a/tests/unit/utils.py
+++ b/tests/unit/utils.py
@@ -235,7 +235,7 @@ def generate_name_macros(package):
             unique_id=f'macro.{package}.{name}',
             package_name=package,
             original_file_path=normalize('macros/macro.sql'),
-            root_path='./dbt_modules/root',
+            root_path='./dbt_packages/root',
             path=normalize('macros/macro.sql'),
             macro_sql=sql,
         )

From d326ddaba2cc975f765155fc4b0c9e7e736a3d01 Mon Sep 17 00:00:00 2001
From: Dylan Kelly <58730363+DigUpTheHatchet@users.noreply.github.com>
Date: Tue, 19 Oct 2021 19:39:01 +1100
Subject: [PATCH 389/860] =?UTF-8?q?Created=20bigquery=5F=5Ftest=5Funique?=
 =?UTF-8?q?=20to=20handle=20expressions=20passed=20to=20BQ=20uniq=E2=80=A6?=
 =?UTF-8?q?=20(#10)?=
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

* Created bigquery__test_unique to handle expressions passed to BQ unique schema test

* Added test on column with the same name as model & updated CHANGELOG.md
---
 CHANGELOG.md                                  |  9 +++++--
 dbt/include/bigquery/macros/adapters.sql      | 24 +++++++++++++++++--
 .../models-v2/bq-models/schema.yml            |  8 +++++++
 .../schema_tests_test/models-v2/seed/seed.csv | 10 ++++----
 .../schema_tests_test/test_schema_v2_tests.py |  2 +-
 5 files changed, 43 insertions(+), 10 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index e78891e0a..bd35256c0 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -4,7 +4,12 @@
 N/A
 
 ### Fixes
-N/A
+
+- Reimplement the `unique` test to handle column expressions and naming overlaps ([#33](https://github.com/dbt-labs/dbt-bigquery/issues/33), [#35](https://github.com/dbt-labs/dbt-bigquery/issues/35), [#10](https://github.com/dbt-labs/dbt-bigquery/pull/10))
 
 ### Under the hood
-First version as a seperate repo.
\ No newline at end of file
+First version as a seperate repo.
+
+### Contributors
+
+- [@DigUpTheHatchet](https://github.com/DigUpTheHatchet) ([#10](https://github.com/dbt-labs/dbt-bigquery/pull/10))
\ No newline at end of file
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index 03f70a1d3..b98a43385 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -168,8 +168,7 @@
 
 
 {% macro bigquery__alter_column_type(relation, column_name, new_column_type) -%}
-  {#
-    Changing a column's data type using a query requires you to scan the entire table.
+  {#-- Changing a column's data type using a query requires you to scan the entire table.
     The query charges can be significant if the table is very large.
 
     https://cloud.google.com/bigquery/docs/manually-changing-schemas#changing_a_columns_data_type
@@ -194,3 +193,24 @@
   {%- endcall %}
 
 {% endmacro %}
+
+
+{% macro bigquery__test_unique(model, column_name) %}
+
+with dbt_test__target as (
+  
+  select {{ column_name }} as unique_field
+  from {{ model }}
+  where {{ column_name }} is not null
+  
+)
+
+select
+    unique_field,
+    count(*) as n_records
+
+from dbt_test__target
+group by unique_field
+having count(*) > 1
+
+{% endmacro %}
diff --git a/tests/integration/schema_tests_test/models-v2/bq-models/schema.yml b/tests/integration/schema_tests_test/models-v2/bq-models/schema.yml
index 15100c18e..16d3e155a 100644
--- a/tests/integration/schema_tests_test/models-v2/bq-models/schema.yml
+++ b/tests/integration/schema_tests_test/models-v2/bq-models/schema.yml
@@ -12,10 +12,18 @@ models:
       - name: favorite_color
         tests:
           - accepted_values: { values: ['blue', 'green'] }
+      - name: ephemeral_copy
+        description: "Testing a column with the same name as the model"
+        tests:
+          - not_null
+          - unique
 
   # this whole model should pass and run
   - name: table_summary
     description: "The summary table"
+    tests: 
+     - unique:
+          column_name: "concat(favorite_color_copy, count)"
     columns:
       - name: favorite_color_copy
         description: "The favorite color"
diff --git a/tests/integration/schema_tests_test/models-v2/seed/seed.csv b/tests/integration/schema_tests_test/models-v2/seed/seed.csv
index dc9bdeaf6..77fa7af00 100644
--- a/tests/integration/schema_tests_test/models-v2/seed/seed.csv
+++ b/tests/integration/schema_tests_test/models-v2/seed/seed.csv
@@ -1,5 +1,5 @@
-favorite_color,id,first_name,email,ip_address,updated_at
-blue,1,Larry,null,69.135.206.194,2008-09-12 19:08:31
-blue,2,Larry,null,64.210.133.162,1978-05-09 04:15:14
-green,99,Paul,pjohnson2q@umn.edu,183.59.198.197,1991-11-14 12:33:55
-green,100,Frank,fgreene2r@blogspot.com,150.143.68.121,2010-06-12 23:55:39
+favorite_color,id,first_name,email,ip_address,updated_at,ephemeral_copy
+blue,1,Larry,null,69.135.206.194,2008-09-12 19:08:31,1000
+blue,2,Larry,null,64.210.133.162,1978-05-09 04:15:14,2000
+green,99,Paul,pjohnson2q@umn.edu,183.59.198.197,1991-11-14 12:33:55,3000
+green,100,Frank,fgreene2r@blogspot.com,150.143.68.121,2010-06-12 23:55:39,4000
diff --git a/tests/integration/schema_tests_test/test_schema_v2_tests.py b/tests/integration/schema_tests_test/test_schema_v2_tests.py
index 7beb5ce07..0709fdb44 100644
--- a/tests/integration/schema_tests_test/test_schema_v2_tests.py
+++ b/tests/integration/schema_tests_test/test_schema_v2_tests.py
@@ -32,7 +32,7 @@ def test_schema_tests_bigquery(self):
         results = self.run_dbt()
         self.assertEqual(len(results), 1)
         test_results = self.run_schema_validations()
-        self.assertEqual(len(test_results), 8)
+        self.assertEqual(len(test_results), 11)
 
         for result in test_results:
             # assert that all deliberately failing tests actually fail

From 1fb81431c4e69abd58bc611ca7ba629796fedbb2 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Tue, 19 Oct 2021 13:59:35 +0200
Subject: [PATCH 390/860] Detect changes only if PR

---
 .github/workflows/integration.yml | 1 +
 1 file changed, 1 insertion(+)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 0ab4066d9..e0f0e5e6e 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -78,6 +78,7 @@ jobs:
           ref: ${{ github.event.pull_request.head.sha }}
 
       - name: Check if relevant files changed
+        if: github.event_name == 'pull_request_target'
         # https://github.com/marketplace/actions/paths-changes-filter
         # For each filter, it sets output variable named by the filter to the text:
         #  'true' - if any of changed files matches any of filter rules

From 24cfa979f1e62b2fcb3d2ce86ccf14ad9a15b5e8 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Thu, 21 Oct 2021 15:00:09 +0200
Subject: [PATCH 391/860] Rework _dbt_max_partition (#41)

* Rework _dbt_max_partition

* Add changelog entry
---
 CHANGELOG.md                                  | 13 +++---
 .../macros/materializations/incremental.sql   | 23 ++++++++---
 ...c_all_columns_dynamic_insert_overwrite.sql | 40 +++++++++++++++++++
 3 files changed, 66 insertions(+), 10 deletions(-)
 create mode 100644 tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_dynamic_insert_overwrite.sql

diff --git a/CHANGELOG.md b/CHANGELOG.md
index bd35256c0..1ef13dea2 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,15 +1,18 @@
 ## dbt-bigquery 1.0.0 (Release TBD)
 
 ### Features
-N/A
+
+- Rework `_dbt_max_partition` logic in dynamic `insert_overwrite` incremental strategy. Make the default logic compatible with `on_schema_change`, and make it possible to disable or reimplement that logic by defining a custom macro `declare_dbt_max_partition` ([#17](https://github.com/dbt-labs/dbt-bigquery/issues/17), [#39](https://github.com/dbt-labs/dbt-bigquery/issues/39), [#41](https://github.com/dbt-labs/dbt-bigquery/pull/41))
 
 ### Fixes
 
 - Reimplement the `unique` test to handle column expressions and naming overlaps ([#33](https://github.com/dbt-labs/dbt-bigquery/issues/33), [#35](https://github.com/dbt-labs/dbt-bigquery/issues/35), [#10](https://github.com/dbt-labs/dbt-bigquery/pull/10))
 
-### Under the hood
-First version as a seperate repo.
-
 ### Contributors
 
-- [@DigUpTheHatchet](https://github.com/DigUpTheHatchet) ([#10](https://github.com/dbt-labs/dbt-bigquery/pull/10))
\ No newline at end of file
+- [@DigUpTheHatchet](https://github.com/DigUpTheHatchet) ([#10](https://github.com/dbt-labs/dbt-bigquery/pull/10))
+
+## dbt-bigquery 1.0.0b1 (October 11, 2021)
+
+### Under the hood
+First version as a separate repo.
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index 0b8e7bac5..4c826e8da 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -1,3 +1,16 @@
+{% macro declare_dbt_max_partition(relation, partition_by, sql) %}
+
+  {% if '_dbt_max_partition' in sql %}
+
+    declare _dbt_max_partition {{ partition_by.data_type }} default (
+      select max({{ partition_by.field }}) from {{ this }}
+      where {{ partition_by.field }} is not null
+    );
+  
+  {% endif %}
+
+{% endmacro %}
+
 
 {% macro dbt_bigquery_validate_get_incremental_strategy(config) %}
   {#-- Find and validate the incremental strategy #}
@@ -49,13 +62,11 @@
 
       -- generated script to merge partitions into {{ target_relation }}
       declare dbt_partitions_for_replacement array<{{ partition_by.data_type }}>;
-      declare _dbt_max_partition {{ partition_by.data_type }} default (
-          select max({{ partition_by.field }}) from {{ this }}
-          where {{ partition_by.field }} is not null
-      );
 
       {# have we already created the temp table to check for schema changes? #}
       {% if not tmp_relation_exists %}
+        {{ declare_dbt_max_partition(this, partition_by, sql) }}
+        
         -- 1. create a temp table
         {{ create_table_as(True, tmp_relation, sql) }}
       {% else %}
@@ -162,7 +173,9 @@
   {% else %}
     {% set tmp_relation_exists = false %}
     {% if on_schema_change != 'ignore' %} {# Check first, since otherwise we may not build a temp table #}
-      {% do run_query(create_table_as(True, tmp_relation, sql)) %}
+      {% do run_query(
+        declare_dbt_max_partition(this, partition_by, sql) + create_table_as(True, tmp_relation, sql)
+      ) %}
       {% set tmp_relation_exists = true %}
       {% do process_schema_changes(on_schema_change, tmp_relation, existing_relation) %}
     {% endif %}
diff --git a/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_dynamic_insert_overwrite.sql b/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_dynamic_insert_overwrite.sql
new file mode 100644
index 000000000..d82e683a3
--- /dev/null
+++ b/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_dynamic_insert_overwrite.sql
@@ -0,0 +1,40 @@
+{{
+    config(
+        materialized='incremental',
+        unique_key='id',
+        on_schema_change='sync_all_columns',
+        partition_by={
+            "field": "id",
+            "data_type": "int64",
+            "range": {
+                "start": 1,
+                "end": 6,
+                "interval": 1
+            }
+        },
+        incremental_strategy='insert_overwrite'
+    )
+}}
+
+WITH source_data AS (SELECT * FROM {{ ref('model_a') }} )
+
+{% set string_type = 'string' %}
+
+{% if is_incremental() %}
+
+SELECT id, 
+       cast(field1 as {{string_type}}) as field1, 
+       cast(field3 as {{string_type}}) as field3, -- to validate new fields
+       cast(field4 as {{string_type}}) AS field4 -- to validate new fields
+
+FROM source_data WHERE id > _dbt_max_partition
+
+{% else %}
+
+select id, 
+       cast(field1 as {{string_type}}) as field1, 
+       cast(field2 as {{string_type}}) as field2
+
+from source_data where id <= 3
+
+{% endif %}
\ No newline at end of file

From 07bc1442c2e0711822983999b18d68ccd029cd41 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Thu, 21 Oct 2021 18:18:47 +0200
Subject: [PATCH 392/860] Rm all logic for ingestion-partitioned tables (#42)

---
 dbt/adapters/bigquery/connections.py          | 46 ------------
 dbt/adapters/bigquery/impl.py                 | 24 -------
 .../macros/materializations/table.sql         | 71 ++++---------------
 .../bigquery_test/dp-models/confirmation.sql  | 20 ------
 .../dp-models/confirmation_noconfig.sql       | 20 ------
 .../dp-models/events_20180101.sql             |  4 --
 .../dp-models/events_20180102.sql             |  4 --
 .../dp-models/events_20180103.sql             |  4 --
 .../bigquery_test/dp-models/partitioned.sql   | 16 -----
 .../dp-models/partitioned_noconfig.sql        |  7 --
 .../dp-models/partitioned_simple.sql          | 14 ----
 .../bigquery_test/dp-models/schema.yml        | 33 ---------
 .../test_bigquery_date_partitioning.py        | 46 ------------
 13 files changed, 12 insertions(+), 297 deletions(-)
 delete mode 100644 tests/integration/bigquery_test/dp-models/confirmation.sql
 delete mode 100644 tests/integration/bigquery_test/dp-models/confirmation_noconfig.sql
 delete mode 100644 tests/integration/bigquery_test/dp-models/events_20180101.sql
 delete mode 100644 tests/integration/bigquery_test/dp-models/events_20180102.sql
 delete mode 100644 tests/integration/bigquery_test/dp-models/events_20180103.sql
 delete mode 100644 tests/integration/bigquery_test/dp-models/partitioned.sql
 delete mode 100644 tests/integration/bigquery_test/dp-models/partitioned_noconfig.sql
 delete mode 100644 tests/integration/bigquery_test/dp-models/partitioned_simple.sql
 delete mode 100644 tests/integration/bigquery_test/dp-models/schema.yml
 delete mode 100644 tests/integration/bigquery_test/test_bigquery_date_partitioning.py

diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 7a831ff96..265959400 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -419,52 +419,6 @@ def standard_to_legacy(table):
             self.raw_execute(sql, fetch='fetch_result', use_legacy_sql=True)
         return self.get_table_from_response(iterator)
 
-    def create_bigquery_table(self, database, schema, table_name, callback,
-                              sql):
-        """Create a bigquery table. The caller must supply a callback
-        that takes one argument, a `google.cloud.bigquery.Table`, and mutates
-        it.
-        """
-        conn = self.get_thread_connection()
-        client = conn.handle
-
-        view_ref = self.table_ref(database, schema, table_name, conn)
-        view = google.cloud.bigquery.Table(view_ref)
-        callback(view)
-
-        def fn():
-            return client.create_table(view)
-        self._retry_and_handle(msg=sql, conn=conn, fn=fn)
-
-    def create_view(self, database, schema, table_name, sql):
-        def callback(table):
-            table.view_query = sql
-            table.view_use_legacy_sql = False
-
-        self.create_bigquery_table(database, schema, table_name, callback, sql)
-
-    def create_table(self, database, schema, table_name, sql):
-        conn = self.get_thread_connection()
-        client = conn.handle
-
-        table_ref = self.table_ref(database, schema, table_name, conn)
-        job_params = {'destination': table_ref,
-                      'write_disposition': WRITE_TRUNCATE}
-
-        timeout = self.get_timeout(conn)
-
-        def fn():
-            return self._query_and_results(client, sql, conn, job_params,
-                                           timeout=timeout)
-        self._retry_and_handle(msg=sql, conn=conn, fn=fn)
-
-    def create_date_partitioned_table(self, database, schema, table_name):
-        def callback(table):
-            table.partitioning_type = 'DAY'
-
-        self.create_bigquery_table(database, schema, table_name, callback,
-                                   'CREATE DAY PARTITIONED TABLE')
-
     def copy_bq_table(self, source, destination, write_disposition):
         conn = self.get_thread_connection()
         client = conn.handle
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 33ec96443..f828c37f8 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -497,30 +497,6 @@ def add_query(self, sql, auto_begin=True, bindings=None,
     ###
     # Special bigquery adapter methods
     ###
-    @available.parse_none
-    def make_date_partitioned_table(self, relation):
-        return self.connections.create_date_partitioned_table(
-            database=relation.database,
-            schema=relation.schema,
-            table_name=relation.identifier
-        )
-
-    @available.parse(lambda *a, **k: '')
-    def execute_model(self, model, materialization, sql_override=None,
-                      decorator=None):
-
-        if sql_override is None:
-            sql_override = model.get('compiled_sql')
-
-        if materialization == 'view':
-            res = self._materialize_as_view(model)
-        elif materialization == 'table':
-            res = self._materialize_as_table(model, sql_override, decorator)
-        else:
-            msg = "Invalid relation type: '{}'".format(materialization)
-            raise dbt.exceptions.RuntimeException(msg, model)
-
-        return res
 
     def _partitions_match(
         self, table, conf_partition: Optional[PartitionConfig]
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index bbfe54771..a7452265b 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -1,79 +1,32 @@
-{% macro make_date_partitioned_table(model, relation, dates, should_create, verbose=False) %}
-
-  {% if should_create %}
-      {{ adapter.make_date_partitioned_table(relation) }}
-  {% endif %}
-
-  {% for date in dates %}
-    {% set date = (date | string) %}
-    {% if verbose %}
-        {% set table_start_time = modules.datetime.datetime.now().strftime("%H:%M:%S") %}
-        {{ log(table_start_time ~ ' | -> Running for day ' ~ date, info=True) }}
-    {% endif %}
-
-    {% set fixed_sql = model['compiled_sql'] | replace('[DBT__PARTITION_DATE]', date) %}
-    {% set _ = adapter.execute_model(model, 'table', fixed_sql, decorator=date) %}
-  {% endfor %}
-
-  {% set num_days = dates | length %}
-  {% if num_days == 1 %}
-      {% set result_str = 'CREATED 1 PARTITION' %}
-  {% else %}
-      {% set result_str = 'CREATED ' ~ num_days ~ ' PARTITIONS' %}
-  {% endif %}
-
-  {{ store_result('main', response=result_str) }}
-
-{% endmacro %}
-
 {% materialization table, adapter='bigquery' -%}
 
   {%- set identifier = model['alias'] -%}
   {%- set old_relation = adapter.get_relation(database=database, schema=schema, identifier=identifier) -%}
   {%- set exists_not_as_table = (old_relation is not none and not old_relation.is_table) -%}
   {%- set target_relation = api.Relation.create(database=database, schema=schema, identifier=identifier, type='table') -%}
-  {%- set verbose = config.get('verbose', False) -%}
-
-  {# partitions: iterate over each partition, running a separate query in a for-loop #}
-  {%- set partitions = config.get('partitions') -%}
-
-  {% if partitions %}
-      {% if partitions is number or partitions is string %}
-        {% set partitions = [(partitions | string)] %}
-      {% endif %}
-
-      {% if partitions is not iterable %}
-        {{ exceptions.raise_compiler_error("Provided `partitions` configuration is not a list. Got: " ~ partitions, model) }}
-      {% endif %}
-  {% endif %}
 
   {{ run_hooks(pre_hooks) }}
 
   {#
-      Since dbt uses WRITE_TRUNCATE mode for tables, we only need to drop this thing
-      if it is not a table. If it _is_ already a table, then we can overwrite it without downtime
+      We only need to drop this thing if it is not a table.
+      If it _is_ already a table, then we can overwrite it without downtime
+      Unlike table -> view, no need for `--full-refresh`: dropping a view is no big deal
   #}
   {%- if exists_not_as_table -%}
       {{ adapter.drop_relation(old_relation) }}
   {%- endif -%}
 
   -- build model
-  {% if partitions %}
-    {# Create the dp-table if 1. it does not exist or 2. it existed, but we just dropped it #}
-    {%- set should_create = (old_relation is none or exists_not_as_table) -%}
-    {{ make_date_partitioned_table(model, target_relation, partitions, should_create, verbose) }}
-  {% else %}
-    {%- set raw_partition_by = config.get('partition_by', none) -%}
-    {%- set partition_by = adapter.parse_partition_by(raw_partition_by) -%}
-    {%- set cluster_by = config.get('cluster_by', none) -%}
-    {% if not adapter.is_replaceable(old_relation, partition_by, cluster_by) %}
-      {% do log("Hard refreshing " ~ old_relation ~ " because it is not replaceable") %}
-      {% do adapter.drop_relation(old_relation) %}
-    {% endif %}
-    {% call statement('main') -%}
-      {{ create_table_as(False, target_relation, sql) }}
-    {% endcall -%}
+  {%- set raw_partition_by = config.get('partition_by', none) -%}
+  {%- set partition_by = adapter.parse_partition_by(raw_partition_by) -%}
+  {%- set cluster_by = config.get('cluster_by', none) -%}
+  {% if not adapter.is_replaceable(old_relation, partition_by, cluster_by) %}
+    {% do log("Hard refreshing " ~ old_relation ~ " because it is not replaceable") %}
+    {% do adapter.drop_relation(old_relation) %}
   {% endif %}
+  {% call statement('main') -%}
+    {{ create_table_as(False, target_relation, sql) }}
+  {% endcall -%}
 
   {{ run_hooks(post_hooks) }}
 
diff --git a/tests/integration/bigquery_test/dp-models/confirmation.sql b/tests/integration/bigquery_test/dp-models/confirmation.sql
deleted file mode 100644
index 858c0e2e3..000000000
--- a/tests/integration/bigquery_test/dp-models/confirmation.sql
+++ /dev/null
@@ -1,20 +0,0 @@
-
--- This model checks to confirm that each date partition was created correctly.
--- Columns day_1, day_2, and day_3 should have a value of 1, and count_days should be 3
-
-with base as (
-
-  select
-    case when _PARTITIONTIME = '2018-01-01' then 1 else 0 end as day_1,
-    case when _PARTITIONTIME = '2018-01-02' then 1 else 0 end as day_2,
-    case when _PARTITIONTIME = '2018-01-03' then 1 else 0 end as day_3
-  from {{ ref('partitioned') }}
-
-)
-
-select distinct
-  sum(day_1) over () as day_1,
-  sum(day_2) over () as day_2,
-  sum(day_3) over () as day_3,
-  count(*) over () as count_days
-from base
diff --git a/tests/integration/bigquery_test/dp-models/confirmation_noconfig.sql b/tests/integration/bigquery_test/dp-models/confirmation_noconfig.sql
deleted file mode 100644
index 7c061f4a5..000000000
--- a/tests/integration/bigquery_test/dp-models/confirmation_noconfig.sql
+++ /dev/null
@@ -1,20 +0,0 @@
-
--- This model checks to confirm that each date partition was created correctly.
--- Columns day_1, day_2, and day_3 should have a value of 1, and count_days should be 3
-
-with base as (
-
-  select
-    case when _PARTITIONTIME = '2018-01-01' then 1 else 0 end as day_1,
-    case when _PARTITIONTIME = '2018-01-02' then 1 else 0 end as day_2,
-    case when _PARTITIONTIME = '2018-01-03' then 1 else 0 end as day_3
-  from {{ ref('partitioned_noconfig') }}
-
-)
-
-select distinct
-  sum(day_1) over () as day_1,
-  sum(day_2) over () as day_2,
-  sum(day_3) over () as day_3,
-  count(*) over () as count_days
-from base
diff --git a/tests/integration/bigquery_test/dp-models/events_20180101.sql b/tests/integration/bigquery_test/dp-models/events_20180101.sql
deleted file mode 100644
index 9a8f54d5b..000000000
--- a/tests/integration/bigquery_test/dp-models/events_20180101.sql
+++ /dev/null
@@ -1,4 +0,0 @@
-
-{{ config(materialized='table') }}
-
-select 1 as id
diff --git a/tests/integration/bigquery_test/dp-models/events_20180102.sql b/tests/integration/bigquery_test/dp-models/events_20180102.sql
deleted file mode 100644
index 63bfcdc13..000000000
--- a/tests/integration/bigquery_test/dp-models/events_20180102.sql
+++ /dev/null
@@ -1,4 +0,0 @@
-
-{{ config(materialized='table') }}
-
-select 2 as id
diff --git a/tests/integration/bigquery_test/dp-models/events_20180103.sql b/tests/integration/bigquery_test/dp-models/events_20180103.sql
deleted file mode 100644
index 09a9f02c7..000000000
--- a/tests/integration/bigquery_test/dp-models/events_20180103.sql
+++ /dev/null
@@ -1,4 +0,0 @@
-
-{{ config(materialized='table') }}
-
-select 3 as id
diff --git a/tests/integration/bigquery_test/dp-models/partitioned.sql b/tests/integration/bigquery_test/dp-models/partitioned.sql
deleted file mode 100644
index 5d77021d3..000000000
--- a/tests/integration/bigquery_test/dp-models/partitioned.sql
+++ /dev/null
@@ -1,16 +0,0 @@
-
-{{
-    config(
-        materialized='table',
-        partitions=['20180101', '20180102', '20180103'],
-        verbose=True
-    )
-}}
-
--- Hack to make sure our events models run first.
--- In practice, these would be source data
--- {{ ref('events_20180101') }}
--- {{ ref('events_20180102') }}
--- {{ ref('events_20180103') }}
-
-select * from `{{ this.schema }}`.`{{ date_sharded_table('events_') }}`
diff --git a/tests/integration/bigquery_test/dp-models/partitioned_noconfig.sql b/tests/integration/bigquery_test/dp-models/partitioned_noconfig.sql
deleted file mode 100644
index c199d37a9..000000000
--- a/tests/integration/bigquery_test/dp-models/partitioned_noconfig.sql
+++ /dev/null
@@ -1,7 +0,0 @@
--- Hack to make sure our events models run first.
--- In practice, these would be source data
--- {{ ref('events_20180101') }}
--- {{ ref('events_20180102') }}
--- {{ ref('events_20180103') }}
-
-select * from `{{ this.schema }}`.`{{ date_sharded_table('events_') }}`
diff --git a/tests/integration/bigquery_test/dp-models/partitioned_simple.sql b/tests/integration/bigquery_test/dp-models/partitioned_simple.sql
deleted file mode 100644
index af65072be..000000000
--- a/tests/integration/bigquery_test/dp-models/partitioned_simple.sql
+++ /dev/null
@@ -1,14 +0,0 @@
-
-{{
-    config(
-        materialized='table',
-        partition_date='20180101',
-        verbose=True
-    )
-}}
-
--- Hack to make sure our events models run first.
--- In practice, these would be source data
--- {{ ref('events_20180101') }}
-
-select * from `{{ this.schema }}`.`events_20180101`
diff --git a/tests/integration/bigquery_test/dp-models/schema.yml b/tests/integration/bigquery_test/dp-models/schema.yml
deleted file mode 100644
index 3daf468dc..000000000
--- a/tests/integration/bigquery_test/dp-models/schema.yml
+++ /dev/null
@@ -1,33 +0,0 @@
-version: 2
-models:
-- name: partitioned_simple
-  columns:
-  - name: id
-    tests:
-    - not_null
-    - unique
-- name: confirmation
-  columns: &conf_columns
-  - name: cast(count_days as string)
-    tests:
-    - accepted_values:
-        values:
-        - 3
-  - name: cast(day_1 as string)
-    tests:
-    - accepted_values:
-        values:
-        - 1
-  - name: cast(day_2 as string)
-    tests:
-    - accepted_values:
-        values:
-        - 1
-  - name: cast(day_3 as string)
-    tests:
-    - accepted_values:
-        values:
-        - 1
-
-- name: confirmation_noconfig
-  columns: *conf_columns
diff --git a/tests/integration/bigquery_test/test_bigquery_date_partitioning.py b/tests/integration/bigquery_test/test_bigquery_date_partitioning.py
deleted file mode 100644
index fad0b50a0..000000000
--- a/tests/integration/bigquery_test/test_bigquery_date_partitioning.py
+++ /dev/null
@@ -1,46 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-import textwrap
-import yaml
-
-
-class TestBigqueryDatePartitioning(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "dp-models"
-
-    @property
-    def profile_config(self):
-        return self.bigquery_profile()
-
-    @property
-    def project_config(self):
-        return yaml.safe_load(textwrap.dedent('''\
-        config-version: 2
-        models:
-            test:
-                partitioned_noconfig:
-                    materialized: table
-                    partitions:
-                        - 20180101
-                        - 20180102
-                        - 20180103
-                    verbose: true
-        '''))
-
-    @use_profile('bigquery')
-    def test__bigquery_date_partitioning(self):
-        results = self.run_dbt()
-        self.assertEqual(len(results), 8)
-
-        test_results = self.run_dbt(['test'])
-
-        self.assertTrue(len(test_results) > 0)
-        for result in test_results:
-            self.assertEqual(result.status, 'pass')
-            self.assertFalse(result.skipped)
-            self.assertEqual(result.failures, 0)

From 76f658fc902ec1b062ffca90611fa5d7813b3cf1 Mon Sep 17 00:00:00 2001
From: Jeremy Yeo 
Date: Fri, 22 Oct 2021 21:41:33 +1300
Subject: [PATCH 393/860] Raise DbtProfileError on invalid default credential
 (#40)

* raise profile error

* add deps test

* remove test

* Add changelog entry

Co-authored-by: Jeremy Cohen 
---
 CHANGELOG.md                         | 2 ++
 dbt/adapters/bigquery/connections.py | 9 +++++++--
 2 files changed, 9 insertions(+), 2 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 1ef13dea2..029b7f82a 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -7,10 +7,12 @@
 ### Fixes
 
 - Reimplement the `unique` test to handle column expressions and naming overlaps ([#33](https://github.com/dbt-labs/dbt-bigquery/issues/33), [#35](https://github.com/dbt-labs/dbt-bigquery/issues/35), [#10](https://github.com/dbt-labs/dbt-bigquery/pull/10))
+- Avoid error in `dbt deps` + `dbt clean` if default project is missing ([#27](https://github.com/dbt-labs/dbt-bigquery/issues/27), [#40](https://github.com/dbt-labs/dbt-bigquery/pull/40))
 
 ### Contributors
 
 - [@DigUpTheHatchet](https://github.com/DigUpTheHatchet) ([#10](https://github.com/dbt-labs/dbt-bigquery/pull/10))
+- [@jeremyyeo](https://github.com/jeremyyeo) ([#40](https://github.com/dbt-labs/dbt-bigquery/pull/40))
 
 ## dbt-bigquery 1.0.0b1 (October 11, 2021)
 
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 265959400..d6988103c 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -21,10 +21,11 @@
 from dbt.adapters.bigquery import gcloud
 from dbt.utils import format_bytes, format_rows_number
 from dbt.clients import agate_helper
+from dbt.config.profile import INVALID_PROFILE_MESSAGE
 from dbt.tracking import active_user
 from dbt.contracts.connection import ConnectionState, AdapterResponse
 from dbt.exceptions import (
-    FailedToConnectException, RuntimeException, DatabaseException
+    FailedToConnectException, RuntimeException, DatabaseException, DbtProfileError
 )
 from dbt.adapters.base import BaseConnectionManager, Credentials
 from dbt.logger import GLOBAL_LOGGER as logger
@@ -58,7 +59,11 @@ def get_bigquery_defaults(scopes=None) -> Tuple[Any, Optional[str]]:
     project_id is returned available from the environment; otherwise None
     """
     # Cached, because the underlying implementation shells out, taking ~1s
-    return google.auth.default(scopes=scopes)
+    try:
+        credentials, _ = google.auth.default(scopes=scopes)
+        return credentials, _
+    except google.auth.exceptions.DefaultCredentialsError as e:
+        raise DbtProfileError(INVALID_PROFILE_MESSAGE.format(error_string=e))
 
 
 class Priority(StrEnum):

From 991595343e0314a2a5d6c56e5bc9f4fe05f76bd5 Mon Sep 17 00:00:00 2001
From: Niall Woodward 
Date: Mon, 25 Oct 2021 18:16:22 +0100
Subject: [PATCH 394/860] Create profile_template.yml (#43)

* Create target_options.yml

* Update to latest config
---
 dbt/include/bigquery/profile_template.yml | 27 +++++++++++++++++++++++
 dbt/include/bigquery/sample_profiles.yml  | 26 ----------------------
 2 files changed, 27 insertions(+), 26 deletions(-)
 create mode 100644 dbt/include/bigquery/profile_template.yml
 delete mode 100644 dbt/include/bigquery/sample_profiles.yml

diff --git a/dbt/include/bigquery/profile_template.yml b/dbt/include/bigquery/profile_template.yml
new file mode 100644
index 000000000..4a9cdc7c9
--- /dev/null
+++ b/dbt/include/bigquery/profile_template.yml
@@ -0,0 +1,27 @@
+fixed:
+  type: bigquery
+  priority: interactive
+  fixed_retries: 1
+prompts:
+  _choose_authentication_method:
+    oauth:
+      _fixed_method: oauth
+    service_account:
+      _fixed_method: service-account
+      keyfile:
+        hint: '/path/to/bigquery/keyfile.json'
+  project:
+    hint: 'GCP project id'
+  dataset:
+    hint: 'the name of your dbt dataset'
+  threads:
+    hint: '1 or more'
+    type: 'int'
+  timeout_seconds:
+    default: 300
+    type: 'int'
+  _choose_location:
+    US:
+      _fixed_location: US
+    EU:
+      _fixed_location: EU
diff --git a/dbt/include/bigquery/sample_profiles.yml b/dbt/include/bigquery/sample_profiles.yml
deleted file mode 100644
index 36f47fdbc..000000000
--- a/dbt/include/bigquery/sample_profiles.yml
+++ /dev/null
@@ -1,26 +0,0 @@
-default:
-  outputs:
-
-    dev:
-      type: bigquery
-      method: oauth
-      project: [GCP project id]
-      dataset: [the name of your dbt dataset] # You can also use "schema" here
-      threads: [1 or more]
-      timeout_seconds: 300
-      location: US # Optional, one of US or EU
-      priority: interactive
-      retries: 1
-
-    prod:
-      type: bigquery
-      method: service-account
-      project: [GCP project id]
-      dataset: [the name of your dbt dataset]
-      threads: [1 or more]
-      keyfile: [/path/to/bigquery/keyfile.json]
-      timeout_seconds: 300
-      priority: interactive
-      retries: 1
-
-  target: dev
\ No newline at end of file

From cec38c67ddc54022ff9f1216bd2bcb048cc647fe Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Mon, 25 Oct 2021 19:26:48 +0200
Subject: [PATCH 395/860] Update changelog

---
 CHANGELOG.md | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 029b7f82a..6c832ae9c 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -9,12 +9,16 @@
 - Reimplement the `unique` test to handle column expressions and naming overlaps ([#33](https://github.com/dbt-labs/dbt-bigquery/issues/33), [#35](https://github.com/dbt-labs/dbt-bigquery/issues/35), [#10](https://github.com/dbt-labs/dbt-bigquery/pull/10))
 - Avoid error in `dbt deps` + `dbt clean` if default project is missing ([#27](https://github.com/dbt-labs/dbt-bigquery/issues/27), [#40](https://github.com/dbt-labs/dbt-bigquery/pull/40))
 
+### Under the hood
+- Replace `sample_profiles.yml` with `profile_template.yml`, for use with new `dbt init` ([#43](https://github.com/dbt-labs/dbt-bigquery/pull/43))
+
 ### Contributors
 
 - [@DigUpTheHatchet](https://github.com/DigUpTheHatchet) ([#10](https://github.com/dbt-labs/dbt-bigquery/pull/10))
 - [@jeremyyeo](https://github.com/jeremyyeo) ([#40](https://github.com/dbt-labs/dbt-bigquery/pull/40))
+- [@NiallRees](https://github.com/NiallRees) ([#43](https://github.com/dbt-labs/dbt-bigquery/pull/43))
 
 ## dbt-bigquery 1.0.0b1 (October 11, 2021)
 
 ### Under the hood
-First version as a separate repo.
+- Initial adapter split out

From ac9e1f8d5b16bedc647cd9fc32a241a95a537238 Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Mon, 25 Oct 2021 13:57:06 -0400
Subject: [PATCH 396/860] Bumping version to 1.0.0b2 (#44)

Co-authored-by: Github Build Bot 
---
 .bumpversion.cfg                     | 2 +-
 dbt/adapters/bigquery/__version__.py | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 39b23539a..bd1104ce6 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.0.0b1
+current_version = 1.0.0b2
 parse = (?P\d+)
 	\.(?P\d+)
 	\.(?P\d+)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 6f075105d..50ece64e0 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = '1.0.0b1'
+version = '1.0.0b2'

From a9b5baa32d82f61977a4ca1a9671f79b8ae225f2 Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Mon, 25 Oct 2021 15:49:30 -0400
Subject: [PATCH 397/860] Bumping changelog version for 1.0.0b2

---
 CHANGELOG.md | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 6c832ae9c..07d711ee4 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,7 @@
 ## dbt-bigquery 1.0.0 (Release TBD)
 
+## dbt-bigquery 1.0.0b2 (October 25, 2021)
+
 ### Features
 
 - Rework `_dbt_max_partition` logic in dynamic `insert_overwrite` incremental strategy. Make the default logic compatible with `on_schema_change`, and make it possible to disable or reimplement that logic by defining a custom macro `declare_dbt_max_partition` ([#17](https://github.com/dbt-labs/dbt-bigquery/issues/17), [#39](https://github.com/dbt-labs/dbt-bigquery/issues/39), [#41](https://github.com/dbt-labs/dbt-bigquery/pull/41))

From 0036da2b682822bd7c6d3e34702d6c62b280ccd3 Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Mon, 25 Oct 2021 16:01:36 -0400
Subject: [PATCH 398/860] Adding changelog sections

---
 CHANGELOG.md | 12 ++++++++++++
 1 file changed, 12 insertions(+)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 07d711ee4..97625bf73 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,17 @@
 ## dbt-bigquery 1.0.0 (Release TBD)
 
+### Features
+N/A
+
+### Fixes
+N/A
+
+### Under the hood
+N/A
+
+### Contributors
+N/A
+
 ## dbt-bigquery 1.0.0b2 (October 25, 2021)
 
 ### Features

From bb9caab8fd771d72bafdcfffb1ae72459ad704a2 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Wed, 27 Oct 2021 15:32:43 +0200
Subject: [PATCH 399/860] Fix tests for dbt-labs/dbt-core#3955 (#46)

---
 tests/integration/docs_generate_tests/test_docs_generate.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/tests/integration/docs_generate_tests/test_docs_generate.py b/tests/integration/docs_generate_tests/test_docs_generate.py
index e05424489..e018c42e9 100644
--- a/tests/integration/docs_generate_tests/test_docs_generate.py
+++ b/tests/integration/docs_generate_tests/test_docs_generate.py
@@ -1165,7 +1165,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
                     'database': self.default_database,
                     'description': 'My table',
                     'external': None,
-                    'freshness': {'error_after': None, 'warn_after': None, 'filter': None},
+                    'freshness': {'error_after': {'count': None, 'period': None}, 'warn_after': {'count': None, 'period': None}, 'filter': None},
                     'identifier': 'seed',
                     'loaded_at_field': None,
                     'loader': 'a_loader',

From d6369963a25fdfc188b205c20bc4191bbe49faa1 Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Wed, 3 Nov 2021 09:29:16 -0400
Subject: [PATCH 400/860] Change Bigquery google-api-core max version
 dependency (#53)

* Change Bigquery google-api-core max version dependency

* Update CHANGELOG.md
---
 CHANGELOG.md | 2 +-
 setup.py     | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 97625bf73..c3485ced7 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -4,7 +4,7 @@
 N/A
 
 ### Fixes
-N/A
+- Capping `google-api-core` to version `1.31.3` due to `protobuf` dependency conflict ([#53](https://github.com/dbt-labs/dbt-bigquery/pull/53))
 
 ### Under the hood
 N/A
diff --git a/setup.py b/setup.py
index c7b7aeef0..950874ef2 100644
--- a/setup.py
+++ b/setup.py
@@ -77,7 +77,7 @@ def _get_dbt_core_version():
         'protobuf>=3.13.0,<4',
         'google-cloud-core>=1.3.0,<2',
         'google-cloud-bigquery>=1.25.0,<3',
-        'google-api-core>=1.16.0,<2',
+        'google-api-core>=1.16.0,<1.31.3',
         'googleapis-common-protos>=1.6.0,<2',
         'six>=1.14.0',
     ],

From 559bec290232c0c444a8228d989e2702c6c69569 Mon Sep 17 00:00:00 2001
From: imartynetz 
Date: Fri, 5 Nov 2021 10:47:28 -0300
Subject: [PATCH 401/860] Fix problem with row access policy doesnt return
 bytes proccessed. (#48)

* FIX: Fix problem with row access policy doesnt return bytes proccessed.

* UPDATE: Update changelog with fix.

* UPDATE: Change methods format_row_number and format_bytes from core to bigquery/connections

* Update changelog

Co-authored-by: Jeremy Cohen 
---
 CHANGELOG.md                         |  8 +++---
 dbt/adapters/bigquery/connections.py | 41 ++++++++++++++++++++--------
 2 files changed, 33 insertions(+), 16 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index c3485ced7..6b14c2450 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -4,13 +4,14 @@
 N/A
 
 ### Fixes
-- Capping `google-api-core` to version `1.31.3` due to `protobuf` dependency conflict ([#53](https://github.com/dbt-labs/dbt-bigquery/pull/53))
+- Fix problem with bytes proccessed return None value when the service account used to connect DBT in bigquery had a row policy access.
+([#47](https://github.com/dbt-labs/dbt-bigquery/issues/47), [#48](https://github.com/dbt-labs/dbt-bigquery/pull/48))
 
 ### Under the hood
-N/A
+- Capping `google-api-core` to version `1.31.3` due to `protobuf` dependency conflict ([#53](https://github.com/dbt-labs/dbt-bigquery/pull/53))
 
 ### Contributors
-N/A
+- [@imartynetz](https://github.com/imartynetz) ([#48](https://github.com/dbt-labs/dbt-bigquery/pull/48))
 
 ## dbt-bigquery 1.0.0b2 (October 25, 2021)
 
@@ -19,7 +20,6 @@ N/A
 - Rework `_dbt_max_partition` logic in dynamic `insert_overwrite` incremental strategy. Make the default logic compatible with `on_schema_change`, and make it possible to disable or reimplement that logic by defining a custom macro `declare_dbt_max_partition` ([#17](https://github.com/dbt-labs/dbt-bigquery/issues/17), [#39](https://github.com/dbt-labs/dbt-bigquery/issues/39), [#41](https://github.com/dbt-labs/dbt-bigquery/pull/41))
 
 ### Fixes
-
 - Reimplement the `unique` test to handle column expressions and naming overlaps ([#33](https://github.com/dbt-labs/dbt-bigquery/issues/33), [#35](https://github.com/dbt-labs/dbt-bigquery/issues/35), [#10](https://github.com/dbt-labs/dbt-bigquery/pull/10))
 - Avoid error in `dbt deps` + `dbt clean` if default project is missing ([#27](https://github.com/dbt-labs/dbt-bigquery/issues/27), [#40](https://github.com/dbt-labs/dbt-bigquery/pull/40))
 
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index d6988103c..4a69b35b2 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -19,7 +19,6 @@
 )
 
 from dbt.adapters.bigquery import gcloud
-from dbt.utils import format_bytes, format_rows_number
 from dbt.clients import agate_helper
 from dbt.config.profile import INVALID_PROFILE_MESSAGE
 from dbt.tracking import active_user
@@ -215,6 +214,28 @@ def begin(self):
     def commit(self):
         pass
 
+    def format_bytes(self, num_bytes):
+        if num_bytes:
+            for unit in ['Bytes', 'KB', 'MB', 'GB', 'TB', 'PB']:
+                if abs(num_bytes) < 1024.0:
+                    return f"{num_bytes:3.1f} {unit}"
+                num_bytes /= 1024.0
+
+            num_bytes *= 1024.0
+            return f"{num_bytes:3.1f} {unit}"
+
+        else:
+            return num_bytes
+
+    def format_rows_number(self, rows_number):
+        for unit in ['', 'k', 'm', 'b', 't']:
+            if abs(rows_number) < 1000.0:
+                return f"{rows_number:3.1f}{unit}".strip()
+            rows_number /= 1000.0
+
+        rows_number *= 1000.0
+        return f"{rows_number:3.1f}{unit}".strip()
+
     @classmethod
     def get_bigquery_credentials(cls, profile_credentials):
         method = profile_credentials.method
@@ -380,27 +401,23 @@ def execute(
             query_table = client.get_table(query_job.destination)
             code = 'CREATE TABLE'
             num_rows = query_table.num_rows
+            num_rows_formated = self.format_rows_number(num_rows)
             bytes_processed = query_job.total_bytes_processed
-            message = '{} ({} rows, {} processed)'.format(
-                code,
-                format_rows_number(num_rows),
-                format_bytes(bytes_processed)
-            )
+            processed_bytes = self.format_bytes(bytes_processed)
+            message = f'{code} ({num_rows_formated} rows, {processed_bytes} processed)'
 
         elif query_job.statement_type == 'SCRIPT':
             code = 'SCRIPT'
             bytes_processed = query_job.total_bytes_processed
-            message = f'{code} ({format_bytes(bytes_processed)} processed)'
+            message = f'{code} ({self.format_bytes(bytes_processed)} processed)'
 
         elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']:
             code = query_job.statement_type
             num_rows = query_job.num_dml_affected_rows
+            num_rows_formated = self.format_rows_number(num_rows)
             bytes_processed = query_job.total_bytes_processed
-            message = '{} ({} rows, {} processed)'.format(
-                code,
-                format_rows_number(num_rows),
-                format_bytes(bytes_processed),
-            )
+            processed_bytes = self.format_bytes(bytes_processed)
+            message = f'{code} ({num_rows_formated} rows, {processed_bytes} processed)'
 
         response = BigQueryAdapterResponse(
             _message=message,

From bff9df32b1f2b74e1a5431cf2e21874b6baabb12 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Sun, 7 Nov 2021 13:08:12 +0100
Subject: [PATCH 402/860] Install dbt-core first (#58)

---
 .github/workflows/integration.yml | 4 ++++
 .github/workflows/main.yml        | 4 ++++
 2 files changed, 8 insertions(+)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index e0f0e5e6e..b09fcb225 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -157,6 +157,10 @@ jobs:
           pip install tox
           pip --version
           tox --version
+          
+      - name: Install dbt-core latest
+        run: |
+          pip install "git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core"
 
       - name: Run tox (bigquery)
         if: matrix.adapter == 'bigquery'
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 88d9e3653..bb2314d81 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -100,6 +100,10 @@ jobs:
           pip install tox
           pip --version
           tox --version
+          
+      - name: Install dbt-core latest
+        run: |
+          pip install "git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core"
 
       - name: Run tox
         run: tox

From 75eaaba75c200403aefe40f6369add8a229dfaa5 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Sun, 7 Nov 2021 13:08:39 +0100
Subject: [PATCH 403/860] Bump artifact versions in tests (#56)

* Install dbt-core first

* Bump artifact versions
---
 .../integration/docs_generate_tests/test_docs_generate.py | 8 ++++----
 tests/integration/sources_test/test_sources.py            | 2 +-
 2 files changed, 5 insertions(+), 5 deletions(-)

diff --git a/tests/integration/docs_generate_tests/test_docs_generate.py b/tests/integration/docs_generate_tests/test_docs_generate.py
index e018c42e9..027475af8 100644
--- a/tests/integration/docs_generate_tests/test_docs_generate.py
+++ b/tests/integration/docs_generate_tests/test_docs_generate.py
@@ -715,7 +715,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
         )
 
         return {
-            'dbt_schema_version': 'https://schemas.getdbt.com/dbt/manifest/v3.json',
+            'dbt_schema_version': 'https://schemas.getdbt.com/dbt/manifest/v4.json',
             'dbt_version': dbt.version.__version__,
             'nodes': {
                 'model.test.model': {
@@ -1300,7 +1300,7 @@ def expected_bigquery_complex_manifest(self):
         my_schema_name = self.unique_schema()
 
         return {
-            'dbt_schema_version': 'https://schemas.getdbt.com/dbt/manifest/v3.json',
+            'dbt_schema_version': 'https://schemas.getdbt.com/dbt/manifest/v4.json',
             'dbt_version': dbt.version.__version__,
             'nodes': {
                 'model.test.clustered': {
@@ -1820,7 +1820,7 @@ def verify_manifest(self, expected_manifest):
             elif key == 'metadata':
                 metadata = manifest['metadata']
                 self.verify_metadata(
-                    metadata, 'https://schemas.getdbt.com/dbt/manifest/v3.json')
+                    metadata, 'https://schemas.getdbt.com/dbt/manifest/v4.json')
                 assert 'project_id' in metadata and metadata[
                     'project_id'] == '098f6bcd4621d373cade4e832627b4f6'
                 assert 'send_anonymous_usage_stats' in metadata and metadata[
@@ -1916,7 +1916,7 @@ def verify_run_results(self, expected_run_results):
         run_results = _read_json('./target/run_results.json')
         assert 'metadata' in run_results
         self.verify_metadata(
-            run_results['metadata'], 'https://schemas.getdbt.com/dbt/run-results/v3.json')
+            run_results['metadata'], 'https://schemas.getdbt.com/dbt/run-results/v4.json')
         self.assertIn('elapsed_time', run_results)
         self.assertGreater(run_results['elapsed_time'], 0)
         self.assertTrue(
diff --git a/tests/integration/sources_test/test_sources.py b/tests/integration/sources_test/test_sources.py
index 553e91287..ec4dd75f6 100644
--- a/tests/integration/sources_test/test_sources.py
+++ b/tests/integration/sources_test/test_sources.py
@@ -104,7 +104,7 @@ def _assert_freshness_results(self, path, state):
         assert isinstance(data['elapsed_time'], float)
         self.assertBetween(data['metadata']['generated_at'],
                            self.freshness_start_time)
-        assert data['metadata']['dbt_schema_version'] == 'https://schemas.getdbt.com/dbt/sources/v2.json'
+        assert data['metadata']['dbt_schema_version'] == 'https://schemas.getdbt.com/dbt/sources/v3.json'
         assert data['metadata']['dbt_version'] == dbt.version.__version__
         assert data['metadata']['invocation_id'] == dbt.tracking.active_user.invocation_id
         key = 'key'

From bc31baeb8f5ea35ee57f21cde6d6140ec9819b9c Mon Sep 17 00:00:00 2001
From: Christophe Oudar 
Date: Mon, 8 Nov 2021 11:36:16 +0100
Subject: [PATCH 404/860] Use common columns for incremental schema changes
 (#51)

* Use common columns for incremental schema changes

* Review changes

* Follow up change for lean return version
---
 CHANGELOG.md                                  |  4 ++-
 .../macros/materializations/incremental.sql   |  8 ++++--
 .../models/incremental_append_new_columns.sql |  6 ++--
 ...remental_append_new_columns_remove_one.sql | 28 +++++++++++++++++++
 ...l_append_new_columns_remove_one_target.sql | 19 +++++++++++++
 .../models/schema.yml                         | 14 ++++++++++
 .../test_incremental_schema.py                | 16 +++++++++++
 ...remental_append_new_columns_remove_one.sql |  1 +
 ...l_append_new_columns_remove_one_target.sql |  1 +
 9 files changed, 90 insertions(+), 7 deletions(-)
 create mode 100644 tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one.sql
 create mode 100644 tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one_target.sql
 create mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one.sql
 create mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one_target.sql

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 6b14c2450..1ecda9392 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -4,14 +4,16 @@
 N/A
 
 ### Fixes
-- Fix problem with bytes proccessed return None value when the service account used to connect DBT in bigquery had a row policy access.
+- Fix problem with bytes processed return None value when the service account used to connect DBT in bigquery had a row policy access.
 ([#47](https://github.com/dbt-labs/dbt-bigquery/issues/47), [#48](https://github.com/dbt-labs/dbt-bigquery/pull/48))
+- When on_schema_change is set, pass common columns as dest_columns in incremental merge macros ([#4144](https://github.com/dbt-labs/dbt-core/issues/4144))
 
 ### Under the hood
 - Capping `google-api-core` to version `1.31.3` due to `protobuf` dependency conflict ([#53](https://github.com/dbt-labs/dbt-bigquery/pull/53))
 
 ### Contributors
 - [@imartynetz](https://github.com/imartynetz) ([#48](https://github.com/dbt-labs/dbt-bigquery/pull/48))
+- [@Kayrnt](https://github.com/Kayrnt) ([#51](https://github.com/dbt-labs/dbt-bigquery/pull/51))
 
 ## dbt-bigquery 1.0.0b2 (October 25, 2021)
 
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index 4c826e8da..56811234f 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -177,10 +177,12 @@
         declare_dbt_max_partition(this, partition_by, sql) + create_table_as(True, tmp_relation, sql)
       ) %}
       {% set tmp_relation_exists = true %}
-      {% do process_schema_changes(on_schema_change, tmp_relation, existing_relation) %}
+      {#-- Process schema changes. Returns dict of changes if successful. Use source columns for upserting/merging --#}
+      {% set dest_columns = process_schema_changes(on_schema_change, tmp_relation, existing_relation) %}
+    {% endif %}
+    {% if not dest_columns %}
+      {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %}
     {% endif %}
-    
-    {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %}
     {% set build_sql = bq_generate_incremental_build_sql(
         strategy, tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
     ) %}
diff --git a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns.sql b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns.sql
index 6111d37d0..351a397b9 100644
--- a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns.sql
+++ b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns.sql
@@ -13,9 +13,9 @@ WITH source_data AS (SELECT * FROM {{ ref('model_a') }} )
 {% if is_incremental()  %}
 
 SELECT id, 
-       cast(field1 as {{string_type}}) as field1, 
-       cast(field2 as {{string_type}}) as field2, 
-       cast(field3 as {{string_type}}) as field3, 
+       cast(field1 as {{string_type}}) as field1,
+       cast(field2 as {{string_type}}) as field2,
+       cast(field3 as {{string_type}}) as field3,
        cast(field4 as {{string_type}}) as field4 
 FROM source_data WHERE id NOT IN (SELECT id from {{ this }} )
 
diff --git a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one.sql b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one.sql
new file mode 100644
index 000000000..2ff6c6f48
--- /dev/null
+++ b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one.sql
@@ -0,0 +1,28 @@
+{{
+    config(
+        materialized='incremental',
+        unique_key='id',
+        on_schema_change='append_new_columns'
+    )
+}}
+
+{% set string_type = 'string' %}
+
+WITH source_data AS (SELECT * FROM {{ ref('model_a') }} )
+
+{% if is_incremental()  %}
+
+SELECT id,
+       cast(field1 as {{string_type}}) as field1,
+       cast(field3 as {{string_type}}) as field3,
+       cast(field4 as {{string_type}}) as field4
+FROM source_data WHERE id NOT IN (SELECT id from {{ this }} )
+
+{% else %}
+
+SELECT id,
+       cast(field1 as {{string_type}}) as field1,
+       cast(field2 as {{string_type}}) as field2
+FROM source_data where id <= 3
+
+{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one_target.sql b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one_target.sql
new file mode 100644
index 000000000..c70029d9b
--- /dev/null
+++ b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one_target.sql
@@ -0,0 +1,19 @@
+{{
+    config(materialized='table')
+}}
+
+{% set string_type = 'string' %}
+
+with source_data as (
+
+    select * from {{ ref('model_a') }}
+
+)
+
+select id,
+       cast(field1 as {{string_type}}) as field1,
+       cast(CASE WHEN id >  3 THEN NULL ELSE field2 END as {{string_type}}) AS field2,
+       cast(CASE WHEN id <= 3 THEN NULL ELSE field3 END as {{string_type}}) AS field3,
+       cast(CASE WHEN id <= 3 THEN NULL ELSE field4 END as {{string_type}}) AS field4
+
+from source_data
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/schema.yml b/tests/integration/incremental_schema_tests/models/schema.yml
index 5546314e4..21aa6095f 100644
--- a/tests/integration/incremental_schema_tests/models/schema.yml
+++ b/tests/integration/incremental_schema_tests/models/schema.yml
@@ -35,6 +35,20 @@ models:
         tags: [column_level_tag]
         tests:
           - unique
+
+  - name: incremental_append_new_columns_remove_one
+    columns:
+      - name: id
+        tags: [column_level_tag]
+        tests:
+          - unique
+
+  - name: incremental_append_new_columns_remove_one_target
+    columns:
+      - name: id
+        tags: [column_level_tag]
+        tests:
+          - unique
  
   - name: incremental_sync_all_columns
     columns:
diff --git a/tests/integration/incremental_schema_tests/test_incremental_schema.py b/tests/integration/incremental_schema_tests/test_incremental_schema.py
index a25b2484d..113a53585 100644
--- a/tests/integration/incremental_schema_tests/test_incremental_schema.py
+++ b/tests/integration/incremental_schema_tests/test_incremental_schema.py
@@ -87,6 +87,21 @@ def run_incremental_append_new_columns(self):
         self.list_tests_and_assert(select, exclude, expected)
         self.run_tests_and_assert(select, exclude, expected, compare_source, compare_target)
 
+    def run_incremental_append_new_columns_remove_one(self):
+        select = 'model_a incremental_append_new_columns_remove_one incremental_append_new_columns_remove_one_target'
+        compare_source = 'incremental_append_new_columns_remove_one'
+        compare_target = 'incremental_append_new_columns_remove_one_target'
+        exclude = None
+        expected = [
+            'select_from_a',
+            'select_from_incremental_append_new_columns_remove_one',
+            'select_from_incremental_append_new_columns_remove_one_target',
+            'unique_model_a_id',
+            'unique_incremental_append_new_columns_remove_one_id',
+            'unique_incremental_append_new_columns_remove_one_target_id'
+        ]
+        self.run_tests_and_assert(select, exclude, expected, compare_source, compare_target)
+
     def run_incremental_sync_all_columns(self):
         select = 'model_a incremental_sync_all_columns incremental_sync_all_columns_target'
         compare_source = 'incremental_sync_all_columns'
@@ -116,6 +131,7 @@ def test__bigquery__run_incremental_ignore(self):
     @use_profile('bigquery')
     def test__bigquery__run_incremental_append_new_columns(self):
         self.run_incremental_append_new_columns()
+        self.run_incremental_append_new_columns_remove_one()
 
     @use_profile('bigquery')
     def test__bigquery__run_incremental_sync_all_columns(self):
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one.sql
new file mode 100644
index 000000000..06d52c6d6
--- /dev/null
+++ b/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one.sql
@@ -0,0 +1 @@
+select * from {{ ref('incremental_append_new_columns_remove_one') }} where false
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one_target.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one_target.sql
new file mode 100644
index 000000000..07d2412b0
--- /dev/null
+++ b/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one_target.sql
@@ -0,0 +1 @@
+select * from {{ ref('incremental_append_new_columns_remove_one_target') }} where false
\ No newline at end of file

From b2bf0f94a06ffe0d22a0f6e8ed86ad3151ffaed9 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Mon, 8 Nov 2021 17:28:19 +0100
Subject: [PATCH 405/860] Bump google dependency upper bounds, remove `six`
 (#57)

* Bump google deps, rm six

* Update changelog
---
 CHANGELOG.md | 1 +
 setup.py     | 5 ++---
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 1ecda9392..99c567717 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -10,6 +10,7 @@ N/A
 
 ### Under the hood
 - Capping `google-api-core` to version `1.31.3` due to `protobuf` dependency conflict ([#53](https://github.com/dbt-labs/dbt-bigquery/pull/53))
+- Bump `google-cloud-core` and `google-api-core` upper bounds to `<3`, thereby removing `<1.31.3` limit on the latter. Remove explicit dependency on `six` ([#57](https://github.com/dbt-labs/dbt-bigquery/pull/57))
 
 ### Contributors
 - [@imartynetz](https://github.com/imartynetz) ([#48](https://github.com/dbt-labs/dbt-bigquery/pull/48))
diff --git a/setup.py b/setup.py
index 950874ef2..f1bc12ddc 100644
--- a/setup.py
+++ b/setup.py
@@ -75,11 +75,10 @@ def _get_dbt_core_version():
     install_requires=[
         'dbt-core~={}'.format(dbt_core_version),
         'protobuf>=3.13.0,<4',
-        'google-cloud-core>=1.3.0,<2',
+        'google-cloud-core>=1.3.0,<3',
         'google-cloud-bigquery>=1.25.0,<3',
-        'google-api-core>=1.16.0,<1.31.3',
+        'google-api-core>=1.16.0,<3',
         'googleapis-common-protos>=1.6.0,<2',
-        'six>=1.14.0',
     ],
     zip_safe=False,
     classifiers=[

From d92a7a2ac768be9189b967d86657f1f7fcffcdc1 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Tue, 9 Nov 2021 00:55:36 +0100
Subject: [PATCH 406/860] Fix tests failing due to dbt-core changes (#60)

* Update tests/unit/utils.py per dbt-core#4212

* Update docs_gen test for dbt-core#4154

* Update docs_gen test for dbt-core#4162
---
 .../docs_generate_tests/test_docs_generate.py | 13 ++++++++-----
 tests/unit/utils.py                           | 19 +++++++++++--------
 2 files changed, 19 insertions(+), 13 deletions(-)

diff --git a/tests/integration/docs_generate_tests/test_docs_generate.py b/tests/integration/docs_generate_tests/test_docs_generate.py
index 027475af8..bb8df64bb 100644
--- a/tests/integration/docs_generate_tests/test_docs_generate.py
+++ b/tests/integration/docs_generate_tests/test_docs_generate.py
@@ -627,8 +627,8 @@ def unrendered_tst_config(self, **updates):
     def _verify_generic_macro_structure(self, manifest):
         # just test a known global macro to avoid having to update this every
         # time they change.
-        self.assertIn('macro.dbt.column_list', manifest['macros'])
-        macro = manifest['macros']['macro.dbt.column_list']
+        self.assertIn('macro.dbt.get_quoted_csv', manifest['macros'])
+        macro = manifest['macros']['macro.dbt.get_quoted_csv']
         self.assertEqual(
             set(macro),
             {
@@ -645,7 +645,7 @@ def _verify_generic_macro_structure(self, manifest):
             if k not in {'macro_sql'}
         }
         # Windows means we can't hard-code these.
-        helpers_path = Normalized('macros/materializations/helpers.sql')
+        helpers_path = Normalized('macros/materializations/models/incremental/column_helpers.sql')
         root_path = Normalized(os.path.join(
             self.dbt_core_install_root, 'include', 'global_project'
         ))
@@ -655,8 +655,8 @@ def _verify_generic_macro_structure(self, manifest):
                 'original_file_path': helpers_path,
                 'package_name': 'dbt',
                 'root_path': root_path,
-                'name': 'column_list',
-                'unique_id': 'macro.dbt.column_list',
+                'name': 'get_quoted_csv',
+                'unique_id': 'macro.dbt.get_quoted_csv',
                 'created_at': ANY,
                 'tags': [],
                 'resource_type': 'macro',
@@ -970,6 +970,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
                     },
                     'deferred': False,
                     'description': '',
+                    'file_key_name': 'models.model',
                     'fqn': ['test', 'not_null_model_id'],
                     'name': 'not_null_model_id',
                     'original_file_path': model_schema_yml_path,
@@ -1061,6 +1062,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
                     },
                     'deferred': False,
                     'description': '',
+                    'file_key_name': 'models.model',
                     'fqn': ['test', 'test_nothing_model_'],
                     'name': 'test_nothing_model_',
                     'original_file_path': model_schema_yml_path,
@@ -1107,6 +1109,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
                     },
                     'deferred': False,
                     'description': '',
+                    'file_key_name': 'models.model',
                     'fqn': ['test', 'unique_model_id'],
                     'name': 'unique_model_id',
                     'original_file_path': model_schema_yml_path,
diff --git a/tests/unit/utils.py b/tests/unit/utils.py
index 051f4a4e3..68d0fc33f 100644
--- a/tests/unit/utils.py
+++ b/tests/unit/utils.py
@@ -1,5 +1,4 @@
 """Unit test utility functions.
-
 Note that all imports should be inside the functions to avoid import/mocking
 issues.
 """
@@ -11,11 +10,11 @@
 import agate
 import pytest
 from dbt.dataclass_schema import ValidationError
+from dbt.config.project import PartialProject
 
 
 def normalize(path):
     """On windows, neither is enough on its own:
-
     >>> normcase('C:\\documents/ALL CAPS/subdir\\..')
     'c:\\documents\\all caps\\subdir\\..'
     >>> normpath('C:\\documents/ALL CAPS/subdir\\..')
@@ -46,7 +45,7 @@ def profile_from_dict(profile, profile_name, cli_vars='{}'):
     if not isinstance(cli_vars, dict):
         cli_vars = parse_cli_vars(cli_vars)
 
-    renderer = ProfileRenderer(generate_base_context(cli_vars))
+    renderer = ProfileRenderer(cli_vars)
     return Profile.from_raw_profile_info(
         profile,
         profile_name,
@@ -62,13 +61,18 @@ def project_from_dict(project, profile, packages=None, selectors=None, cli_vars=
     if not isinstance(cli_vars, dict):
         cli_vars = parse_cli_vars(cli_vars)
 
-    renderer = DbtProjectYamlRenderer(generate_target_context(profile, cli_vars))
+    renderer = DbtProjectYamlRenderer(profile, cli_vars)
 
     project_root = project.pop('project-root', os.getcwd())
 
-    return Project.render_from_dict(
-            project_root, project, packages, selectors, renderer
-        )
+    partial = PartialProject.from_dicts(
+        project_root=project_root,
+        project_dict=project,
+        packages_dict=packages,
+        selectors_dict=selectors,
+    )
+    return partial.render(renderer)
+
 
 
 def config_from_parts_or_dicts(project, profile, packages=None, selectors=None, cli_vars='{}'):
@@ -368,4 +372,3 @@ def replace_config(n, **kwargs):
         config=n.config.replace(**kwargs),
         unrendered_config=dict_replace(n.unrendered_config, **kwargs),
     )
-

From bc6113cc8efbc6a6f4618b5839e5a8873a48d2d2 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Tue, 9 Nov 2021 02:12:30 +0100
Subject: [PATCH 407/860] Rm py36 support + fix failing test (#59)

* Rm py36 tests, pkg metadata, bump reqs

* Add metrics to docs_generate test
---
 .github/scripts/integration-test-matrix.js                 | 2 +-
 .github/workflows/main.yml                                 | 4 ++--
 CHANGELOG.md                                               | 1 +
 setup.py                                                   | 7 +++----
 .../integration/docs_generate_tests/test_docs_generate.py  | 3 +++
 tox.ini                                                    | 6 +++---
 6 files changed, 13 insertions(+), 10 deletions(-)

diff --git a/.github/scripts/integration-test-matrix.js b/.github/scripts/integration-test-matrix.js
index ae1768bf7..6a33653a2 100644
--- a/.github/scripts/integration-test-matrix.js
+++ b/.github/scripts/integration-test-matrix.js
@@ -1,6 +1,6 @@
 module.exports = ({ context }) => {
   const defaultPythonVersion = "3.8";
-  const supportedPythonVersions = ["3.6", "3.7", "3.8", "3.9"];
+  const supportedPythonVersions = ["3.7", "3.8", "3.9"];
   const supportedAdapters = ["bigquery"];
 
   // if PR, generate matrix based on files changed and PR labels
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index bb2314d81..23cdb4502 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -77,7 +77,7 @@ jobs:
     strategy:
       fail-fast: false
       matrix:
-        python-version: [3.6, 3.7, 3.8] # TODO: support unit testing for python 3.9 (https://github.com/dbt-labs/dbt/issues/3689)
+        python-version: [3.7, 3.8] # TODO: support unit testing for python 3.9 (https://github.com/dbt-labs/dbt/issues/3689)
 
     env:
       TOXENV: "unit"
@@ -171,7 +171,7 @@ jobs:
       fail-fast: false
       matrix:
         os: [ubuntu-latest, macos-latest, windows-latest]
-        python-version: [3.6, 3.7, 3.8, 3.9]
+        python-version: [3.7, 3.8, 3.9]
 
     steps:
       - name: Set up Python ${{ matrix.python-version }}
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 99c567717..b9d087bde 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -11,6 +11,7 @@ N/A
 ### Under the hood
 - Capping `google-api-core` to version `1.31.3` due to `protobuf` dependency conflict ([#53](https://github.com/dbt-labs/dbt-bigquery/pull/53))
 - Bump `google-cloud-core` and `google-api-core` upper bounds to `<3`, thereby removing `<1.31.3` limit on the latter. Remove explicit dependency on `six` ([#57](https://github.com/dbt-labs/dbt-bigquery/pull/57))
+- Remove official support for python 3.6, which is reaching end of life on December 23, 2021 ([dbt-core#4134](https://github.com/dbt-labs/dbt-core/issues/4134), [#59](https://github.com/dbt-labs/dbt-bigquery/pull/59))
 
 ### Contributors
 - [@imartynetz](https://github.com/imartynetz) ([#48](https://github.com/dbt-labs/dbt-bigquery/pull/48))
diff --git a/setup.py b/setup.py
index f1bc12ddc..191c69a44 100644
--- a/setup.py
+++ b/setup.py
@@ -4,9 +4,9 @@
 import re
 
 # require python 3.6 or newer
-if sys.version_info < (3, 6):
+if sys.version_info < (3, 7):
     print('Error: dbt does not support this version of Python.')
-    print('Please upgrade to Python 3.6 or higher.')
+    print('Please upgrade to Python 3.7 or higher.')
     sys.exit(1)
 
 
@@ -90,10 +90,9 @@ def _get_dbt_core_version():
         'Operating System :: MacOS :: MacOS X',
         'Operating System :: POSIX :: Linux',
 
-        'Programming Language :: Python :: 3.6',
         'Programming Language :: Python :: 3.7',
         'Programming Language :: Python :: 3.8',
         'Programming Language :: Python :: 3.9',
     ],
-    python_requires=">=3.6.2",
+    python_requires=">=3.7",
 )
diff --git a/tests/integration/docs_generate_tests/test_docs_generate.py b/tests/integration/docs_generate_tests/test_docs_generate.py
index bb8df64bb..8621d1e15 100644
--- a/tests/integration/docs_generate_tests/test_docs_generate.py
+++ b/tests/integration/docs_generate_tests/test_docs_generate.py
@@ -1252,6 +1252,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
                     'tags': []
                 }
             },
+            'metrics': {},
             'selectors': {},
             'parent_map': {
                 'model.test.model': ['seed.test.seed'],
@@ -1728,6 +1729,7 @@ def expected_bigquery_complex_manifest(self):
             },
             'sources': {},
             'exposures': {},
+            'metrics': {},
             'selectors': {},
             'child_map': {
                 'model.test.clustered': [],
@@ -1812,6 +1814,7 @@ def verify_manifest(self, expected_manifest):
         manifest_keys = frozenset({
             'nodes', 'sources', 'macros', 'parent_map', 'child_map',
             'docs', 'metadata', 'docs', 'disabled', 'exposures', 'selectors',
+            'metrics',
         })
 
         self.assertEqual(frozenset(manifest), manifest_keys)
diff --git a/tox.ini b/tox.ini
index 53971de00..f938e6d4f 100644
--- a/tox.ini
+++ b/tox.ini
@@ -1,6 +1,6 @@
 [tox]
 skipsdist = True
-envlist = py36,py37,py38,py39,flake8
+envlist = py37,py38,py39,flake8
 
 [testenv:flake8]
 description = flake8 code checks
@@ -11,7 +11,7 @@ commands = flake8 --select=E,W,F --ignore=W504,E741 --max-line-length 99 \
 deps =
   -rdev_requirements.txt
 
-[testenv:{unit,py36,py37,py38,py39,py}]
+[testenv:{unit,py37,py38,py39,py}]
 description = unit testing
 skip_install = true
 passenv = DBT_* PYTEST_ADDOPTS
@@ -20,7 +20,7 @@ deps =
   -rdev_requirements.txt
   -e.
 
-[testenv:{integration,py36,py37,py38,py39,py}-{bigquery}]
+[testenv:{integration,py37,py38,py39,py}-{bigquery}]
 description = adapter plugin integration testing
 skip_install = true
 passenv = DBT_* BIGQUERY_TEST_* PYTEST_ADDOPTS

From af65d91f2ef807b8559421b03be3f20ebddc21a7 Mon Sep 17 00:00:00 2001
From: Emily Rockman 
Date: Wed, 10 Nov 2021 07:00:52 -0600
Subject: [PATCH 408/860] converted to use new struct logging (#55)

* converted to use new struct logging

* Add changelog entry

* Update integration tests

* Fix connections.py call sites

Co-authored-by: Jeremy Cohen 
---
 CHANGELOG.md                                  |  1 +
 dbt/adapters/bigquery/connections.py          | 14 +++++++----
 dbt/adapters/bigquery/gcloud.py               |  4 +++-
 dbt/adapters/bigquery/impl.py                 |  9 +++----
 tests/integration/base.py                     | 16 +++++++------
 .../test_query_comments.py                    | 24 ++++++++++---------
 6 files changed, 40 insertions(+), 28 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index b9d087bde..530ef5a65 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -12,6 +12,7 @@ N/A
 - Capping `google-api-core` to version `1.31.3` due to `protobuf` dependency conflict ([#53](https://github.com/dbt-labs/dbt-bigquery/pull/53))
 - Bump `google-cloud-core` and `google-api-core` upper bounds to `<3`, thereby removing `<1.31.3` limit on the latter. Remove explicit dependency on `six` ([#57](https://github.com/dbt-labs/dbt-bigquery/pull/57))
 - Remove official support for python 3.6, which is reaching end of life on December 23, 2021 ([dbt-core#4134](https://github.com/dbt-labs/dbt-core/issues/4134), [#59](https://github.com/dbt-labs/dbt-bigquery/pull/59))
+- Add support for structured logging [#55](https://github.com/dbt-labs/dbt-bigquery/pull/55)
 
 ### Contributors
 - [@imartynetz](https://github.com/imartynetz) ([#48](https://github.com/dbt-labs/dbt-bigquery/pull/48))
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 4a69b35b2..984da6888 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -27,11 +27,14 @@
     FailedToConnectException, RuntimeException, DatabaseException, DbtProfileError
 )
 from dbt.adapters.base import BaseConnectionManager, Credentials
-from dbt.logger import GLOBAL_LOGGER as logger
+from dbt.events import AdapterLogger
+from dbt.events.functions import fire_event
+from dbt.events.types import SQLQuery
 from dbt.version import __version__ as dbt_version
 
 from dbt.dataclass_schema import StrEnum
 
+logger = AdapterLogger("BigQuery")
 
 BQ_QUERY_JOB_SPLIT = '-----Query Job SQL Follows-----'
 
@@ -344,7 +347,7 @@ def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False):
         conn = self.get_thread_connection()
         client = conn.handle
 
-        logger.debug('On {}: {}', conn.name, sql)
+        fire_event(SQLQuery(conn_name=conn.name, sql=sql))
 
         if self.profile.query_comment and self.profile.query_comment.job_label:
             query_comment = self.query_header.comment.query_comment
@@ -534,7 +537,7 @@ def _retry_and_handle(self, msg, conn, fn):
         """retry a function call within the context of exception_handler."""
         def reopen_conn_on_error(error):
             if isinstance(error, REOPENABLE_ERRORS):
-                logger.warning('Reopening connection after {!r}', error)
+                logger.warning('Reopening connection after {!r}'.format(error))
                 self.close(conn)
                 self.open(conn)
                 return
@@ -577,8 +580,9 @@ def count_error(self, error):
         self.error_count += 1
         if _is_retryable(error) and self.error_count <= self.retries:
             logger.debug(
-                'Retry attempt {} of {} after error: {}',
-                self.error_count, self.retries, repr(error))
+                'Retry attempt {} of {} after error: {}'.format(
+                    self.error_count, self.retries, repr(error)
+                ))
             return True
         else:
             return False
diff --git a/dbt/adapters/bigquery/gcloud.py b/dbt/adapters/bigquery/gcloud.py
index 77ed74fdc..28e7e1a74 100644
--- a/dbt/adapters/bigquery/gcloud.py
+++ b/dbt/adapters/bigquery/gcloud.py
@@ -1,4 +1,4 @@
-from dbt.logger import GLOBAL_LOGGER as logger
+from dbt.events import AdapterLogger
 import dbt.exceptions
 from dbt.clients.system import run_cmd
 
@@ -9,6 +9,8 @@
 https://cloud.google.com/sdk/
 """
 
+logger = AdapterLogger("BigQuery")
+
 
 def gcloud_installed():
     try:
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index f828c37f8..8e3384dd1 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -14,7 +14,7 @@
 from dbt.adapters.bigquery import BigQueryColumn
 from dbt.adapters.bigquery import BigQueryConnectionManager
 from dbt.contracts.graph.manifest import Manifest
-from dbt.logger import GLOBAL_LOGGER as logger, print_timestamped_line
+from dbt.events import AdapterLogger
 from dbt.utils import filter_null_values
 
 import google.auth
@@ -29,6 +29,8 @@
 import agate
 import json
 
+logger = AdapterLogger("BigQuery")
+
 # Write dispositions for bigquery.
 WRITE_APPEND = google.cloud.bigquery.job.WriteDisposition.WRITE_APPEND
 WRITE_TRUNCATE = google.cloud.bigquery.job.WriteDisposition.WRITE_TRUNCATE
@@ -481,9 +483,8 @@ def _bq_table_to_relation(self, bq_table):
     @classmethod
     def warning_on_hooks(hook_type):
         msg = "{} is not supported in bigquery and will be ignored"
-        print_timestamped_line(
-            msg.format(hook_type), ui.COLOR_FG_YELLOW
-        )
+        warn_msg = dbt.ui.color(msg, ui.COLOR_FG_YELLOW)
+        logger.info(warn_msg)
 
     @available
     def add_query(self, sql, auto_begin=True, bindings=None,
diff --git a/tests/integration/base.py b/tests/integration/base.py
index 6e31d2d71..243ac92d8 100644
--- a/tests/integration/base.py
+++ b/tests/integration/base.py
@@ -22,10 +22,15 @@
 from dbt.clients.jinja import template_cache
 from dbt.config import RuntimeConfig
 from dbt.context import providers
-from dbt.logger import GLOBAL_LOGGER as logger, log_manager
+from dbt.logger import log_manager
+from dbt.events.functions import (
+    capture_stdout_logs, fire_event, setup_event_logger, stop_capture_stdout_logs
+)
+from dbt.events import AdapterLogger
 from dbt.contracts.graph.manifest import Manifest
 
 
+logger = AdapterLogger("Bigquery")
 INITIAL_ROOT = os.getcwd()
 
 
@@ -231,6 +236,7 @@ def setUp(self):
         os.chdir(self.initial_dir)
         # before we go anywhere, collect the initial path info
         self._logs_dir = os.path.join(self.initial_dir, 'logs', self.prefix)
+        setup_event_logger(self._logs_dir)
         _really_makedirs(self._logs_dir)
         self.test_original_source_path = _pytest_get_test_root()
         self.test_root_dir = self._generate_test_root_dir()
@@ -403,16 +409,12 @@ def run_dbt(self, args=None, expect_pass=True, profiles_dir=True):
 
     def run_dbt_and_capture(self, *args, **kwargs):
         try:
-            initial_stdout = log_manager.stdout
-            initial_stderr = log_manager.stderr
-            stringbuf = io.StringIO()
-            log_manager.set_output_stream(stringbuf)
-
+            stringbuf = capture_stdout_logs()
             res = self.run_dbt(*args, **kwargs)
             stdout = stringbuf.getvalue()
 
         finally:
-            log_manager.set_output_stream(initial_stdout, initial_stderr)
+            stop_capture_stdout_logs()
 
         return res, stdout
 
diff --git a/tests/integration/query_comments_test/test_query_comments.py b/tests/integration/query_comments_test/test_query_comments.py
index 11a816a21..bc813a6ae 100644
--- a/tests/integration/query_comments_test/test_query_comments.py
+++ b/tests/integration/query_comments_test/test_query_comments.py
@@ -2,6 +2,7 @@
 import io
 import json
 import os
+import re
 
 import dbt.exceptions
 from dbt.version import __version__ as dbt_version
@@ -52,28 +53,29 @@ def tearDown(self):
         super().tearDown()
 
     def run_get_json(self, expect_pass=True):
-        self.run_dbt(
+        res, raw_logs = self.run_dbt_and_capture(
             ['--debug', '--log-format=json', 'run'],
             expect_pass=expect_pass
         )
-        logs = []
-        for line in self.stringbuf.getvalue().split('\n'):
+        parsed_logs = []
+        for line in raw_logs.split('\n'):
             try:
                 log = json.loads(line)
             except ValueError:
                 continue
 
-            if log['extra'].get('run_state') != 'running':
-                continue
-            logs.append(log)
-        self.assertGreater(len(logs), 0)
-        return logs
+            parsed_logs.append(log)
+
+        # empty lists evaluate as False
+        self.assertTrue(parsed_logs)
+        return parsed_logs
 
     def query_comment(self, model_name, log):
+        log_msg = re.sub("(?:[01]\d|2[0123]):(?:[012345]\d):(?:[012345]\d \| )", "", log['msg'])
         prefix = 'On {}: '.format(model_name)
 
-        if log['message'].startswith(prefix):
-            msg = log['message'][len(prefix):]
+        if log_msg.startswith(prefix):
+            msg = log_msg[len(prefix):]
             if msg in {'COMMIT', 'BEGIN', 'ROLLBACK'}:
                 return None
             return msg
@@ -88,7 +90,7 @@ def run_assert_comments(self):
             if msg is not None and self.matches_comment(msg):
                 seen = True
 
-        self.assertTrue(seen, 'Never saw a matching log message! Logs:\n{}'.format('\n'.join(l['message'] for l in logs)))
+        self.assertTrue(seen, 'Never saw a matching log message! Logs:\n{}'.format('\n'.join(l['msg'] for l in logs)))
 
     @use_profile('bigquery')
     def test_bigquery_comments(self):

From 95ca84b5100c2b940751d8a65aaeb914df211ac7 Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Wed, 10 Nov 2021 17:04:38 +0100
Subject: [PATCH 409/860] Bumping version to 1.0.0rc1 (#62)

* Bumping version to 1.0.0rc1

* Update changelog

Co-authored-by: Github Build Bot 
Co-authored-by: Jeremy Cohen 
---
 .bumpversion.cfg                     | 2 +-
 CHANGELOG.md                         | 3 +--
 dbt/adapters/bigquery/__version__.py | 2 +-
 3 files changed, 3 insertions(+), 4 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index bd1104ce6..1906822cf 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.0.0b2
+current_version = 1.0.0rc1
 parse = (?P\d+)
 	\.(?P\d+)
 	\.(?P\d+)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 530ef5a65..8893752ad 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,7 +1,6 @@
 ## dbt-bigquery 1.0.0 (Release TBD)
 
-### Features
-N/A
+## dbt-bigquery 1.0.0rc1 (November 10, 2021)
 
 ### Fixes
 - Fix problem with bytes processed return None value when the service account used to connect DBT in bigquery had a row policy access.
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 50ece64e0..2c69af52f 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = '1.0.0b2'
+version = '1.0.0rc1'

From 6605218c5abea33ab6b3f295c7a8c40dace136e8 Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Tue, 16 Nov 2021 14:39:13 -0500
Subject: [PATCH 410/860] Slack message for failed nightly runs (#69)

---
 .github/workflows/integration.yml | 16 ++++++++++++++++
 1 file changed, 16 insertions(+)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index b09fcb225..85f8bef07 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -208,3 +208,19 @@ jobs:
             "You do not have permissions to run integration tests, @dbt-labs/core "\
             "needs to label this PR with `ok to test` in order to run integration tests!"
           check_for_duplicate_msg: true
+
+  slack-results:
+    runs-on: ubuntu-latest
+    needs: test 
+    if: always()
+
+    steps:
+      - name: Posting scheduled run failures
+        uses: ravsamhq/notify-slack-action@v1
+        if: ${{ github.event_name == 'schedule' }}
+        with:
+          notification_title: 'BigQuery nightly integration test failed'
+          status: ${{ job.status }}
+          notify_when: 'failure'
+    env:
+      SLACK_WEBHOOK_URL: ${{ secrets.SLACK_DEV_CORE_ALERTS }}

From fb061ae7847af35dd42c42b798803aa310bf92ea Mon Sep 17 00:00:00 2001
From: pgoslatara 
Date: Fri, 19 Nov 2021 11:59:50 +0100
Subject: [PATCH 411/860] Adding execution_project to target object (#66)

* Adding execution_project to target object

* Updating CHANGELOG.md

Co-authored-by: Jeremy Cohen 
---
 CHANGELOG.md                         | 8 ++++++++
 dbt/adapters/bigquery/connections.py | 3 ++-
 2 files changed, 10 insertions(+), 1 deletion(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 8893752ad..302e6afe5 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,13 @@
 ## dbt-bigquery 1.0.0 (Release TBD)
 
+### Fixes
+
+### Under the hood
+- Adding `execution_project` to `target` object ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))
+
+### Contributors
+- [@pgoslatara](https://github.com/pgoslatara) ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))
+
 ## dbt-bigquery 1.0.0rc1 (November 10, 2021)
 
 ### Fixes
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 984da6888..daa987763 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -127,7 +127,8 @@ def unique_field(self):
 
     def _connection_keys(self):
         return ('method', 'database', 'schema', 'location', 'priority',
-                'timeout_seconds', 'maximum_bytes_billed')
+                'timeout_seconds', 'maximum_bytes_billed',
+                'execution_project')
 
     @classmethod
     def __pre_deserialize__(cls, d: Dict[Any, Any]) -> Dict[Any, Any]:

From 7e496af5ef17246a379ab14225162819bc5d3069 Mon Sep 17 00:00:00 2001
From: Borys Borysenko 
Date: Fri, 19 Nov 2021 14:16:27 +0200
Subject: [PATCH 412/860] Making scopes a configurable profile config option
 for BigQuery (#63)

* Making scopes a configurable profile config option for BigQuery

* Updating CHANGELOG

* Rm duplication in changelog

Co-authored-by: Jeremy Cohen 
---
 CHANGELOG.md                         |  4 +++-
 dbt/adapters/bigquery/connections.py | 20 +++++++++++---------
 2 files changed, 14 insertions(+), 10 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 302e6afe5..7e4bbefcb 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,12 +1,14 @@
 ## dbt-bigquery 1.0.0 (Release TBD)
 
-### Fixes
+### Features
+- Add optional `scopes` profile configuration argument to reduce the BigQuery OAuth scopes down to the minimal set needed. ([#23](https://github.com/dbt-labs/dbt-bigquery/issues/23), [#63](https://github.com/dbt-labs/dbt-bigquery/pull/63))
 
 ### Under the hood
 - Adding `execution_project` to `target` object ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))
 
 ### Contributors
 - [@pgoslatara](https://github.com/pgoslatara) ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))
+- [@bborysenko](https://github.com/bborysenko) ([#63](https://github.com/dbt-labs/dbt-bigquery/pull/63))
 
 ## dbt-bigquery 1.0.0rc1 (November 10, 2021)
 
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index daa987763..cfd4a3a04 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -110,6 +110,12 @@ class BigQueryCredentials(Credentials):
     client_secret: Optional[str] = None
     token_uri: Optional[str] = None
 
+    scopes: Optional[Tuple[str, ...]] = (
+        'https://www.googleapis.com/auth/bigquery',
+        'https://www.googleapis.com/auth/cloud-platform',
+        'https://www.googleapis.com/auth/drive'
+    )
+
     _ALIASES = {
         'project': 'database',
         'dataset': 'schema',
@@ -149,10 +155,6 @@ def __pre_deserialize__(cls, d: Dict[Any, Any]) -> Dict[Any, Any]:
 class BigQueryConnectionManager(BaseConnectionManager):
     TYPE = 'bigquery'
 
-    SCOPE = ('https://www.googleapis.com/auth/bigquery',
-             'https://www.googleapis.com/auth/cloud-platform',
-             'https://www.googleapis.com/auth/drive')
-
     QUERY_TIMEOUT = 300
     RETRIES = 1
     DEFAULT_INITIAL_DELAY = 1.0  # Seconds
@@ -246,16 +248,16 @@ def get_bigquery_credentials(cls, profile_credentials):
         creds = GoogleServiceAccountCredentials.Credentials
 
         if method == BigQueryConnectionMethod.OAUTH:
-            credentials, _ = get_bigquery_defaults(scopes=cls.SCOPE)
+            credentials, _ = get_bigquery_defaults(scopes=profile_credentials.scopes)
             return credentials
 
         elif method == BigQueryConnectionMethod.SERVICE_ACCOUNT:
             keyfile = profile_credentials.keyfile
-            return creds.from_service_account_file(keyfile, scopes=cls.SCOPE)
+            return creds.from_service_account_file(keyfile, scopes=profile_credentials.scopes)
 
         elif method == BigQueryConnectionMethod.SERVICE_ACCOUNT_JSON:
             details = profile_credentials.keyfile_json
-            return creds.from_service_account_info(details, scopes=cls.SCOPE)
+            return creds.from_service_account_info(details, scopes=profile_credentials.scopes)
 
         elif method == BigQueryConnectionMethod.OAUTH_SECRETS:
             return GoogleCredentials.Credentials(
@@ -264,7 +266,7 @@ def get_bigquery_credentials(cls, profile_credentials):
                 client_id=profile_credentials.client_id,
                 client_secret=profile_credentials.client_secret,
                 token_uri=profile_credentials.token_uri,
-                scopes=cls.SCOPE
+                scopes=profile_credentials.scopes
             )
 
         error = ('Invalid `method` in profile: "{}"'.format(method))
@@ -276,7 +278,7 @@ def get_impersonated_bigquery_credentials(cls, profile_credentials):
         return impersonated_credentials.Credentials(
             source_credentials=source_credentials,
             target_principal=profile_credentials.impersonate_service_account,
-            target_scopes=list(cls.SCOPE),
+            target_scopes=list(profile_credentials.scopes),
             lifetime=profile_credentials.timeout_seconds,
         )
 

From 69f5cbf0f8dbf8d70bdbe1a0b9273f0f0cf286ce Mon Sep 17 00:00:00 2001
From: Yu Ishikawa 
Date: Sat, 20 Nov 2021 00:03:15 +0900
Subject: [PATCH 413/860] Don't apply `require_partition_filter` to a temporary
 table (#65)

* Don't apply `require_partition_filter` to a temporary table

* Correct the partition_expiration_days option

* Update CHANGELOG.md

* Move up changelog entry, add contributor

Co-authored-by: Jeremy Cohen 
---
 CHANGELOG.md                  |  4 ++++
 dbt/adapters/bigquery/impl.py | 24 +++++++++++++-----------
 2 files changed, 17 insertions(+), 11 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 7e4bbefcb..93fef4953 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -3,12 +3,16 @@
 ### Features
 - Add optional `scopes` profile configuration argument to reduce the BigQuery OAuth scopes down to the minimal set needed. ([#23](https://github.com/dbt-labs/dbt-bigquery/issues/23), [#63](https://github.com/dbt-labs/dbt-bigquery/pull/63))
 
+### Fixes
+- Don't apply `require_partition_filter` to temporary tables, thereby fixing `insert_overwrite` strategy when partition filter is required ([#64](https://github.com/dbt-labs/dbt-bigquery/issues/64)), ([#65](https://github.com/dbt-labs/dbt-bigquery/pull/65))
+
 ### Under the hood
 - Adding `execution_project` to `target` object ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))
 
 ### Contributors
 - [@pgoslatara](https://github.com/pgoslatara) ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))
 - [@bborysenko](https://github.com/bborysenko) ([#63](https://github.com/dbt-labs/dbt-bigquery/pull/63))
+- [@yu-iskw](https://github.com/yu-iskw) ([#65](https://github.com/dbt-labs/dbt-bigquery/pull/65))
 
 ## dbt-bigquery 1.0.0rc1 (November 10, 2021)
 
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 8e3384dd1..4007ba0e8 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -757,20 +757,22 @@ def get_table_options(
     ) -> Dict[str, Any]:
         opts = self.get_common_options(config, node, temporary)
 
-        if temporary:
-            expiration = 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)'
-            opts['expiration_timestamp'] = expiration
-
         if config.get('kms_key_name') is not None:
             opts['kms_key_name'] = "'{}'".format(config.get('kms_key_name'))
 
-        if config.get('require_partition_filter'):
-            opts['require_partition_filter'] = config.get(
-                'require_partition_filter')
-
-        if config.get('partition_expiration_days') is not None:
-            opts['partition_expiration_days'] = config.get(
-                'partition_expiration_days')
+        if temporary:
+            expiration = 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)'
+            opts['expiration_timestamp'] = expiration
+        else:
+            # It doesn't apply the `require_partition_filter` option for a temporary table
+            # so that we avoid the error by not specifying a partition with a temporary table
+            # in the incremental model.
+            if config.get('require_partition_filter') is not None:
+                opts['require_partition_filter'] = config.get(
+                    'require_partition_filter')
+            if config.get('partition_expiration_days') is not None:
+                opts['partition_expiration_days'] = config.get(
+                    'partition_expiration_days')
 
         return opts
 

From f0839b587c48f989cdf243437589d9fc98ce1f05 Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Wed, 24 Nov 2021 17:32:01 +0100
Subject: [PATCH 414/860] Bumping version to 1.0.0rc2 (#74)

* Bumping version to 1.0.0rc2

* Update changelog

Co-authored-by: Github Build Bot 
Co-authored-by: Jeremy Cohen 
---
 .bumpversion.cfg                     | 2 +-
 CHANGELOG.md                         | 2 ++
 dbt/adapters/bigquery/__version__.py | 2 +-
 3 files changed, 4 insertions(+), 2 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 1906822cf..c6ff55c6c 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.0.0rc1
+current_version = 1.0.0rc2
 parse = (?P\d+)
 	\.(?P\d+)
 	\.(?P\d+)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 93fef4953..3721a3474 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,7 @@
 ## dbt-bigquery 1.0.0 (Release TBD)
 
+## dbt-bigquery 1.0.0rc2 (November 24, 2021)
+
 ### Features
 - Add optional `scopes` profile configuration argument to reduce the BigQuery OAuth scopes down to the minimal set needed. ([#23](https://github.com/dbt-labs/dbt-bigquery/issues/23), [#63](https://github.com/dbt-labs/dbt-bigquery/pull/63))
 
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 2c69af52f..a3f4934fd 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = '1.0.0rc1'
+version = '1.0.0rc2'

From 090172bd2422c4598be479bc522dab3926b55aec Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Fri, 3 Dec 2021 14:12:32 -0500
Subject: [PATCH 415/860] Bumping version to 1.0.0 (#80)

Co-authored-by: Github Build Bot 
---
 .bumpversion.cfg                     | 2 +-
 dbt/adapters/bigquery/__version__.py | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index c6ff55c6c..70b828614 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.0.0rc2
+current_version = 1.0.0
 parse = (?P\d+)
 	\.(?P\d+)
 	\.(?P\d+)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index a3f4934fd..de43469d2 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = '1.0.0rc2'
+version = '1.0.0'

From c4d34fc530fcccc00c47f4103a700b4c8ff08d78 Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Fri, 10 Dec 2021 11:32:37 -0500
Subject: [PATCH 416/860] Fixing Action to post test failures (#84)

---
 .github/workflows/integration.yml | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 85f8bef07..2cc9cd207 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -209,10 +209,10 @@ jobs:
             "needs to label this PR with `ok to test` in order to run integration tests!"
           check_for_duplicate_msg: true
 
-  slack-results:
+  post-failure:
     runs-on: ubuntu-latest
     needs: test 
-    if: always()
+    if: ${{ failure() }}
 
     steps:
       - name: Posting scheduled run failures
@@ -221,6 +221,5 @@ jobs:
         with:
           notification_title: 'BigQuery nightly integration test failed'
           status: ${{ job.status }}
-          notify_when: 'failure'
     env:
       SLACK_WEBHOOK_URL: ${{ secrets.SLACK_DEV_CORE_ALERTS }}

From 8df44b7d2f3da69aadace58c4a875835c4acb261 Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Thu, 16 Dec 2021 09:10:20 -0500
Subject: [PATCH 417/860] Updating version bumping script (#90)

---
 .bumpversion.cfg | 2 ++
 setup.py         | 9 ++-------
 2 files changed, 4 insertions(+), 7 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 70b828614..8a8ca719c 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -22,4 +22,6 @@ values =
 [bumpversion:part:num]
 first_value = 1
 
+[bumpversion:file:setup.py]
+
 [bumpversion:file:dbt/adapters/bigquery/__version__.py]
diff --git a/setup.py b/setup.py
index 191c69a44..ded7d3bcc 100644
--- a/setup.py
+++ b/setup.py
@@ -3,7 +3,7 @@
 import sys
 import re
 
-# require python 3.6 or newer
+# require python 3.7 or newer
 if sys.version_info < (3, 7):
     print('Error: dbt does not support this version of Python.')
     print('Please upgrade to Python 3.7 or higher.')
@@ -43,11 +43,6 @@ def _get_plugin_version_dict():
         return match.groupdict()
 
 
-def _get_plugin_version():
-    parts = _get_plugin_version_dict()
-    return "{major}.{minor}.{patch}{prekind}{pre}".format(**parts)
-
-
 # require a compatible minor version (~=), prerelease if this is a prerelease
 def _get_dbt_core_version():
     parts = _get_plugin_version_dict()
@@ -57,7 +52,7 @@ def _get_dbt_core_version():
 
 
 package_name = "dbt-bigquery"
-package_version = _get_plugin_version()
+package_version = "1.0.0"
 dbt_core_version = _get_dbt_core_version()
 description = """The BigQuery adapter plugin for dbt"""
 

From 05a436400304c21d5e3fcdf696fe68487dfb7ee3 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Tue, 21 Dec 2021 16:16:12 -0600
Subject: [PATCH 418/860] updated conversion tests to include child types (#93)

* updated conversion tests to include child types

* added changelog.md

* Update CHANGELOG.md

added reference to issue

Co-authored-by: Kyle Wigley 

Co-authored-by: Kyle Wigley 
---
 CHANGELOG.md        | 1 +
 tests/unit/utils.py | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 3721a3474..8ff5c6aef 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -10,6 +10,7 @@
 
 ### Under the hood
 - Adding `execution_project` to `target` object ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))
+- Fix test related to preventing coercion of boolean values (True, False) to numeric values (0, 1) in query results ([#93](https://github.com/dbt-labs/dbt-bigquery/issues/93))
 
 ### Contributors
 - [@pgoslatara](https://github.com/pgoslatara) ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))
diff --git a/tests/unit/utils.py b/tests/unit/utils.py
index 68d0fc33f..a4a382c5d 100644
--- a/tests/unit/utils.py
+++ b/tests/unit/utils.py
@@ -253,7 +253,7 @@ def _get_tester_for(self, column_type):
             return agate.TimeDelta()
 
         for instance in agate_helper.DEFAULT_TYPE_TESTER._possible_types:
-            if type(instance) is column_type:
+            if isinstance(instance, column_type): # include child types 
                 return instance
 
         raise ValueError(f'no tester for {column_type}')

From 9fbb35dd16ac05a63749fe240bd8726e8141dbd4 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Mon, 3 Jan 2022 11:03:45 -0600
Subject: [PATCH 419/860] =?UTF-8?q?updating=20changlog,=20adding=20tbd=20f?=
 =?UTF-8?q?or=201.1.0,=20updating=201.0.0=20release=20date=20=E2=80=A6=20(?=
 =?UTF-8?q?#95)?=
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

* updating changlog, adding tbd for 1.1.0, updating 1.0.0 release date moving #93

* updating changelog to be Fixes section instead of Under the hood
---
 CHANGELOG.md | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 8ff5c6aef..b5b84199c 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,9 @@
-## dbt-bigquery 1.0.0 (Release TBD)
+## dbt-bigquery 1.1.0 (Release TBD)
+
+### Fixes
+- Fix test related to preventing coercion of boolean values (True, False) to numeric values (0, 1) in query results ([#93](https://github.com/dbt-labs/dbt-bigquery/issues/93))
+
+## dbt-bigquery 1.0.0 (December 3, 2021)
 
 ## dbt-bigquery 1.0.0rc2 (November 24, 2021)
 
@@ -10,7 +15,6 @@
 
 ### Under the hood
 - Adding `execution_project` to `target` object ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))
-- Fix test related to preventing coercion of boolean values (True, False) to numeric values (0, 1) in query results ([#93](https://github.com/dbt-labs/dbt-bigquery/issues/93))
 
 ### Contributors
 - [@pgoslatara](https://github.com/pgoslatara) ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))

From d20a31d03a4da36fabea139516f0e906531b47c6 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 6 Jan 2022 11:15:00 -0500
Subject: [PATCH 420/860] Bump freezegun from 0.3.12 to 1.1.0 (#37)

Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
---
 dev_requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev_requirements.txt b/dev_requirements.txt
index e0641f9c9..0b3983354 100644
--- a/dev_requirements.txt
+++ b/dev_requirements.txt
@@ -5,7 +5,7 @@ git+https://github.com/dbt-labs/dbt.git#egg=dbt-core&subdirectory=core
 bumpversion
 flake8
 flaky
-freezegun==0.3.12
+freezegun==1.1.0
 ipdb
 mypy==0.782
 pip-tools

From cb96c3469a0ca7cf6ec018ebd2cce3c63c7e3972 Mon Sep 17 00:00:00 2001
From: Robert Hill <30398720+rjh336@users.noreply.github.com>
Date: Fri, 7 Jan 2022 15:40:48 -0500
Subject: [PATCH 421/860] enrich adapter response for UPDATE and SELECT queries
 (#79)

* enirch adapter response for UPDATE and SELECT queries

* code style change

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 dbt/adapters/bigquery/connections.py | 14 +++++++++++++-
 1 file changed, 13 insertions(+), 1 deletion(-)

diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index cfd4a3a04..6077af402 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -417,7 +417,7 @@ def execute(
             bytes_processed = query_job.total_bytes_processed
             message = f'{code} ({self.format_bytes(bytes_processed)} processed)'
 
-        elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE']:
+        elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE', 'UPDATE']:
             code = query_job.statement_type
             num_rows = query_job.num_dml_affected_rows
             num_rows_formated = self.format_rows_number(num_rows)
@@ -425,6 +425,18 @@ def execute(
             processed_bytes = self.format_bytes(bytes_processed)
             message = f'{code} ({num_rows_formated} rows, {processed_bytes} processed)'
 
+        elif query_job.statement_type == 'SELECT':
+            conn = self.get_thread_connection()
+            client = conn.handle
+            # use anonymous table for num_rows
+            query_table = client.get_table(query_job.destination)
+            code = 'SELECT'
+            num_rows = query_table.num_rows
+            num_rows_formated = self.format_rows_number(num_rows)
+            bytes_processed = query_job.total_bytes_processed
+            processed_bytes = self.format_bytes(bytes_processed)
+            message = f'{code} ({num_rows_formated} rows, {processed_bytes} processed)'
+
         response = BigQueryAdapterResponse(
             _message=message,
             rows_affected=num_rows,

From 141b86749df813cf3a3a90a90e7a7dfc401ba9b0 Mon Sep 17 00:00:00 2001
From: Hui Zheng <2249215+hui-zheng@users.noreply.github.com>
Date: Tue, 11 Jan 2022 22:06:05 -0800
Subject: [PATCH 422/860] add new bq job timeout and retry config (#50)

* add new bq job timeout and retry config

* fix

* fix

* update changelog

* fix

* Fix merge

* fix test_query_and_results

* 2nd attempt to fix test_query_and_results

* fixed other test cases

* fix aliases

* update changelog

* update changelog

* fix linting

Co-authored-by: Jeremy Cohen 
---
 CHANGELOG.md                              |  6 ++
 dbt/adapters/bigquery/connections.py      | 67 ++++++++++++++++-------
 dbt/adapters/bigquery/impl.py             |  2 +-
 dbt/include/bigquery/profile_template.yml |  4 +-
 tests/unit/test_bigquery_adapter.py       | 16 +++---
 5 files changed, 66 insertions(+), 29 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index b5b84199c..02857f198 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,8 +1,13 @@
 ## dbt-bigquery 1.1.0 (Release TBD)
+### Features
+- Provide a fine-grained control of the timeout and retry of BigQuery query with four new dbt profile configs: `job_creation_timeout_seconds`, `job_execution_timeout_seconds`, `job_retry_deadline_seconds`, and `job_retries` ([#45](https://github.com/dbt-labs/dbt-bigquery/issues/45), [#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
 
 ### Fixes
 - Fix test related to preventing coercion of boolean values (True, False) to numeric values (0, 1) in query results ([#93](https://github.com/dbt-labs/dbt-bigquery/issues/93))
 
+### Contributors
+- [@hui-zheng](https://github.com/hui-zheng)([#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
+
 ## dbt-bigquery 1.0.0 (December 3, 2021)
 
 ## dbt-bigquery 1.0.0rc2 (November 24, 2021)
@@ -19,6 +24,7 @@
 ### Contributors
 - [@pgoslatara](https://github.com/pgoslatara) ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))
 - [@bborysenko](https://github.com/bborysenko) ([#63](https://github.com/dbt-labs/dbt-bigquery/pull/63))
+- [@hui-zheng](https://github.com/hui-zheng)([#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
 - [@yu-iskw](https://github.com/yu-iskw) ([#65](https://github.com/dbt-labs/dbt-bigquery/pull/65))
 
 ## dbt-bigquery 1.0.0rc1 (November 10, 2021)
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 6077af402..d235ea4cf 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -92,13 +92,16 @@ class BigQueryCredentials(Credentials):
     # environment for the project
     database: Optional[str]
     execution_project: Optional[str] = None
-    timeout_seconds: Optional[int] = 300
     location: Optional[str] = None
     priority: Optional[Priority] = None
-    retries: Optional[int] = 1
     maximum_bytes_billed: Optional[int] = None
     impersonate_service_account: Optional[str] = None
 
+    job_retry_deadline_seconds: Optional[int] = None
+    job_retries: Optional[int] = 1
+    job_creation_timeout_seconds: Optional[int] = None
+    job_execution_timeout_seconds: Optional[int] = 300
+
     # Keyfile json creds
     keyfile: Optional[str] = None
     keyfile_json: Optional[Dict[str, Any]] = None
@@ -117,10 +120,13 @@ class BigQueryCredentials(Credentials):
     )
 
     _ALIASES = {
+        # 'legacy_name': 'current_name'
         'project': 'database',
         'dataset': 'schema',
         'target_project': 'target_database',
         'target_dataset': 'target_schema',
+        'retries': 'job_retries',
+        'timeout_seconds': 'job_execution_timeout_seconds',
     }
 
     @property
@@ -133,8 +139,9 @@ def unique_field(self):
 
     def _connection_keys(self):
         return ('method', 'database', 'schema', 'location', 'priority',
-                'timeout_seconds', 'maximum_bytes_billed',
-                'execution_project')
+                'timeout_seconds', 'maximum_bytes_billed', 'execution_project',
+                'job_retry_deadline_seconds', 'job_retries',
+                'job_creation_timeout_seconds', 'job_execution_timeout_seconds')
 
     @classmethod
     def __pre_deserialize__(cls, d: Dict[Any, Any]) -> Dict[Any, Any]:
@@ -279,7 +286,7 @@ def get_impersonated_bigquery_credentials(cls, profile_credentials):
             source_credentials=source_credentials,
             target_principal=profile_credentials.impersonate_service_account,
             target_scopes=list(profile_credentials.scopes),
-            lifetime=profile_credentials.timeout_seconds,
+            lifetime=profile_credentials.job_execution_timeout_seconds,
         )
 
     @classmethod
@@ -329,17 +336,24 @@ def open(cls, connection):
         return connection
 
     @classmethod
-    def get_timeout(cls, conn):
+    def get_job_execution_timeout_seconds(cls, conn):
         credentials = conn.credentials
-        return credentials.timeout_seconds
+        return credentials.job_execution_timeout_seconds
 
     @classmethod
-    def get_retries(cls, conn) -> int:
+    def get_job_retries(cls, conn) -> int:
         credentials = conn.credentials
-        if credentials.retries is not None:
-            return credentials.retries
-        else:
-            return 1
+        return credentials.job_retries
+
+    @classmethod
+    def get_job_creation_timeout_seconds(cls, conn):
+        credentials = conn.credentials
+        return credentials.job_creation_timeout_seconds
+
+    @classmethod
+    def get_job_retry_deadline_seconds(cls, conn):
+        credentials = conn.credentials
+        return credentials.job_retry_deadline_seconds
 
     @classmethod
     def get_table_from_response(cls, resp):
@@ -374,8 +388,15 @@ def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False):
         if maximum_bytes_billed is not None and maximum_bytes_billed != 0:
             job_params['maximum_bytes_billed'] = maximum_bytes_billed
 
+        job_creation_timeout = self.get_job_creation_timeout_seconds(conn)
+        job_execution_timeout = self.get_job_execution_timeout_seconds(conn)
+
         def fn():
-            return self._query_and_results(client, sql, conn, job_params)
+            return self._query_and_results(
+                client, sql, job_params,
+                job_creation_timeout=job_creation_timeout,
+                job_execution_timeout=job_execution_timeout
+            )
 
         query_job, iterator = self._retry_and_handle(msg=sql, conn=conn, fn=fn)
 
@@ -490,7 +511,7 @@ def copy_and_results():
                 write_disposition=write_disposition)
             copy_job = client.copy_table(
                 source_ref_array, destination_ref, job_config=job_config)
-            iterator = copy_job.result(timeout=self.get_timeout(conn))
+            iterator = copy_job.result(timeout=self.get_job_execution_timeout_seconds(conn))
             return copy_job, iterator
 
         self._retry_and_handle(
@@ -539,12 +560,20 @@ def fn():
             return client.create_dataset(dataset, exists_ok=True)
         self._retry_and_handle(msg='create dataset', conn=conn, fn=fn)
 
-    def _query_and_results(self, client, sql, conn, job_params, timeout=None):
+    def _query_and_results(
+        self, client, sql, job_params,
+        job_creation_timeout=None,
+        job_execution_timeout=None
+    ):
         """Query the client and wait for results."""
         # Cannot reuse job_config if destination is set and ddl is used
         job_config = google.cloud.bigquery.QueryJobConfig(**job_params)
-        query_job = client.query(sql, job_config=job_config)
-        iterator = query_job.result(timeout=timeout)
+        query_job = client.query(
+            query=sql,
+            job_config=job_config,
+            timeout=job_creation_timeout
+        )
+        iterator = query_job.result(timeout=job_execution_timeout)
 
         return query_job, iterator
 
@@ -560,9 +589,9 @@ def reopen_conn_on_error(error):
         with self.exception_handler(msg):
             return retry.retry_target(
                 target=fn,
-                predicate=_ErrorCounter(self.get_retries(conn)).count_error,
+                predicate=_ErrorCounter(self.get_job_retries(conn)).count_error,
                 sleep_generator=self._retry_generator(),
-                deadline=None,
+                deadline=self.get_job_retry_deadline_seconds(conn),
                 on_error=reopen_conn_on_error)
 
     def _retry_generator(self):
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 4007ba0e8..e9988b430 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -698,7 +698,7 @@ def load_dataframe(self, database, schema, table_name, agate_table,
             job = client.load_table_from_file(f, table, rewind=True,
                                               job_config=load_config)
 
-        timeout = self.connections.get_timeout(conn)
+        timeout = self.connections.get_job_execution_timeout_seconds(conn)
         with self.connections.exception_handler("LOAD TABLE"):
             self.poll_until_job_completes(job, timeout)
 
diff --git a/dbt/include/bigquery/profile_template.yml b/dbt/include/bigquery/profile_template.yml
index 4a9cdc7c9..d6cefdfb8 100644
--- a/dbt/include/bigquery/profile_template.yml
+++ b/dbt/include/bigquery/profile_template.yml
@@ -1,7 +1,7 @@
 fixed:
   type: bigquery
   priority: interactive
-  fixed_retries: 1
+  job_retries: 1
 prompts:
   _choose_authentication_method:
     oauth:
@@ -17,7 +17,7 @@ prompts:
   threads:
     hint: '1 or more'
     type: 'int'
-  timeout_seconds:
+  job_execution_timeout_seconds:
     default: 300
     type: 'int'
   _choose_location:
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 46f38afde..1380871c9 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -512,6 +512,7 @@ def setUp(self):
         credentials = Mock(BigQueryCredentials)
         profile = Mock(query_comment=None, credentials=credentials)
         self.connections = BigQueryConnectionManager(profile=profile)
+
         self.mock_client = Mock(
           dbt.adapters.bigquery.impl.google.cloud.bigquery.Client)
         self.mock_connection = MagicMock()
@@ -519,12 +520,14 @@ def setUp(self):
         self.mock_connection.handle = self.mock_client
 
         self.connections.get_thread_connection = lambda: self.mock_connection
+        self.connections.get_job_retry_deadline_seconds = lambda x: None
+        self.connections.get_job_retries = lambda x: 1
 
     @patch(
         'dbt.adapters.bigquery.connections._is_retryable', return_value=True)
     def test_retry_and_handle(self, is_retryable):
         self.connections.DEFAULT_MAXIMUM_DELAY = 2.0
-
+    
         @contextmanager
         def dummy_handler(msg):
             yield
@@ -587,9 +590,9 @@ def test_is_retryable(self):
         self.assertTrue(_is_retryable(rate_limit_error))
 
     def test_drop_dataset(self):
+
         mock_table = Mock()
         mock_table.reference = 'table1'
-
         self.mock_client.list_tables.return_value = [mock_table]
 
         self.connections.drop_dataset('project', 'dataset')
@@ -600,15 +603,14 @@ def test_drop_dataset(self):
 
     @patch('dbt.adapters.bigquery.impl.google.cloud.bigquery')
     def test_query_and_results(self, mock_bq):
-        self.connections.get_timeout = lambda x: 100.0
-
         self.connections._query_and_results(
-          self.mock_client, 'sql', self.mock_connection,
-          {'description': 'blah'})
+            self.mock_client, 'sql', {'job_param_1': 'blah'},
+            job_creation_timeout=15,
+            job_execution_timeout=100)
 
         mock_bq.QueryJobConfig.assert_called_once()
         self.mock_client.query.assert_called_once_with(
-          'sql', job_config=mock_bq.QueryJobConfig())
+            query='sql', job_config=mock_bq.QueryJobConfig(), timeout=15)
 
     def test_copy_bq_table_appends(self):
         self._copy_table(

From 8a109ebb54c8c6363985d1c0857d49895d52820a Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 20 Jan 2022 15:11:12 -0600
Subject: [PATCH 423/860] adding stale.yml to .github/workflows to trigger auto
 staling and messaging for issues and PRs (#103)

---
 .github/workflows/stale.yml | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
 create mode 100644 .github/workflows/stale.yml

diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml
new file mode 100644
index 000000000..2848ce8f7
--- /dev/null
+++ b/.github/workflows/stale.yml
@@ -0,0 +1,17 @@
+name: 'Close stale issues and PRs'
+on:
+  schedule:
+    - cron: "30 1 * * *"
+jobs:
+  stale:
+    runs-on: ubuntu-latest
+    steps:
+      # pinned at v4 (https://github.com/actions/stale/releases/tag/v4.0.0)
+      - uses: actions/stale@cdf15f641adb27a71842045a94023bef6945e3aa
+        with:
+          stale-issue-message: "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 remove the stale label or comment on the issue, or it will be closed in 7 days."
+          stale-pr-message: "This PR has been marked as Stale because it has been open for 180 days with no activity. If you would like the PR to remain open, please remove the stale label or comment on the PR, or it will be closed in 7 days."
+          # mark issues/PRs stale when they haven't seen activity in 180 days
+          days-before-stale: 180
+          # ignore checking issues with the following labels
+          exempt-issue-labels: "epic, discussion"
\ No newline at end of file

From 97acd8c5d56036b9d9b231bcefc28e143baa7032 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Mon, 24 Jan 2022 15:05:05 -0600
Subject: [PATCH 424/860] first draft of CONTRIBUTING.MD (#73)

* first draft of CONTRIBUTING.MD

* second draft at adapter template

* moving section in intial setup from dbt-core to adapter repos

* added test.env.example

* making suggested changes based off kyle's review

* minor updates noticed when making first drafts for other adapter repos.

* updated test => tests for naming convention

* responding to comments on github

* continued responses to comments

* updates based on review from kyle
---
 CONTRIBUTING.md  | 100 +++++++++++++++++++++++++++++++++++++++++++++++
 test.env.example |   3 ++
 2 files changed, 103 insertions(+)
 create mode 100644 CONTRIBUTING.md
 create mode 100644 test.env.example

diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md
new file mode 100644
index 000000000..504710c70
--- /dev/null
+++ b/CONTRIBUTING.md
@@ -0,0 +1,100 @@
+# Contributing to `dbt-bigquery`
+
+1. [About this document](#about-this-document)
+3. [Getting the code](#getting-the-code)
+5. [Running `dbt-bigquery` in development](#running-dbt-bigquery-in-development)
+6. [Testing](#testing)
+7. [Updating Docs](#updating-docs)
+7. [Submitting a Pull Request](#submitting-a-pull-request)
+
+## About this document
+This document is a guide intended for folks interested in contributing to `dbt-bigquery`. Below, we document the process by which members of the community should create issues and submit pull requests (PRs) in this repository. It is not intended as a guide for using `dbt-bigquery`, and it assumes a certain level of familiarity with Python concepts such as virtualenvs, `pip`, python modules, filesystems, and so on. This guide assumes you are using macOS or Linux and are comfortable with the command line.
+
+For those wishing to contribute we highly suggest reading the [dbt-core](https://github.com/dbt-labs/dbt-core/blob/main/CONTRIBUTING.md), if you haven't already. Almost all of the information there is applicable to contributing here, too!
+
+### Signing the CLA
+
+Please note that all contributors to `dbt-bigquery` must sign the [Contributor License Agreement](https://docs.getdbt.com/docs/contributor-license-agreements) to have their Pull Request merged into an `dbt-bigquery` codebase. If you are unable to sign the CLA, then the `dbt-bigquery` maintainers will unfortunately be unable to merge your Pull Request. You are, however, welcome to open issues and comment on existing ones.
+
+
+## Getting the code 
+
+You will need `git` in order to download and modify the `dbt-bigquery` source code. You can find direction [here](https://github.com/git-guides/install-git) on how to install `git`.
+
+### External contributors
+
+If you are not a member of the `dbt-labs` GitHub organization, you can contribute to `dbt-bigquery` by forking the `dbt-bigquery` repository. For a detailed overview on forking, check out the [GitHub docs on forking](https://help.github.com/en/articles/fork-a-repo). In short, you will need to:
+
+1. fork the `dbt-bigquery` repository
+2. clone your fork locally
+3. check out a new branch for your proposed changes
+4. push changes to your fork
+5. open a pull request against `dbt-labs/dbt-bigquery` from your forked repository
+
+### dbt Labs contributors
+
+If you are a member of the `dbt Labs` GitHub organization, you will have push access to the `dbt-bigquery` repo. Rather than forking `dbt-bigquery` to make your changes, just clone the repository, check out a new branch, and push directly to that branch.
+
+
+## Running `dbt-bigquery` in development
+
+### Installation
+
+First make sure that you set up your `virtualenv` as described in [Setting up an environment](https://github.com/dbt-labs/dbt-core/blob/HEAD/CONTRIBUTING.md#setting-up-an-environment).  Ensure you have the latest version of pip installed with `pip install --upgrade pip`. Next, install `dbt-bigquery` latest dependencies:
+
+```sh
+pip install -e . -r dev-requirements.txt
+```
+
+When `dbt-bigquery` is installed this way, any changes you make to the `dbt-bigquery` source code will be reflected immediately in your next `dbt-bigquery` run.
+
+To confirm you have the correct version of `dbt-core` installed please run `dbt --version` and `which dbt`.
+
+## Testing
+
+### Initial Setup
+
+`dbt-bigquery` contains [unit](https://github.com/dbt-labs/dbt-bigquery/tree/main/tests/unit) and [integration](https://github.com/dbt-labs/dbt-bigquery/tree/main/tests/integration) tests. Integration tests require testing against an actual BigQuery warehouse. We have CI set up to test against a BigQuery warehouse. In order to run integration tests locally, you will need a `test.env` file in the root of the repository that contains credentials for BigQuery.
+
+Note: This `test.env` file is git-ignored, but please be _extra_ careful to never check in credentials or other sensitive information when developing. To create your `test.env` file, copy the provided example file, then supply your relevant credentials.
+
+```
+cp test.env.example test.env
+$EDITOR test.env
+```
+
+### Test commands
+There are a few methods for running tests locally.
+
+#### `tox`
+`tox` takes care of managing Python virtualenvs and installing dependencies in order to run tests. You can also run tests in parallel, for example you can run unit tests for Python 3.7, Python 3.8, Python 3.9, and `flake8` checks in parallel with `tox -p`. Also, you can run unit tests for specific python versions with `tox -e py37`. The configuration of these tests are located in `tox.ini`.
+
+#### `pytest`
+Finally, you can also run a specific test or group of tests using `pytest` directly. With a Python virtualenv active and dev dependencies installed you can do things like:
+
+```sh
+# run specific bigquery integration tests
+python -m pytest -m profile_bigquery tests/integration/simple_copy_test
+# run all unit tests in a file
+python -m pytest tests/unit/test_bigquery_adapter.py
+# run a specific unit test
+python -m pytest tests/unit/test_bigquery_adapter.py::TestBigQueryAdapter::test_copy_table_materialization_table
+```
+## Updating Docs
+
+Many changes will require and update to the `dbt-bigquery` docs here are some useful resources.
+
+- Docs are [here](https://docs.getdbt.com/).
+- The docs repo for making changes is located [here]( https://github.com/dbt-labs/docs.getdbt.com).
+- The changes made are likely to impact one or both of [BigQuery Profile](https://docs.getdbt.com/reference/warehouse-profiles/bigquery-profile), or [BigQuery Configs](https://docs.getdbt.com/reference/resource-configs/bigquery-configs).
+- We ask every community member who makes a user-facing change to open an issue or PR regarding doc changes.
+
+
+
+## Submitting a Pull Request
+
+dbt Labs provides a CI environment to test changes to the `dbt-bigquery` adapter and periodic checks against the development version of `dbt-core` through Github Actions.  
+
+A `dbt-bigquery` maintainer will review your PR. They may suggest code revision for style or clarity, or request that you add unit or integration test(s). These are good things! We believe that, with a little bit of help, anyone can contribute high-quality code.
+
+Once all tests are passing, you have updated the changelog to reflect and tag your issue/pr for reference with a small description of the change, and your PR has been approved, a `dbt-bigquery` maintainer will merge your changes into the active development branch. And that's it! Happy developing :tada:
diff --git a/test.env.example b/test.env.example
new file mode 100644
index 000000000..a69a35fe7
--- /dev/null
+++ b/test.env.example
@@ -0,0 +1,3 @@
+BIGQUERY_TEST_DATABASE=
+BIGQUERY_TEST_ALT_DATABASE=
+BIGQUERY_TEST_SERVICE_ACCOUNT_JSON='{}'
\ No newline at end of file

From afd49d92012cd317f144179f4982bb596d01cb3d Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Mon, 24 Jan 2022 16:15:14 -0500
Subject: [PATCH 425/860] Adding Jira mirring Actions (#106)

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .github/workflows/jira-creation.yml   | 26 ++++++++++++++++++++++++++
 .github/workflows/jira-label.yml      | 27 +++++++++++++++++++++++++++
 .github/workflows/jira-transition.yml | 24 ++++++++++++++++++++++++
 3 files changed, 77 insertions(+)
 create mode 100644 .github/workflows/jira-creation.yml
 create mode 100644 .github/workflows/jira-label.yml
 create mode 100644 .github/workflows/jira-transition.yml

diff --git a/.github/workflows/jira-creation.yml b/.github/workflows/jira-creation.yml
new file mode 100644
index 000000000..c84e106a7
--- /dev/null
+++ b/.github/workflows/jira-creation.yml
@@ -0,0 +1,26 @@
+# **what?**
+# Mirrors issues into Jira. Includes the information: title,
+# GitHub Issue ID and URL
+
+# **why?**
+# Jira is our tool for tracking and we need to see these issues in there
+
+# **when?**
+# On issue creation or when an issue is labeled `Jira`
+
+name: Jira Issue Creation
+
+on:
+  issues:
+    types: [opened, labeled]
+    
+permissions:
+  issues: write
+
+jobs:
+  call-label-action:
+    uses: dbt-labs/jira-actions/.github/workflows/jira-creation.yml@main
+    secrets:
+      JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
+      JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
+      JIRA_API_TOKEN: ${{ secrets.JIRA_API_TOKEN }}
diff --git a/.github/workflows/jira-label.yml b/.github/workflows/jira-label.yml
new file mode 100644
index 000000000..fd533a170
--- /dev/null
+++ b/.github/workflows/jira-label.yml
@@ -0,0 +1,27 @@
+# **what?**
+# Calls mirroring Jira label Action. Includes adding a new label
+# to an existing issue or removing a label as well
+
+# **why?**
+# Jira is our tool for tracking and we need to see these labels in there
+
+# **when?**
+# On labels being added or removed from issues
+
+name: Jira Label Mirroring
+
+on:
+  issues:
+    types: [labeled, unlabeled]
+    
+permissions:
+  issues: read
+
+jobs:
+  call-label-action:
+    uses: dbt-labs/jira-actions/.github/workflows/jira-label.yml@main
+    secrets:
+      JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
+      JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
+      JIRA_API_TOKEN: ${{ secrets.JIRA_API_TOKEN }}
+    
diff --git a/.github/workflows/jira-transition.yml b/.github/workflows/jira-transition.yml
new file mode 100644
index 000000000..71273c7a9
--- /dev/null
+++ b/.github/workflows/jira-transition.yml
@@ -0,0 +1,24 @@
+# **what?**
+# Transition a Jira issue to a new state
+# Only supports these GitHub Issue transitions:
+#   closed, deleted, reopened
+
+# **why?**
+# Jira needs to be kept up-to-date
+
+# **when?**
+# On issue closing, deletion, reopened
+
+name: Jira Issue Transition
+
+on:
+  issues:
+    types: [closed, deleted, reopened]
+
+jobs:
+  call-label-action:
+    uses: dbt-labs/jira-actions/.github/workflows/jira-transition.yml@main
+    secrets:
+      JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
+      JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
+      JIRA_API_TOKEN: ${{ secrets.JIRA_API_TOKEN }}
\ No newline at end of file

From 0d76e04495168bc9973d5e9fdf08378485d89c42 Mon Sep 17 00:00:00 2001
From: Drew McDonald 
Date: Wed, 26 Jan 2022 15:49:17 -0500
Subject: [PATCH 426/860] Simplify usage of BigQuery's TableReference and
 DatasetReference Classes (#98)

* drop conn method argument in _query_and_results.

* update Table Ref and Dataset Ref usage.

* update changelog.

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 CHANGELOG.md                                  |  3 +
 dbt/adapters/bigquery/connections.py          | 35 +++++-----
 dbt/adapters/bigquery/impl.py                 | 66 ++++++-------------
 .../test_adapter_methods.py                   |  5 +-
 tests/unit/test_bigquery_adapter.py           | 14 ++--
 5 files changed, 49 insertions(+), 74 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 02857f198..48991628b 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,6 +5,9 @@
 ### Fixes
 - Fix test related to preventing coercion of boolean values (True, False) to numeric values (0, 1) in query results ([#93](https://github.com/dbt-labs/dbt-bigquery/issues/93))
 
+### Under the hood
+- Address BigQuery API deprecation warning and simplify usage of `TableReference` and `DatasetReference` objects ([#97](https://github.com/dbt-labs/dbt-bigquery/issues/97))
+
 ### Contributors
 - [@hui-zheng](https://github.com/hui-zheng)([#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
 
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index d235ea4cf..ad49cfd9f 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -495,11 +495,13 @@ def copy_bq_table(self, source, destination, write_disposition):
         if type(source) is not list:
             source = [source]
 
-        source_ref_array = [self.table_ref(
-            src_table.database, src_table.schema, src_table.table, conn)
-            for src_table in source]
+        source_ref_array = [
+            self.table_ref(src_table.database, src_table.schema, src_table.table)
+            for src_table in source
+        ]
         destination_ref = self.table_ref(
-            destination.database, destination.schema, destination.table, conn)
+            destination.database, destination.schema, destination.table
+        )
 
         logger.debug(
             'Copying table(s) "{}" to "{}" with disposition: "{}"',
@@ -521,32 +523,27 @@ def copy_and_results():
             conn=conn, fn=copy_and_results)
 
     @staticmethod
-    def dataset(database, schema, conn):
-        dataset_ref = conn.handle.dataset(schema, database)
-        return google.cloud.bigquery.Dataset(dataset_ref)
+    def dataset_ref(database, schema):
+        return google.cloud.bigquery.DatasetReference(project=database, dataset_id=schema)
 
     @staticmethod
-    def dataset_from_id(dataset_id):
-        return google.cloud.bigquery.Dataset.from_string(dataset_id)
-
-    def table_ref(self, database, schema, table_name, conn):
-        dataset = self.dataset(database, schema, conn)
-        return dataset.table(table_name)
+    def table_ref(database, schema, table_name):
+        dataset_ref = google.cloud.bigquery.DatasetReference(database, schema)
+        return google.cloud.bigquery.TableReference(dataset_ref, table_name)
 
     def get_bq_table(self, database, schema, identifier):
         """Get a bigquery table for a schema/model."""
         conn = self.get_thread_connection()
-        table_ref = self.table_ref(database, schema, identifier, conn)
+        table_ref = self.table_ref(database, schema, identifier)
         return conn.handle.get_table(table_ref)
 
     def drop_dataset(self, database, schema):
         conn = self.get_thread_connection()
-        dataset = self.dataset(database, schema, conn)
+        dataset_ref = self.dataset_ref(database, schema)
         client = conn.handle
 
         def fn():
-            return client.delete_dataset(
-                dataset, delete_contents=True, not_found_ok=True)
+            return client.delete_dataset(dataset_ref, delete_contents=True, not_found_ok=True)
 
         self._retry_and_handle(
             msg='drop dataset', conn=conn, fn=fn)
@@ -554,10 +551,10 @@ def fn():
     def create_dataset(self, database, schema):
         conn = self.get_thread_connection()
         client = conn.handle
-        dataset = self.dataset(database, schema, conn)
+        dataset_ref = self.dataset_ref(database, schema)
 
         def fn():
-            return client.create_dataset(dataset, exists_ok=True)
+            return client.create_dataset(dataset_ref, exists_ok=True)
         self._retry_and_handle(msg='create dataset', conn=conn, fn=fn)
 
     def _query_and_results(
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index e9988b430..73bf66e20 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -150,12 +150,9 @@ def drop_relation(self, relation: BigQueryRelation) -> None:
             self.cache_dropped(relation)
 
         conn = self.connections.get_thread_connection()
-        client = conn.handle
 
-        dataset = self.connections.dataset(relation.database, relation.schema,
-                                           conn)
-        relation_object = dataset.table(relation.identifier)
-        client.delete_table(relation_object)
+        table_ref = self.get_table_ref_from_relation(relation)
+        conn.handle.delete_table(table_ref)
 
     def truncate_relation(self, relation: BigQueryRelation) -> None:
         raise dbt.exceptions.NotImplementedException(
@@ -169,10 +166,7 @@ def rename_relation(
         conn = self.connections.get_thread_connection()
         client = conn.handle
 
-        from_table_ref = self.connections.table_ref(from_relation.database,
-                                                    from_relation.schema,
-                                                    from_relation.identifier,
-                                                    conn)
+        from_table_ref = self.get_table_ref_from_relation(from_relation)
         from_table = client.get_table(from_table_ref)
         if from_table.table_type == "VIEW" or \
                 from_relation.type == RelationType.View or \
@@ -181,10 +175,7 @@ def rename_relation(
                 'Renaming of views is not currently supported in BigQuery'
             )
 
-        to_table_ref = self.connections.table_ref(to_relation.database,
-                                                  to_relation.schema,
-                                                  to_relation.identifier,
-                                                  conn)
+        to_table_ref = self.get_table_ref_from_relation(to_relation)
 
         self.cache_renamed(from_relation, to_relation)
         client.copy_table(from_table_ref, to_table_ref)
@@ -212,15 +203,13 @@ def check_schema_exists(self, database: str, schema: str) -> bool:
         conn = self.connections.get_thread_connection()
         client = conn.handle
 
-        bigquery_dataset = self.connections.dataset(
-            database, schema, conn
-        )
+        dataset_ref = self.connections.dataset_ref(database, schema)
         # try to do things with the dataset. If it doesn't exist it will 404.
         # we have to do it this way to handle underscore-prefixed datasets,
         # which appear in neither the information_schema.schemata view nor the
         # list_datasets method.
         try:
-            next(iter(client.list_tables(bigquery_dataset, max_results=1)))
+            next(iter(client.list_tables(dataset_ref, max_results=1)))
         except StopIteration:
             pass
         except google.api_core.exceptions.NotFound:
@@ -262,12 +251,12 @@ def list_relations_without_caching(
         connection = self.connections.get_thread_connection()
         client = connection.handle
 
-        bigquery_dataset = self.connections.dataset(
-            schema_relation.database, schema_relation.schema, connection
+        dataset_ref = self.connections.dataset_ref(
+            schema_relation.database, schema_relation.schema
         )
 
         all_tables = client.list_tables(
-            bigquery_dataset,
+            dataset_ref,
             # BigQuery paginates tables by alphabetizing them, and using
             # the name of the last table on a page as the key for the
             # next page. If that key table gets dropped before we run
@@ -583,11 +572,10 @@ def parse_partition_by(
         """
         return PartitionConfig.parse(raw_partition_by)
 
-    def get_table_ref_from_relation(self, conn, relation):
-        return self.connections.table_ref(relation.database,
-                                          relation.schema,
-                                          relation.identifier,
-                                          conn)
+    def get_table_ref_from_relation(self, relation):
+        return self.connections.table_ref(
+            relation.database, relation.schema, relation.identifier
+        )
 
     def _update_column_dict(self, bq_column_dict, dbt_columns, parent=''):
         """
@@ -629,7 +617,7 @@ def update_columns(self, relation, columns):
             return
 
         conn = self.connections.get_thread_connection()
-        table_ref = self.get_table_ref_from_relation(conn, relation)
+        table_ref = self.get_table_ref_from_relation(relation)
         table = conn.handle.get_table(table_ref)
 
         new_schema = []
@@ -651,12 +639,7 @@ def update_table_description(
         conn = self.connections.get_thread_connection()
         client = conn.handle
 
-        table_ref = self.connections.table_ref(
-            database,
-            schema,
-            identifier,
-            conn
-        )
+        table_ref = self.connections.table_ref(database, schema, identifier)
         table = client.get_table(table_ref)
         table.description = description
         client.update_table(table, ['description'])
@@ -670,9 +653,7 @@ def alter_table_add_columns(self, relation, columns):
         conn = self.connections.get_thread_connection()
         client = conn.handle
 
-        table_ref = self.connections.table_ref(relation.database,
-                                               relation.schema,
-                                               relation.identifier, conn)
+        table_ref = self.get_table_ref_from_relation(relation)
         table = client.get_table(table_ref)
 
         new_columns = [col.column_to_bq_schema() for col in columns]
@@ -688,14 +669,14 @@ def load_dataframe(self, database, schema, table_name, agate_table,
         conn = self.connections.get_thread_connection()
         client = conn.handle
 
-        table = self.connections.table_ref(database, schema, table_name, conn)
+        table_ref = self.connections.table_ref(database, schema, table_name)
 
         load_config = google.cloud.bigquery.LoadJobConfig()
         load_config.skip_leading_rows = 1
         load_config.schema = bq_schema
 
         with open(agate_table.original_abspath, "rb") as f:
-            job = client.load_table_from_file(f, table, rewind=True,
+            job = client.load_table_from_file(f, table_ref, rewind=True,
                                               job_config=load_config)
 
         timeout = self.connections.get_job_execution_timeout_seconds(conn)
@@ -793,16 +774,11 @@ def grant_access_to(self, entity, entity_type, role, grant_target_dict):
 
         GrantTarget.validate(grant_target_dict)
         grant_target = GrantTarget.from_dict(grant_target_dict)
-        dataset = client.get_dataset(
-            self.connections.dataset_from_id(grant_target.render())
-        )
+        dataset_ref = self.connections.dataset_ref(grant_target.project, grant_target.dataset)
+        dataset = client.get_dataset(dataset_ref)
 
         if entity_type == 'view':
-            entity = self.connections.table_ref(
-                entity.database,
-                entity.schema,
-                entity.identifier,
-                conn).to_api_repr()
+            entity = self.get_table_ref_from_relation(entity).to_api_repr()
 
         access_entry = AccessEntry(role, entity_type, entity)
         access_entries = dataset.access_entries
diff --git a/tests/integration/adapter_methods_test/test_adapter_methods.py b/tests/integration/adapter_methods_test/test_adapter_methods.py
index fd6595fcf..cc5137dea 100644
--- a/tests/integration/adapter_methods_test/test_adapter_methods.py
+++ b/tests/integration/adapter_methods_test/test_adapter_methods.py
@@ -92,9 +92,10 @@ def test_bigquery_adapter_methods(self):
         client = conn.handle
 
         grant_target = GrantTarget.from_dict(ae_grant_target_dict)
-        dataset = client.get_dataset(
-            self.adapter.connections.dataset_from_id(grant_target.render())
+        dataset_ref = self.adapter.connections.dataset_ref(
+            grant_target.project, grant_target.dataset
         )
+        dataset = client.get_dataset(dataset_ref)
 
         expected_access_entry = AccessEntry(ae_role, ae_entity_type, ae_entity)
         self.assertTrue(expected_access_entry in dataset.access_entries)
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 1380871c9..eee92f1ab 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -617,8 +617,8 @@ def test_copy_bq_table_appends(self):
             write_disposition=dbt.adapters.bigquery.impl.WRITE_APPEND)
         args, kwargs = self.mock_client.copy_table.call_args
         self.mock_client.copy_table.assert_called_once_with(
-            [self._table_ref('project', 'dataset', 'table1', None)],
-            self._table_ref('project', 'dataset', 'table2', None),
+            [self._table_ref('project', 'dataset', 'table1')],
+            self._table_ref('project', 'dataset', 'table2'),
             job_config=ANY)
         args, kwargs = self.mock_client.copy_table.call_args
         self.assertEqual(
@@ -630,8 +630,8 @@ def test_copy_bq_table_truncates(self):
             write_disposition=dbt.adapters.bigquery.impl.WRITE_TRUNCATE)
         args, kwargs = self.mock_client.copy_table.call_args
         self.mock_client.copy_table.assert_called_once_with(
-            [self._table_ref('project', 'dataset', 'table1', None)],
-            self._table_ref('project', 'dataset', 'table2', None),
+            [self._table_ref('project', 'dataset', 'table1')],
+            self._table_ref('project', 'dataset', 'table2'),
             job_config=ANY)
         args, kwargs = self.mock_client.copy_table.call_args
         self.assertEqual(
@@ -647,12 +647,10 @@ def test_job_labels_invalid_json(self):
         labels = self.connections._labels_from_query_comment("not json")
         self.assertEqual(labels, {"query_comment": "not_json"})
 
-    def _table_ref(self, proj, ds, table, conn):
-        return google.cloud.bigquery.table.TableReference.from_string(
-            '{}.{}.{}'.format(proj, ds, table))
+    def _table_ref(self, proj, ds, table):
+        return self.connections.table_ref(proj, ds, table)
 
     def _copy_table(self, write_disposition):
-        self.connections.table_ref = self._table_ref
         source = BigQueryRelation.create(
             database='project', schema='dataset', identifier='table1')
         destination = BigQueryRelation.create(

From 6f68dce0dfb2f4ee7b778435b9296e645352bf3c Mon Sep 17 00:00:00 2001
From: Ollie <35436396+oliverrmaa@users.noreply.github.com>
Date: Fri, 28 Jan 2022 07:16:51 -0800
Subject: [PATCH 427/860] Make sure partition_by is in config if setting
 require_partition_filter (#109)

* initial commit

* update changelog

* fix typo in changelog

* Hui's feedback
---
 CHANGELOG.md                  | 3 +++
 dbt/adapters/bigquery/impl.py | 3 ++-
 2 files changed, 5 insertions(+), 1 deletion(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 48991628b..4b8d765e9 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -4,12 +4,15 @@
 
 ### Fixes
 - Fix test related to preventing coercion of boolean values (True, False) to numeric values (0, 1) in query results ([#93](https://github.com/dbt-labs/dbt-bigquery/issues/93))
+- Add a check in `get_table_options` to check that the table has a `partition_by` in the config.
+This will prevent BigQuery from throwing an error since non-partitioned tables cannot have `require_partition_filter` ([#107](https://github.com/dbt-labs/dbt-bigquery/issues/107))
 
 ### Under the hood
 - Address BigQuery API deprecation warning and simplify usage of `TableReference` and `DatasetReference` objects ([#97](https://github.com/dbt-labs/dbt-bigquery/issues/97))
 
 ### Contributors
 - [@hui-zheng](https://github.com/hui-zheng)([#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
+- [@oliverrmaa](https://github.com/oliverrmaa)([#109](https://github.com/dbt-labs/dbt-bigquery/pull/109))
 
 ## dbt-bigquery 1.0.0 (December 3, 2021)
 
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 73bf66e20..cb703f8b0 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -748,7 +748,8 @@ def get_table_options(
             # It doesn't apply the `require_partition_filter` option for a temporary table
             # so that we avoid the error by not specifying a partition with a temporary table
             # in the incremental model.
-            if config.get('require_partition_filter') is not None:
+            if config.get('require_partition_filter') is not None and \
+                    config.get('partition_by') is not None:
                 opts['require_partition_filter'] = config.get(
                     'require_partition_filter')
             if config.get('partition_expiration_days') is not None:

From 87095c4dec1a7b64496d32e33a0f0f7390083d70 Mon Sep 17 00:00:00 2001
From: Yu Ishikawa 
Date: Tue, 8 Feb 2022 02:06:50 +0900
Subject: [PATCH 428/860] [#104] Ignore the `Forbidden` exception in
 `list_relations_without_caching` (#108)

* [#104] Ignore the forbidden exception in `list_relations_without_caching`

* Update CHANGELOG.md

* Further update CHANGELOG.md

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 CHANGELOG.md                  | 2 ++
 dbt/adapters/bigquery/impl.py | 3 +++
 2 files changed, 5 insertions(+)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 4b8d765e9..7049c2456 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -6,6 +6,7 @@
 - Fix test related to preventing coercion of boolean values (True, False) to numeric values (0, 1) in query results ([#93](https://github.com/dbt-labs/dbt-bigquery/issues/93))
 - Add a check in `get_table_options` to check that the table has a `partition_by` in the config.
 This will prevent BigQuery from throwing an error since non-partitioned tables cannot have `require_partition_filter` ([#107](https://github.com/dbt-labs/dbt-bigquery/issues/107))
+- Ignore errors of the lack of permissions in `list_relations_without_caching` ([#104](https://github.com/dbt-labs/dbt-bigquery/issues/104))
 
 ### Under the hood
 - Address BigQuery API deprecation warning and simplify usage of `TableReference` and `DatasetReference` objects ([#97](https://github.com/dbt-labs/dbt-bigquery/issues/97))
@@ -13,6 +14,7 @@ This will prevent BigQuery from throwing an error since non-partitioned tables c
 ### Contributors
 - [@hui-zheng](https://github.com/hui-zheng)([#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
 - [@oliverrmaa](https://github.com/oliverrmaa)([#109](https://github.com/dbt-labs/dbt-bigquery/pull/109))
+- [@yu-iskw](https://github.com/yu-iskw)([#108](https://github.com/dbt-labs/dbt-bigquery/pull/108))
 
 ## dbt-bigquery 1.0.0 (December 3, 2021)
 
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index cb703f8b0..abc56bd5b 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -273,6 +273,9 @@ def list_relations_without_caching(
             return [self._bq_table_to_relation(table) for table in all_tables]
         except google.api_core.exceptions.NotFound:
             return []
+        except google.api_core.exceptions.Forbidden as exc:
+            logger.debug('list_relations_without_caching error: {}'.format(str(exc)))
+            return []
 
     def get_relation(
         self, database: str, schema: str, identifier: str

From 610a96d80719f77cc819c1bc83fde8c881e9af12 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 9 Feb 2022 16:14:15 -0600
Subject: [PATCH 429/860] adding unique_key to various objects in docs_generate
 in regards to dbt-core changes (#119)

---
 tests/integration/docs_generate_tests/test_docs_generate.py | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/tests/integration/docs_generate_tests/test_docs_generate.py b/tests/integration/docs_generate_tests/test_docs_generate.py
index 8621d1e15..1ec52c3a2 100644
--- a/tests/integration/docs_generate_tests/test_docs_generate.py
+++ b/tests/integration/docs_generate_tests/test_docs_generate.py
@@ -532,6 +532,7 @@ def rendered_model_config(self, **updates):
             'full_refresh': None,
             'on_schema_change': 'ignore',
             'meta': {},
+            'unique_key': None,
         }
         result.update(updates)
         return result
@@ -556,6 +557,7 @@ def rendered_seed_config(self, **updates):
             'schema': None,
             'alias': None,
             'meta': {},
+            'unique_key': None,
         }
         result.update(updates)
         return result

From d5d94f9ca5ab6e7380d705420f5681bd84e0b2a9 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Fri, 11 Feb 2022 14:14:54 -0600
Subject: [PATCH 430/860] adding new env  to integration.yml to be used to test
 against changes in another pr (#116)

---
 .github/workflows/integration.yml | 1 +
 1 file changed, 1 insertion(+)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 2cc9cd207..01c7dfba8 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -167,6 +167,7 @@ jobs:
         env:
           BIGQUERY_TEST_SERVICE_ACCOUNT_JSON: ${{ secrets.BIGQUERY_TEST_SERVICE_ACCOUNT_JSON }}
           BIGQUERY_TEST_ALT_DATABASE: ${{ secrets.BIGQUERY_TEST_ALT_DATABASE }}
+          BIGQUERY_TEST_NO_ACCESS_DATABASE: ${{ secrets.BIGQUERY_TEST_NO_ACCESS_DATABASE }}
         run: tox
 
       - uses: actions/upload-artifact@v2

From 5835c39b1e8b1d08229c3f40db6e7d2ec92a4bdf Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Tue, 22 Feb 2022 16:00:58 -0600
Subject: [PATCH 431/860] =?UTF-8?q?added=20unique=5Fkey=20to=20the=20test?=
 =?UTF-8?q?=5Fdocs=5Fgenerate=20places=20relevant=20to=20changes=E2=80=A6?=
 =?UTF-8?q?=20(#114)?=
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

* added unique_key to the test_docs_generate places relevant to changes made in core, TODO: double check that any other references to unique_key in bigquery there are two more on that file need to be changed to None value as well, and design an integration test

* init attempt at test, for unique key, getting type error for hashable currently

* moved to using basic hash method, and changing up calls and saving of values, failing now due to being unequal

* fixed minor errors have all models building properly including incremental model now only have to fine tune till assertions are correct

* remove items from docs_generate.py objects to move to secondary pr

* using assertTableEqual

* added new models and test for single unique key

* adding tests for single unique key, and removing unused import

* removing import

* add first draft for changelog which links to bigquery issue which points back to core pr to initial contributor issue/pr

* working on new version of unique_key as list test in incremental_schema_tests after pair coding with kyle, similiar test in vain to my previous version which is still in code but is using modified prexisting test functionality which seems a much cleaner simple test

* comments and docstring addition

* beginning transition to unify tests between adapters, only issue seems to be around models using ::date will fix tomorrow

* applied castings to expected tables as bigquery auto detects certain patterns from seeds, created a new class to cut down on runs of a test that was being inherited

* minor fixes

* minor naming changes and removal of file from older tests

* updates implementing mila final versions to remain similar

* removing unnecessary line
---
 CHANGELOG.md                                  |   1 +
 .../duplicated_unary_unique_key_list.sql      |  17 +
 .../models/empty_str_unique_key.sql           |  14 +
 .../models/empty_unique_key_list.sql          |  12 +
 .../models/expected/one_str__overwrite.sql    |  21 ++
 .../unique_key_list__inplace_overwrite.sql    |  21 ++
 .../models/no_unique_key.sql                  |  13 +
 .../nontyped_trinary_unique_key_list.sql      |  19 ++
 .../models/not_found_unique_key.sql           |  14 +
 .../models/not_found_unique_key_list.sql      |   8 +
 .../models/str_unique_key.sql                 |  17 +
 .../models/trinary_unique_key_list.sql        |  19 ++
 .../models/unary_unique_key_list.sql          |  17 +
 .../seeds/add_new_rows.sql                    |   9 +
 .../seeds/duplicate_insert.sql                |   5 +
 .../incremental_unique_id_test/seeds/seed.csv |   7 +
 .../test_incremental_unique_id.py             | 296 ++++++++++++++++++
 17 files changed, 510 insertions(+)
 create mode 100644 tests/integration/incremental_unique_id_test/models/duplicated_unary_unique_key_list.sql
 create mode 100644 tests/integration/incremental_unique_id_test/models/empty_str_unique_key.sql
 create mode 100644 tests/integration/incremental_unique_id_test/models/empty_unique_key_list.sql
 create mode 100644 tests/integration/incremental_unique_id_test/models/expected/one_str__overwrite.sql
 create mode 100644 tests/integration/incremental_unique_id_test/models/expected/unique_key_list__inplace_overwrite.sql
 create mode 100644 tests/integration/incremental_unique_id_test/models/no_unique_key.sql
 create mode 100644 tests/integration/incremental_unique_id_test/models/nontyped_trinary_unique_key_list.sql
 create mode 100644 tests/integration/incremental_unique_id_test/models/not_found_unique_key.sql
 create mode 100644 tests/integration/incremental_unique_id_test/models/not_found_unique_key_list.sql
 create mode 100644 tests/integration/incremental_unique_id_test/models/str_unique_key.sql
 create mode 100644 tests/integration/incremental_unique_id_test/models/trinary_unique_key_list.sql
 create mode 100644 tests/integration/incremental_unique_id_test/models/unary_unique_key_list.sql
 create mode 100644 tests/integration/incremental_unique_id_test/seeds/add_new_rows.sql
 create mode 100644 tests/integration/incremental_unique_id_test/seeds/duplicate_insert.sql
 create mode 100644 tests/integration/incremental_unique_id_test/seeds/seed.csv
 create mode 100644 tests/integration/incremental_unique_id_test/test_incremental_unique_id.py

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 7049c2456..bf416cce7 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,6 +1,7 @@
 ## dbt-bigquery 1.1.0 (Release TBD)
 ### Features
 - Provide a fine-grained control of the timeout and retry of BigQuery query with four new dbt profile configs: `job_creation_timeout_seconds`, `job_execution_timeout_seconds`, `job_retry_deadline_seconds`, and `job_retries` ([#45](https://github.com/dbt-labs/dbt-bigquery/issues/45), [#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
+- Adds new integration test to check against new ability to allow unique_key to be a list. [#112](https://github.com/dbt-labs/dbt-bigquery/issues/112)
 
 ### Fixes
 - Fix test related to preventing coercion of boolean values (True, False) to numeric values (0, 1) in query results ([#93](https://github.com/dbt-labs/dbt-bigquery/issues/93))
diff --git a/tests/integration/incremental_unique_id_test/models/duplicated_unary_unique_key_list.sql b/tests/integration/incremental_unique_id_test/models/duplicated_unary_unique_key_list.sql
new file mode 100644
index 000000000..7290b6c43
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/models/duplicated_unary_unique_key_list.sql
@@ -0,0 +1,17 @@
+{{
+    config(
+        materialized='incremental',
+        unique_key=['state', 'state']
+    )
+}}
+
+select
+    state as state,
+    county as county,
+    city as city,
+    last_visit_date as last_visit_date
+from {{ ref('seed') }}
+
+{% if is_incremental() %}
+    where last_visit_date > (select max(last_visit_date) from {{ this }})
+{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/empty_str_unique_key.sql b/tests/integration/incremental_unique_id_test/models/empty_str_unique_key.sql
new file mode 100644
index 000000000..5260e177c
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/models/empty_str_unique_key.sql
@@ -0,0 +1,14 @@
+{{
+    config(
+        materialized='incremental',
+        unique_key=''
+    )
+}}
+
+select
+    *
+from {{ ref('seed') }}
+
+{% if is_incremental() %}
+    where last_visit_date > (select max(last_visit_date) from {{ this }})
+{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/empty_unique_key_list.sql b/tests/integration/incremental_unique_id_test/models/empty_unique_key_list.sql
new file mode 100644
index 000000000..c582d532c
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/models/empty_unique_key_list.sql
@@ -0,0 +1,12 @@
+{{
+    config(
+        materialized='incremental',
+        unique_key=[]
+    )
+}}
+
+select * from {{ ref('seed') }}
+
+{% if is_incremental() %}
+    where last_visit_date > (select max(last_visit_date) from {{ this }})
+{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/expected/one_str__overwrite.sql b/tests/integration/incremental_unique_id_test/models/expected/one_str__overwrite.sql
new file mode 100644
index 000000000..baa37c601
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/models/expected/one_str__overwrite.sql
@@ -0,0 +1,21 @@
+{{
+    config(
+        materialized='table'
+    )
+}}
+
+select
+    'CT' as state,
+    'Hartford' as county,
+    'Hartford' as city,
+    cast('2022-02-14' as date) as last_visit_date
+union all
+select 'MA','Suffolk','Boston','2020-02-12'
+union all
+select 'NJ','Mercer','Trenton','2022-01-01'
+union all
+select 'NY','Kings','Brooklyn','2021-04-02'
+union all
+select 'NY','New York','Manhattan','2021-04-01'
+union all
+select 'PA','Philadelphia','Philadelphia','2021-05-21'
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/expected/unique_key_list__inplace_overwrite.sql b/tests/integration/incremental_unique_id_test/models/expected/unique_key_list__inplace_overwrite.sql
new file mode 100644
index 000000000..baa37c601
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/models/expected/unique_key_list__inplace_overwrite.sql
@@ -0,0 +1,21 @@
+{{
+    config(
+        materialized='table'
+    )
+}}
+
+select
+    'CT' as state,
+    'Hartford' as county,
+    'Hartford' as city,
+    cast('2022-02-14' as date) as last_visit_date
+union all
+select 'MA','Suffolk','Boston','2020-02-12'
+union all
+select 'NJ','Mercer','Trenton','2022-01-01'
+union all
+select 'NY','Kings','Brooklyn','2021-04-02'
+union all
+select 'NY','New York','Manhattan','2021-04-01'
+union all
+select 'PA','Philadelphia','Philadelphia','2021-05-21'
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/no_unique_key.sql b/tests/integration/incremental_unique_id_test/models/no_unique_key.sql
new file mode 100644
index 000000000..44a63e75c
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/models/no_unique_key.sql
@@ -0,0 +1,13 @@
+{{
+    config(
+        materialized='incremental'
+    )
+}}
+
+select
+    *
+from {{ ref('seed') }}
+
+{% if is_incremental() %}
+    where last_visit_date > (select max(last_visit_date) from {{ this }})
+{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/nontyped_trinary_unique_key_list.sql b/tests/integration/incremental_unique_id_test/models/nontyped_trinary_unique_key_list.sql
new file mode 100644
index 000000000..52b4509f0
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/models/nontyped_trinary_unique_key_list.sql
@@ -0,0 +1,19 @@
+-- for comparing against auto-typed seeds
+
+{{
+    config(
+        materialized='incremental',
+        unique_key=['state', 'county', 'city']
+    )
+}}
+
+select
+    state as state,
+    county as county,
+    city as city,
+    last_visit_date as last_visit_date
+from {{ ref('seed') }}
+
+{% if is_incremental() %}
+    where last_visit_date > (select max(last_visit_date) from {{ this }})
+{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/not_found_unique_key.sql b/tests/integration/incremental_unique_id_test/models/not_found_unique_key.sql
new file mode 100644
index 000000000..d247aa341
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/models/not_found_unique_key.sql
@@ -0,0 +1,14 @@
+{{
+    config(
+        materialized='incremental',
+        unique_key='thisisnotacolumn'
+    )
+}}
+
+select
+    *
+from {{ ref('seed') }}
+
+{% if is_incremental() %}
+    where last_visit_date > (select max(last_visit_date) from {{ this }})
+{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/not_found_unique_key_list.sql b/tests/integration/incremental_unique_id_test/models/not_found_unique_key_list.sql
new file mode 100644
index 000000000..f1462a48f
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/models/not_found_unique_key_list.sql
@@ -0,0 +1,8 @@
+{{
+    config(
+        materialized='incremental',
+        unique_key=['state', 'thisisnotacolumn']
+    )
+}}
+
+select * from {{ ref('seed') }}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/str_unique_key.sql b/tests/integration/incremental_unique_id_test/models/str_unique_key.sql
new file mode 100644
index 000000000..2f9fc2987
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/models/str_unique_key.sql
@@ -0,0 +1,17 @@
+{{
+    config(
+        materialized='incremental',
+        unique_key='state'
+    )
+}}
+
+select
+    state as state,
+    county as county,
+    city as city,
+    last_visit_date as last_visit_date
+from {{ ref('seed') }}
+
+{% if is_incremental() %}
+    where last_visit_date > (select max(last_visit_date) from {{ this }})
+{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/trinary_unique_key_list.sql b/tests/integration/incremental_unique_id_test/models/trinary_unique_key_list.sql
new file mode 100644
index 000000000..0359546bf
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/models/trinary_unique_key_list.sql
@@ -0,0 +1,19 @@
+-- types needed to compare against expected model reliably
+
+{{
+    config(
+        materialized='incremental',
+        unique_key=['state', 'county', 'city']
+    )
+}}
+
+select
+    state as state,
+    county as county,
+    city as city,
+    last_visit_date as last_visit_date
+from {{ ref('seed') }}
+
+{% if is_incremental() %}
+    where last_visit_date > (select max(last_visit_date) from {{ this }})
+{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/unary_unique_key_list.sql b/tests/integration/incremental_unique_id_test/models/unary_unique_key_list.sql
new file mode 100644
index 000000000..7f5875f85
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/models/unary_unique_key_list.sql
@@ -0,0 +1,17 @@
+{{
+    config(
+        materialized='incremental',
+        unique_key=['state']
+    )
+}}
+
+select
+    state as state,
+    county as county,
+    city as city,
+    last_visit_date as last_visit_date
+from {{ ref('seed') }}
+
+{% if is_incremental() %}
+    where last_visit_date > (select max(last_visit_date) from {{ this }})
+{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/seeds/add_new_rows.sql b/tests/integration/incremental_unique_id_test/seeds/add_new_rows.sql
new file mode 100644
index 000000000..6d515ca3a
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/seeds/add_new_rows.sql
@@ -0,0 +1,9 @@
+-- insert two new rows, both of which should be in incremental model
+--   with any unique columns
+insert into {schema}.seed
+    (state, county, city, last_visit_date)
+values ('WA','King','Seattle','2022-02-01');
+
+insert into {schema}.seed
+    (state, county, city, last_visit_date)
+values ('CA','Los Angeles','Los Angeles','2022-02-01');
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/seeds/duplicate_insert.sql b/tests/integration/incremental_unique_id_test/seeds/duplicate_insert.sql
new file mode 100644
index 000000000..bd73cee86
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/seeds/duplicate_insert.sql
@@ -0,0 +1,5 @@
+-- insert new row, which should not be in incremental model
+--  with primary or first three columns unique
+insert into {schema}.seed
+    (state, county, city, last_visit_date)
+values ('CT','Hartford','Hartford','2022-02-14');
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/seeds/seed.csv b/tests/integration/incremental_unique_id_test/seeds/seed.csv
new file mode 100644
index 000000000..b988827fb
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/seeds/seed.csv
@@ -0,0 +1,7 @@
+state,county,city,last_visit_date
+CT,Hartford,Hartford,2020-09-23
+MA,Suffolk,Boston,2020-02-12
+NJ,Mercer,Trenton,2022-01-01
+NY,Kings,Brooklyn,2021-04-02
+NY,New York,Manhattan,2021-04-01
+PA,Philadelphia,Philadelphia,2021-05-21
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/test_incremental_unique_id.py b/tests/integration/incremental_unique_id_test/test_incremental_unique_id.py
new file mode 100644
index 000000000..a21d84327
--- /dev/null
+++ b/tests/integration/incremental_unique_id_test/test_incremental_unique_id.py
@@ -0,0 +1,296 @@
+from tests.integration.base import DBTIntegrationTest, use_profile
+from dbt.contracts.results import RunStatus
+from collections import namedtuple
+from pathlib import Path
+
+
+TestResults = namedtuple(
+    'TestResults',
+    ['seed_count', 'model_count', 'seed_rows', 'inc_test_model_count',
+     'opt_model_count', 'relation'],
+)
+
+
+class TestIncrementalUniqueKey(DBTIntegrationTest):
+    @property
+    def schema(self):
+        return 'incremental_unique_key'
+
+    @property
+    def models(self):
+        return 'models'
+
+    def update_incremental_model(self, incremental_model):
+        '''update incremental model after the seed table has been updated'''
+        model_result_set = self.run_dbt(['run', '--select', incremental_model])
+        return len(model_result_set)
+
+    def setup_test(self, seed, incremental_model, update_sql_file):
+        '''build a test case and return values for assertions'''
+        
+        # Idempotently create some number of seeds and incremental models
+        seed_count = len(self.run_dbt(
+            ['seed', '--select', seed, '--full-refresh']
+        ))
+        model_count = len(self.run_dbt(
+            ['run', '--select', incremental_model, '--full-refresh']
+        ))
+        
+         # Upate seed and return new row count
+        row_count_query = 'select * from {}.{}'.format(
+            self.unique_schema(),
+            seed
+        )
+        self.run_sql_file(Path('seeds') / Path(update_sql_file + '.sql'))
+        seed_rows = len(self.run_sql(row_count_query, fetch='all'))
+
+        inc_test_model_count = self.update_incremental_model(
+            incremental_model=incremental_model
+        )
+
+        return (seed_count, model_count, seed_rows, inc_test_model_count)
+
+    def test_scenario_correctness(self, expected_fields, test_case_fields):
+        '''Invoke assertions to verify correct build functionality'''
+        # 1. test seed(s) should build afresh
+        self.assertEqual(
+            expected_fields.seed_count, test_case_fields.seed_count
+        )
+        # 2. test model(s) should build afresh
+        self.assertEqual(
+            expected_fields.model_count, test_case_fields.model_count
+        )
+        # 3. seeds should have intended row counts post update
+        self.assertEqual(
+            expected_fields.seed_rows, test_case_fields.seed_rows
+        )
+        # 4. incremental test model(s) should be updated
+        self.assertEqual(
+            expected_fields.inc_test_model_count,
+            test_case_fields.inc_test_model_count
+        )
+        # 5. extra incremental model(s) should be built; optional since
+        #   comparison may be between an incremental model and seed
+        if (expected_fields.opt_model_count and
+            test_case_fields.opt_model_count):
+            self.assertEqual(
+                expected_fields.opt_model_count,
+                test_case_fields.opt_model_count
+            )
+        # 6. result table should match intended result set (itself a relation)
+        self.assertTablesEqual(
+            expected_fields.relation, test_case_fields.relation
+        )
+
+    def stub_expected_fields(
+        self, relation, seed_rows, opt_model_count=None
+    ):
+        return TestResults(
+            seed_count=1, model_count=1, seed_rows=seed_rows,
+            inc_test_model_count=1, opt_model_count=opt_model_count,
+            relation=relation
+        )
+
+    def fail_to_build_inc_missing_unique_key_column(self, incremental_model_name):
+        '''should pass back error state when trying build an incremental
+           model whose unique key or keylist includes a column missing
+           from the incremental model'''
+        seed_count = len(self.run_dbt(
+            ['seed', '--select', 'seed', '--full-refresh']
+        ))
+        # unique keys are not applied on first run, so two are needed
+        self.run_dbt(
+            ['run', '--select', incremental_model_name, '--full-refresh'],
+            expect_pass=True
+        )
+        run_result = self.run_dbt(
+            ['run', '--select', incremental_model_name],
+            expect_pass=False
+        ).results[0]
+
+        return run_result.status, run_result.message
+
+
+class TestNoIncrementalUniqueKey(TestIncrementalUniqueKey):
+    @use_profile('bigquery')
+    def test__bigquery_no_unique_keys(self):
+        '''with no unique keys, seed and model should match'''
+        seed='seed'
+        seed_rows=8
+        incremental_model='no_unique_key'
+        update_sql_file='add_new_rows'
+
+        expected_fields = self.stub_expected_fields(
+            relation=seed, seed_rows=seed_rows
+        )
+        test_case_fields = TestResults(
+            *self.setup_test(seed, incremental_model, update_sql_file),
+            opt_model_count=None, relation=incremental_model
+        )
+
+        self.test_scenario_correctness(expected_fields, test_case_fields)
+
+
+class TestIncrementalStrUniqueKey(TestIncrementalUniqueKey):
+    @use_profile('bigquery')
+    def test__bigquery_empty_str_unique_key(self):
+        '''with empty string for unique key, seed and model should match'''
+        seed='seed'
+        seed_rows=8
+        incremental_model='empty_str_unique_key'
+        update_sql_file='add_new_rows'
+
+        expected_fields = self.stub_expected_fields(
+            relation=seed, seed_rows=seed_rows
+        )
+        test_case_fields = TestResults(
+            *self.setup_test(seed, incremental_model, update_sql_file),
+            opt_model_count=None, relation=incremental_model
+        )
+
+        self.test_scenario_correctness(expected_fields, test_case_fields)
+
+    @use_profile('bigquery')
+    def test__bigquery_one_unique_key(self):
+        '''with one unique key, model will overwrite existing row'''
+        seed='seed'
+        seed_rows=7
+        incremental_model='str_unique_key'
+        update_sql_file='duplicate_insert'
+        expected_model='one_str__overwrite'
+
+        expected_fields = self.stub_expected_fields(
+            relation=expected_model, seed_rows=seed_rows, opt_model_count=1
+        )
+        test_case_fields = TestResults(
+            *self.setup_test(seed, incremental_model, update_sql_file),
+            opt_model_count=self.update_incremental_model(expected_model),
+            relation=incremental_model
+        )
+
+        self.test_scenario_correctness(expected_fields, test_case_fields)
+
+    @use_profile('bigquery')
+    def test__bigquery_bad_unique_key(self):
+        '''expect compilation error from unique key not being a column'''
+
+        err_msg = "Name thisisnotacolumn not found inside DBT_INTERNAL_SOURCE"
+
+        (status, exc) = self.fail_to_build_inc_missing_unique_key_column(
+            incremental_model_name='not_found_unique_key'
+        )
+
+        self.assertEqual(status, RunStatus.Error)
+        self.assertTrue(err_msg in exc)
+
+
+class TestIncrementalListUniqueKey(TestIncrementalUniqueKey):
+    @use_profile('bigquery')
+    def test__bigquery_empty_unique_key_list(self):
+        '''with no unique keys, seed and model should match'''
+        seed='seed'
+        seed_rows=8
+        incremental_model='empty_unique_key_list'
+        update_sql_file='add_new_rows'
+
+        expected_fields = self.stub_expected_fields(
+            relation=seed, seed_rows=seed_rows
+        )
+        test_case_fields = TestResults(
+            *self.setup_test(seed, incremental_model, update_sql_file),
+            opt_model_count=None, relation=incremental_model
+        )
+
+        self.test_scenario_correctness(expected_fields, test_case_fields)
+
+    @use_profile('bigquery')
+    def test__bigquery_unary_unique_key_list(self):
+        '''with one unique key, model will overwrite existing row'''
+        seed='seed'
+        seed_rows=7
+        incremental_model='unary_unique_key_list'
+        update_sql_file='duplicate_insert'
+        expected_model='unique_key_list__inplace_overwrite'
+
+        expected_fields = self.stub_expected_fields(
+            relation=expected_model, seed_rows=seed_rows, opt_model_count=1
+        )
+        test_case_fields = TestResults(
+            *self.setup_test(seed, incremental_model, update_sql_file),
+            opt_model_count=self.update_incremental_model(expected_model),
+            relation=incremental_model
+        )
+
+        self.test_scenario_correctness(expected_fields, test_case_fields)
+
+    @use_profile('bigquery')
+    def test__bigquery_duplicated_unary_unique_key_list(self):
+        '''with two of the same unique key, model will overwrite existing row'''
+        seed='seed'
+        seed_rows=7
+        incremental_model='duplicated_unary_unique_key_list'
+        update_sql_file='duplicate_insert'
+        expected_model='unique_key_list__inplace_overwrite'
+
+        expected_fields = self.stub_expected_fields(
+            relation=expected_model, seed_rows=seed_rows, opt_model_count=1
+        )
+        test_case_fields = TestResults(
+            *self.setup_test(seed, incremental_model, update_sql_file),
+            opt_model_count=self.update_incremental_model(expected_model),
+            relation=incremental_model
+        )
+
+        self.test_scenario_correctness(expected_fields, test_case_fields)
+
+    @use_profile('bigquery')
+    def test__bigquery_trinary_unique_key_list(self):
+        '''with three unique keys, model will overwrite existing row'''
+        seed='seed'
+        seed_rows=7
+        incremental_model='trinary_unique_key_list'
+        update_sql_file='duplicate_insert'
+        expected_model='unique_key_list__inplace_overwrite'
+
+        expected_fields = self.stub_expected_fields(
+            relation=expected_model, seed_rows=seed_rows, opt_model_count=1
+        )
+        test_case_fields = TestResults(
+            *self.setup_test(seed, incremental_model, update_sql_file),
+            opt_model_count=self.update_incremental_model(expected_model),
+            relation=incremental_model
+        )
+
+        self.test_scenario_correctness(expected_fields, test_case_fields)
+
+    @use_profile('bigquery')
+    def test__bigquery_trinary_unique_key_list_no_update(self):
+        '''even with three unique keys, adding distinct rows to seed does not
+           cause seed and model to diverge'''
+        seed='seed'
+        seed_rows=8
+        incremental_model='nontyped_trinary_unique_key_list'
+        update_sql_file='add_new_rows'
+
+        expected_fields = self.stub_expected_fields(
+            relation=seed, seed_rows=seed_rows
+        )
+        test_case_fields = TestResults(
+            *self.setup_test(seed, incremental_model, update_sql_file),
+            opt_model_count=None, relation=incremental_model
+        )
+
+        self.test_scenario_correctness(expected_fields, test_case_fields)
+
+    @use_profile('bigquery')
+    def test__bigquery_bad_unique_key_list(self):
+        '''expect compilation error from unique key not being a column'''
+
+        err_msg = "Name thisisnotacolumn not found inside DBT_INTERNAL_SOURCE"
+
+        (status, exc) = self.fail_to_build_inc_missing_unique_key_column(
+            incremental_model_name='not_found_unique_key_list'
+        )
+
+        self.assertEqual(status, RunStatus.Error)
+        self.assertTrue(err_msg in exc)

From a426d3aaee7cb2b642fc0cbfe421d785c67529e3 Mon Sep 17 00:00:00 2001
From: pgoslatara 
Date: Thu, 10 Mar 2022 22:03:37 +0100
Subject: [PATCH 432/860] Adding upload_file adapter (#121)

* Adding upload_file adapter

* flake8 formatting

* Replacing get_timeout with newer get_job_execution_timeout_seconds

* Adding integration tests for upload_file macro

* Removing conn arg from table_ref method

* Updating schema method to upload_file

* Adding checks on created tables

* Correcting class name

* Updating CHANGELOG.md

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 CHANGELOG.md                                  |   2 +
 dbt/adapters/bigquery/impl.py                 |  23 ++++
 dbt/include/bigquery/macros/adapters.sql      |   8 ++
 .../upload_file_tests/csv/source.csv          | 101 ++++++++++++++++++
 .../upload_file_tests/macros/upload_file.sql  |   3 +
 .../upload_file_tests/ndjson/source.ndjson    | 100 +++++++++++++++++
 .../upload_file_tests/parquet/source.parquet  | Bin 0 -> 10468 bytes
 .../upload_file_tests/test_upload_file.py     |  93 ++++++++++++++++
 8 files changed, 330 insertions(+)
 create mode 100644 tests/integration/upload_file_tests/csv/source.csv
 create mode 100644 tests/integration/upload_file_tests/macros/upload_file.sql
 create mode 100644 tests/integration/upload_file_tests/ndjson/source.ndjson
 create mode 100644 tests/integration/upload_file_tests/parquet/source.parquet
 create mode 100644 tests/integration/upload_file_tests/test_upload_file.py

diff --git a/CHANGELOG.md b/CHANGELOG.md
index bf416cce7..fb2d5960b 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -2,6 +2,7 @@
 ### Features
 - Provide a fine-grained control of the timeout and retry of BigQuery query with four new dbt profile configs: `job_creation_timeout_seconds`, `job_execution_timeout_seconds`, `job_retry_deadline_seconds`, and `job_retries` ([#45](https://github.com/dbt-labs/dbt-bigquery/issues/45), [#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
 - Adds new integration test to check against new ability to allow unique_key to be a list. [#112](https://github.com/dbt-labs/dbt-bigquery/issues/112)
+- Added upload_file macro to upload a local file to a table. [#102](https://github.com/dbt-labs/dbt-bigquery/issues/102)
 
 ### Fixes
 - Fix test related to preventing coercion of boolean values (True, False) to numeric values (0, 1) in query results ([#93](https://github.com/dbt-labs/dbt-bigquery/issues/93))
@@ -16,6 +17,7 @@ This will prevent BigQuery from throwing an error since non-partitioned tables c
 - [@hui-zheng](https://github.com/hui-zheng)([#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
 - [@oliverrmaa](https://github.com/oliverrmaa)([#109](https://github.com/dbt-labs/dbt-bigquery/pull/109))
 - [@yu-iskw](https://github.com/yu-iskw)([#108](https://github.com/dbt-labs/dbt-bigquery/pull/108))
+- [@pgoslatara](https://github.com/pgoslatara) ([#66](https://github.com/dbt-labs/dbt-bigquery/pull/121))
 
 ## dbt-bigquery 1.0.0 (December 3, 2021)
 
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index abc56bd5b..ff983c0b8 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -686,6 +686,29 @@ def load_dataframe(self, database, schema, table_name, agate_table,
         with self.connections.exception_handler("LOAD TABLE"):
             self.poll_until_job_completes(job, timeout)
 
+    @available.parse_none
+    def upload_file(self, local_file_path: str, database: str, table_schema: str,
+                    table_name: str, **kwargs) -> None:
+        conn = self.connections.get_thread_connection()
+        client = conn.handle
+
+        table_ref = self.connections.table_ref(database, table_schema, table_name)
+
+        load_config = google.cloud.bigquery.LoadJobConfig()
+        for k, v in kwargs['kwargs'].items():
+            if k == "schema":
+                setattr(load_config, k, json.loads(v))
+            else:
+                setattr(load_config, k, v)
+
+        with open(local_file_path, "rb") as f:
+            job = client.load_table_from_file(f, table_ref, rewind=True,
+                                              job_config=load_config)
+
+        timeout = self.connections.get_job_execution_timeout_seconds(conn)
+        with self.connections.exception_handler("LOAD TABLE"):
+            self.poll_until_job_completes(job, timeout)
+
     @classmethod
     def _catalog_filter_table(
         cls, table: agate.Table, manifest: Manifest
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index b98a43385..fa30922fe 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -214,3 +214,11 @@ group by unique_field
 having count(*) > 1
 
 {% endmacro %}
+
+{% macro bigquery__upload_file(local_file_path, database, table_schema, table_name) %}
+
+  {{ log("kwargs: " ~ kwargs) }}
+
+  {% do adapter.upload_file(local_file_path, database, table_schema, table_name, kwargs=kwargs) %}
+
+{% endmacro %}
diff --git a/tests/integration/upload_file_tests/csv/source.csv b/tests/integration/upload_file_tests/csv/source.csv
new file mode 100644
index 000000000..a8f87412e
--- /dev/null
+++ b/tests/integration/upload_file_tests/csv/source.csv
@@ -0,0 +1,101 @@
+favorite_color,id,first_name,email,ip_address,updated_at
+blue,1,Larry,lking0@miitbeian.gov.cn,'69.135.206.194',2008-09-12 19:08:31
+blue,2,Larry,lperkins1@toplist.cz,'64.210.133.162',1978-05-09 04:15:14
+blue,3,Anna,amontgomery2@miitbeian.gov.cn,'168.104.64.114',2011-10-16 04:07:57
+blue,4,Sandra,sgeorge3@livejournal.com,'229.235.252.98',1973-07-19 10:52:43
+blue,5,Fred,fwoods4@google.cn,'78.229.170.124',2012-09-30 16:38:29
+blue,6,Stephen,shanson5@livejournal.com,'182.227.157.105',1995-11-07 21:40:50
+blue,7,William,wmartinez6@upenn.edu,'135.139.249.50',1982-09-05 03:11:59
+blue,8,Jessica,jlong7@hao123.com,'203.62.178.210',1991-10-16 11:03:15
+blue,9,Douglas,dwhite8@tamu.edu,'178.187.247.1',1979-10-01 09:49:48
+blue,10,Lisa,lcoleman9@nydailynews.com,'168.234.128.249',2011-05-26 07:45:49
+blue,11,Ralph,rfieldsa@home.pl,'55.152.163.149',1972-11-18 19:06:11
+blue,12,Louise,lnicholsb@samsung.com,'141.116.153.154',2014-11-25 20:56:14
+blue,13,Clarence,cduncanc@sfgate.com,'81.171.31.133',2011-11-17 07:02:36
+blue,14,Daniel,dfranklind@omniture.com,'8.204.211.37',1980-09-13 00:09:04
+blue,15,Katherine,klanee@auda.org.au,'176.96.134.59',1997-08-22 19:36:56
+blue,16,Billy,bwardf@wikia.com,'214.108.78.85',2003-10-19 02:14:47
+blue,17,Annie,agarzag@ocn.ne.jp,'190.108.42.70',1988-10-28 15:12:35
+blue,18,Shirley,scolemanh@fastcompany.com,'109.251.164.84',1988-08-24 10:50:57
+blue,19,Roger,rfrazieri@scribd.com,'38.145.218.108',1985-12-31 15:17:15
+blue,20,Lillian,lstanleyj@goodreads.com,'47.57.236.17',1970-06-08 02:09:05
+blue,21,Aaron,arodriguezk@nps.gov,'205.245.118.221',1985-10-11 23:07:49
+blue,22,Patrick,pparkerl@techcrunch.com,'19.8.100.182',2006-03-29 12:53:56
+blue,23,Phillip,pmorenom@intel.com,'41.38.254.103',2011-11-07 15:35:43
+blue,24,Henry,hgarcian@newsvine.com,'1.191.216.252',2008-08-28 08:30:44
+blue,25,Irene,iturnero@opera.com,'50.17.60.190',1994-04-01 07:15:02
+blue,26,Andrew,adunnp@pen.io,'123.52.253.176',2000-11-01 06:03:25
+blue,27,David,dgutierrezq@wp.com,'238.23.203.42',1988-01-25 07:29:18
+blue,28,Henry,hsanchezr@cyberchimps.com,'248.102.2.185',1983-01-01 13:36:37
+blue,29,Evelyn,epetersons@gizmodo.com,'32.80.46.119',1979-07-16 17:24:12
+blue,30,Tammy,tmitchellt@purevolume.com,'249.246.167.88',2001-04-03 10:00:23
+blue,31,Jacqueline,jlittleu@domainmarket.com,'127.181.97.47',1986-02-11 21:35:50
+blue,32,Earl,eortizv@opera.com,'166.47.248.240',1996-07-06 08:16:27
+blue,33,Juan,jgordonw@sciencedirect.com,'71.77.2.200',1987-01-31 03:46:44
+blue,34,Diane,dhowellx@nyu.edu,'140.94.133.12',1994-06-11 02:30:05
+blue,35,Randy,rkennedyy@microsoft.com,'73.255.34.196',2005-05-26 20:28:39
+blue,36,Janice,jriveraz@time.com,'22.214.227.32',1990-02-09 04:16:52
+blue,37,Laura,lperry10@diigo.com,'159.148.145.73',2015-03-17 05:59:25
+blue,38,Gary,gray11@statcounter.com,'40.193.124.56',1970-01-27 10:04:51
+blue,39,Jesse,jmcdonald12@typepad.com,'31.7.86.103',2009-03-14 08:14:29
+blue,40,Sandra,sgonzalez13@goodreads.com,'223.80.168.239',1993-05-21 14:08:54
+blue,41,Scott,smoore14@archive.org,'38.238.46.83',1980-08-30 11:16:56
+blue,42,Phillip,pevans15@cisco.com,'158.234.59.34',2011-12-15 23:26:31
+blue,43,Steven,sriley16@google.ca,'90.247.57.68',2011-10-29 19:03:28
+blue,44,Deborah,dbrown17@hexun.com,'179.125.143.240',1995-04-10 14:36:07
+blue,45,Lori,lross18@ow.ly,'64.80.162.180',1980-12-27 16:49:15
+blue,46,Sean,sjackson19@tumblr.com,'240.116.183.69',1988-06-12 21:24:45
+blue,47,Terry,tbarnes1a@163.com,'118.38.213.137',1997-09-22 16:43:19
+blue,48,Dorothy,dross1b@ebay.com,'116.81.76.49',2005-02-28 13:33:24
+blue,49,Samuel,swashington1c@house.gov,'38.191.253.40',1989-01-19 21:15:48
+blue,50,Ralph,rcarter1d@tinyurl.com,'104.84.60.174',2007-08-11 10:21:49
+green,51,Wayne,whudson1e@princeton.edu,'90.61.24.102',1983-07-03 16:58:12
+green,52,Rose,rjames1f@plala.or.jp,'240.83.81.10',1995-06-08 11:46:23
+green,53,Louise,lcox1g@theglobeandmail.com,'105.11.82.145',2016-09-19 14:45:51
+green,54,Kenneth,kjohnson1h@independent.co.uk,'139.5.45.94',1976-08-17 11:26:19
+green,55,Donna,dbrown1i@amazon.co.uk,'19.45.169.45',2006-05-27 16:51:40
+green,56,Johnny,jvasquez1j@trellian.com,'118.202.238.23',1975-11-17 08:42:32
+green,57,Patrick,pramirez1k@tamu.edu,'231.25.153.198',1997-08-06 11:51:09
+green,58,Helen,hlarson1l@prweb.com,'8.40.21.39',1993-08-04 19:53:40
+green,59,Patricia,pspencer1m@gmpg.org,'212.198.40.15',1977-08-03 16:37:27
+green,60,Joseph,jspencer1n@marriott.com,'13.15.63.238',2005-07-23 20:22:06
+green,61,Phillip,pschmidt1o@blogtalkradio.com,'177.98.201.190',1976-05-19 21:47:44
+green,62,Joan,jwebb1p@google.ru,'105.229.170.71',1972-09-07 17:53:47
+green,63,Phyllis,pkennedy1q@imgur.com,'35.145.8.244',2000-01-01 22:33:37
+green,64,Katherine,khunter1r@smh.com.au,'248.168.205.32',1991-01-09 06:40:24
+green,65,Laura,lvasquez1s@wiley.com,'128.129.115.152',1997-10-23 12:04:56
+green,66,Juan,jdunn1t@state.gov,'44.228.124.51',2004-11-10 05:07:35
+green,67,Judith,jholmes1u@wiley.com,'40.227.179.115',1977-08-02 17:01:45
+green,68,Beverly,bbaker1v@wufoo.com,'208.34.84.59',2016-03-06 20:07:23
+green,69,Lawrence,lcarr1w@flickr.com,'59.158.212.223',1988-09-13 06:07:21
+green,70,Gloria,gwilliams1x@mtv.com,'245.231.88.33',1995-03-18 22:32:46
+green,71,Steven,ssims1y@cbslocal.com,'104.50.58.255',2001-08-05 21:26:20
+green,72,Betty,bmills1z@arstechnica.com,'103.177.214.220',1981-12-14 21:26:54
+green,73,Mildred,mfuller20@prnewswire.com,'151.158.8.130',2000-04-19 10:13:55
+green,74,Donald,dday21@icq.com,'9.178.102.255',1972-12-03 00:58:24
+green,75,Eric,ethomas22@addtoany.com,'85.2.241.227',1992-11-01 05:59:30
+green,76,Joyce,jarmstrong23@sitemeter.com,'169.224.20.36',1985-10-24 06:50:01
+green,77,Maria,mmartinez24@amazonaws.com,'143.189.167.135',2005-10-05 05:17:42
+green,78,Harry,hburton25@youtube.com,'156.47.176.237',1978-03-26 05:53:33
+green,79,Kevin,klawrence26@hao123.com,'79.136.183.83',1994-10-12 04:38:52
+green,80,David,dhall27@prweb.com,'133.149.172.153',1976-12-15 16:24:24
+green,81,Kathy,kperry28@twitter.com,'229.242.72.228',1979-03-04 02:58:56
+green,82,Adam,aprice29@elegantthemes.com,'13.145.21.10',1982-11-07 11:46:59
+green,83,Brandon,bgriffin2a@va.gov,'73.249.128.212',2013-10-30 05:30:36
+green,84,Henry,hnguyen2b@discovery.com,'211.36.214.242',1985-01-09 06:37:27
+green,85,Eric,esanchez2c@edublogs.org,'191.166.188.251',2004-05-01 23:21:42
+green,86,Jason,jlee2d@jimdo.com,'193.92.16.182',1973-01-08 09:05:39
+green,87,Diana,drichards2e@istockphoto.com,'19.130.175.245',1994-10-05 22:50:49
+green,88,Andrea,awelch2f@abc.net.au,'94.155.233.96',2002-04-26 08:41:44
+green,89,Louis,lwagner2g@miitbeian.gov.cn,'26.217.34.111',2003-08-25 07:56:39
+green,90,Jane,jsims2h@seesaa.net,'43.4.220.135',1987-03-20 20:39:04
+green,91,Larry,lgrant2i@si.edu,'97.126.79.34',2000-09-07 20:26:19
+green,92,Louis,ldean2j@prnewswire.com,'37.148.40.127',2011-09-16 20:12:14
+green,93,Jennifer,jcampbell2k@xing.com,'38.106.254.142',1988-07-15 05:06:49
+green,94,Wayne,wcunningham2l@google.com.hk,'223.28.26.187',2009-12-15 06:16:54
+green,95,Lori,lstevens2m@icq.com,'181.250.181.58',1984-10-28 03:29:19
+green,96,Judy,jsimpson2n@marriott.com,'180.121.239.219',1986-02-07 15:18:10
+green,97,Phillip,phoward2o@usa.gov,'255.247.0.175',2002-12-26 08:44:45
+green,98,Gloria,gwalker2p@usa.gov,'156.140.7.128',1997-10-04 07:58:58
+green,99,Paul,pjohnson2q@umn.edu,'183.59.198.197',1991-11-14 12:33:55
+green,100,Frank,fgreene2r@blogspot.com,'150.143.68.121',2010-06-12 23:55:39
diff --git a/tests/integration/upload_file_tests/macros/upload_file.sql b/tests/integration/upload_file_tests/macros/upload_file.sql
new file mode 100644
index 000000000..5436170f3
--- /dev/null
+++ b/tests/integration/upload_file_tests/macros/upload_file.sql
@@ -0,0 +1,3 @@
+{% macro upload_file(local_file_path, database, table_schema, table_name) %}
+  {% do adapter.upload_file(local_file_path, database, table_schema, table_name, kwargs=kwargs) %}
+{% endmacro %}
\ No newline at end of file
diff --git a/tests/integration/upload_file_tests/ndjson/source.ndjson b/tests/integration/upload_file_tests/ndjson/source.ndjson
new file mode 100644
index 000000000..365d8330b
--- /dev/null
+++ b/tests/integration/upload_file_tests/ndjson/source.ndjson
@@ -0,0 +1,100 @@
+{"favorite_color":"blue","id":1,"first_name":"Larry","email":"lking0@miitbeian.gov.cn","ip_address":"'69.135.206.194'","updated_at":"2008-09-12 19:08:31"}
+{"favorite_color":"blue","id":2,"first_name":"Larry","email":"lperkins1@toplist.cz","ip_address":"'64.210.133.162'","updated_at":"1978-05-09 04:15:14"}
+{"favorite_color":"blue","id":3,"first_name":"Anna","email":"amontgomery2@miitbeian.gov.cn","ip_address":"'168.104.64.114'","updated_at":"2011-10-16 04:07:57"}
+{"favorite_color":"blue","id":4,"first_name":"Sandra","email":"sgeorge3@livejournal.com","ip_address":"'229.235.252.98'","updated_at":"1973-07-19 10:52:43"}
+{"favorite_color":"blue","id":5,"first_name":"Fred","email":"fwoods4@google.cn","ip_address":"'78.229.170.124'","updated_at":"2012-09-30 16:38:29"}
+{"favorite_color":"blue","id":6,"first_name":"Stephen","email":"shanson5@livejournal.com","ip_address":"'182.227.157.105'","updated_at":"1995-11-07 21:40:50"}
+{"favorite_color":"blue","id":7,"first_name":"William","email":"wmartinez6@upenn.edu","ip_address":"'135.139.249.50'","updated_at":"1982-09-05 03:11:59"}
+{"favorite_color":"blue","id":8,"first_name":"Jessica","email":"jlong7@hao123.com","ip_address":"'203.62.178.210'","updated_at":"1991-10-16 11:03:15"}
+{"favorite_color":"blue","id":9,"first_name":"Douglas","email":"dwhite8@tamu.edu","ip_address":"'178.187.247.1'","updated_at":"1979-10-01 09:49:48"}
+{"favorite_color":"blue","id":10,"first_name":"Lisa","email":"lcoleman9@nydailynews.com","ip_address":"'168.234.128.249'","updated_at":"2011-05-26 07:45:49"}
+{"favorite_color":"blue","id":11,"first_name":"Ralph","email":"rfieldsa@home.pl","ip_address":"'55.152.163.149'","updated_at":"1972-11-18 19:06:11"}
+{"favorite_color":"blue","id":12,"first_name":"Louise","email":"lnicholsb@samsung.com","ip_address":"'141.116.153.154'","updated_at":"2014-11-25 20:56:14"}
+{"favorite_color":"blue","id":13,"first_name":"Clarence","email":"cduncanc@sfgate.com","ip_address":"'81.171.31.133'","updated_at":"2011-11-17 07:02:36"}
+{"favorite_color":"blue","id":14,"first_name":"Daniel","email":"dfranklind@omniture.com","ip_address":"'8.204.211.37'","updated_at":"1980-09-13 00:09:04"}
+{"favorite_color":"blue","id":15,"first_name":"Katherine","email":"klanee@auda.org.au","ip_address":"'176.96.134.59'","updated_at":"1997-08-22 19:36:56"}
+{"favorite_color":"blue","id":16,"first_name":"Billy","email":"bwardf@wikia.com","ip_address":"'214.108.78.85'","updated_at":"2003-10-19 02:14:47"}
+{"favorite_color":"blue","id":17,"first_name":"Annie","email":"agarzag@ocn.ne.jp","ip_address":"'190.108.42.70'","updated_at":"1988-10-28 15:12:35"}
+{"favorite_color":"blue","id":18,"first_name":"Shirley","email":"scolemanh@fastcompany.com","ip_address":"'109.251.164.84'","updated_at":"1988-08-24 10:50:57"}
+{"favorite_color":"blue","id":19,"first_name":"Roger","email":"rfrazieri@scribd.com","ip_address":"'38.145.218.108'","updated_at":"1985-12-31 15:17:15"}
+{"favorite_color":"blue","id":20,"first_name":"Lillian","email":"lstanleyj@goodreads.com","ip_address":"'47.57.236.17'","updated_at":"1970-06-08 02:09:05"}
+{"favorite_color":"blue","id":21,"first_name":"Aaron","email":"arodriguezk@nps.gov","ip_address":"'205.245.118.221'","updated_at":"1985-10-11 23:07:49"}
+{"favorite_color":"blue","id":22,"first_name":"Patrick","email":"pparkerl@techcrunch.com","ip_address":"'19.8.100.182'","updated_at":"2006-03-29 12:53:56"}
+{"favorite_color":"blue","id":23,"first_name":"Phillip","email":"pmorenom@intel.com","ip_address":"'41.38.254.103'","updated_at":"2011-11-07 15:35:43"}
+{"favorite_color":"blue","id":24,"first_name":"Henry","email":"hgarcian@newsvine.com","ip_address":"'1.191.216.252'","updated_at":"2008-08-28 08:30:44"}
+{"favorite_color":"blue","id":25,"first_name":"Irene","email":"iturnero@opera.com","ip_address":"'50.17.60.190'","updated_at":"1994-04-01 07:15:02"}
+{"favorite_color":"blue","id":26,"first_name":"Andrew","email":"adunnp@pen.io","ip_address":"'123.52.253.176'","updated_at":"2000-11-01 06:03:25"}
+{"favorite_color":"blue","id":27,"first_name":"David","email":"dgutierrezq@wp.com","ip_address":"'238.23.203.42'","updated_at":"1988-01-25 07:29:18"}
+{"favorite_color":"blue","id":28,"first_name":"Henry","email":"hsanchezr@cyberchimps.com","ip_address":"'248.102.2.185'","updated_at":"1983-01-01 13:36:37"}
+{"favorite_color":"blue","id":29,"first_name":"Evelyn","email":"epetersons@gizmodo.com","ip_address":"'32.80.46.119'","updated_at":"1979-07-16 17:24:12"}
+{"favorite_color":"blue","id":30,"first_name":"Tammy","email":"tmitchellt@purevolume.com","ip_address":"'249.246.167.88'","updated_at":"2001-04-03 10:00:23"}
+{"favorite_color":"blue","id":31,"first_name":"Jacqueline","email":"jlittleu@domainmarket.com","ip_address":"'127.181.97.47'","updated_at":"1986-02-11 21:35:50"}
+{"favorite_color":"blue","id":32,"first_name":"Earl","email":"eortizv@opera.com","ip_address":"'166.47.248.240'","updated_at":"1996-07-06 08:16:27"}
+{"favorite_color":"blue","id":33,"first_name":"Juan","email":"jgordonw@sciencedirect.com","ip_address":"'71.77.2.200'","updated_at":"1987-01-31 03:46:44"}
+{"favorite_color":"blue","id":34,"first_name":"Diane","email":"dhowellx@nyu.edu","ip_address":"'140.94.133.12'","updated_at":"1994-06-11 02:30:05"}
+{"favorite_color":"blue","id":35,"first_name":"Randy","email":"rkennedyy@microsoft.com","ip_address":"'73.255.34.196'","updated_at":"2005-05-26 20:28:39"}
+{"favorite_color":"blue","id":36,"first_name":"Janice","email":"jriveraz@time.com","ip_address":"'22.214.227.32'","updated_at":"1990-02-09 04:16:52"}
+{"favorite_color":"blue","id":37,"first_name":"Laura","email":"lperry10@diigo.com","ip_address":"'159.148.145.73'","updated_at":"2015-03-17 05:59:25"}
+{"favorite_color":"blue","id":38,"first_name":"Gary","email":"gray11@statcounter.com","ip_address":"'40.193.124.56'","updated_at":"1970-01-27 10:04:51"}
+{"favorite_color":"blue","id":39,"first_name":"Jesse","email":"jmcdonald12@typepad.com","ip_address":"'31.7.86.103'","updated_at":"2009-03-14 08:14:29"}
+{"favorite_color":"blue","id":40,"first_name":"Sandra","email":"sgonzalez13@goodreads.com","ip_address":"'223.80.168.239'","updated_at":"1993-05-21 14:08:54"}
+{"favorite_color":"blue","id":41,"first_name":"Scott","email":"smoore14@archive.org","ip_address":"'38.238.46.83'","updated_at":"1980-08-30 11:16:56"}
+{"favorite_color":"blue","id":42,"first_name":"Phillip","email":"pevans15@cisco.com","ip_address":"'158.234.59.34'","updated_at":"2011-12-15 23:26:31"}
+{"favorite_color":"blue","id":43,"first_name":"Steven","email":"sriley16@google.ca","ip_address":"'90.247.57.68'","updated_at":"2011-10-29 19:03:28"}
+{"favorite_color":"blue","id":44,"first_name":"Deborah","email":"dbrown17@hexun.com","ip_address":"'179.125.143.240'","updated_at":"1995-04-10 14:36:07"}
+{"favorite_color":"blue","id":45,"first_name":"Lori","email":"lross18@ow.ly","ip_address":"'64.80.162.180'","updated_at":"1980-12-27 16:49:15"}
+{"favorite_color":"blue","id":46,"first_name":"Sean","email":"sjackson19@tumblr.com","ip_address":"'240.116.183.69'","updated_at":"1988-06-12 21:24:45"}
+{"favorite_color":"blue","id":47,"first_name":"Terry","email":"tbarnes1a@163.com","ip_address":"'118.38.213.137'","updated_at":"1997-09-22 16:43:19"}
+{"favorite_color":"blue","id":48,"first_name":"Dorothy","email":"dross1b@ebay.com","ip_address":"'116.81.76.49'","updated_at":"2005-02-28 13:33:24"}
+{"favorite_color":"blue","id":49,"first_name":"Samuel","email":"swashington1c@house.gov","ip_address":"'38.191.253.40'","updated_at":"1989-01-19 21:15:48"}
+{"favorite_color":"blue","id":50,"first_name":"Ralph","email":"rcarter1d@tinyurl.com","ip_address":"'104.84.60.174'","updated_at":"2007-08-11 10:21:49"}
+{"favorite_color":"green","id":51,"first_name":"Wayne","email":"whudson1e@princeton.edu","ip_address":"'90.61.24.102'","updated_at":"1983-07-03 16:58:12"}
+{"favorite_color":"green","id":52,"first_name":"Rose","email":"rjames1f@plala.or.jp","ip_address":"'240.83.81.10'","updated_at":"1995-06-08 11:46:23"}
+{"favorite_color":"green","id":53,"first_name":"Louise","email":"lcox1g@theglobeandmail.com","ip_address":"'105.11.82.145'","updated_at":"2016-09-19 14:45:51"}
+{"favorite_color":"green","id":54,"first_name":"Kenneth","email":"kjohnson1h@independent.co.uk","ip_address":"'139.5.45.94'","updated_at":"1976-08-17 11:26:19"}
+{"favorite_color":"green","id":55,"first_name":"Donna","email":"dbrown1i@amazon.co.uk","ip_address":"'19.45.169.45'","updated_at":"2006-05-27 16:51:40"}
+{"favorite_color":"green","id":56,"first_name":"Johnny","email":"jvasquez1j@trellian.com","ip_address":"'118.202.238.23'","updated_at":"1975-11-17 08:42:32"}
+{"favorite_color":"green","id":57,"first_name":"Patrick","email":"pramirez1k@tamu.edu","ip_address":"'231.25.153.198'","updated_at":"1997-08-06 11:51:09"}
+{"favorite_color":"green","id":58,"first_name":"Helen","email":"hlarson1l@prweb.com","ip_address":"'8.40.21.39'","updated_at":"1993-08-04 19:53:40"}
+{"favorite_color":"green","id":59,"first_name":"Patricia","email":"pspencer1m@gmpg.org","ip_address":"'212.198.40.15'","updated_at":"1977-08-03 16:37:27"}
+{"favorite_color":"green","id":60,"first_name":"Joseph","email":"jspencer1n@marriott.com","ip_address":"'13.15.63.238'","updated_at":"2005-07-23 20:22:06"}
+{"favorite_color":"green","id":61,"first_name":"Phillip","email":"pschmidt1o@blogtalkradio.com","ip_address":"'177.98.201.190'","updated_at":"1976-05-19 21:47:44"}
+{"favorite_color":"green","id":62,"first_name":"Joan","email":"jwebb1p@google.ru","ip_address":"'105.229.170.71'","updated_at":"1972-09-07 17:53:47"}
+{"favorite_color":"green","id":63,"first_name":"Phyllis","email":"pkennedy1q@imgur.com","ip_address":"'35.145.8.244'","updated_at":"2000-01-01 22:33:37"}
+{"favorite_color":"green","id":64,"first_name":"Katherine","email":"khunter1r@smh.com.au","ip_address":"'248.168.205.32'","updated_at":"1991-01-09 06:40:24"}
+{"favorite_color":"green","id":65,"first_name":"Laura","email":"lvasquez1s@wiley.com","ip_address":"'128.129.115.152'","updated_at":"1997-10-23 12:04:56"}
+{"favorite_color":"green","id":66,"first_name":"Juan","email":"jdunn1t@state.gov","ip_address":"'44.228.124.51'","updated_at":"2004-11-10 05:07:35"}
+{"favorite_color":"green","id":67,"first_name":"Judith","email":"jholmes1u@wiley.com","ip_address":"'40.227.179.115'","updated_at":"1977-08-02 17:01:45"}
+{"favorite_color":"green","id":68,"first_name":"Beverly","email":"bbaker1v@wufoo.com","ip_address":"'208.34.84.59'","updated_at":"2016-03-06 20:07:23"}
+{"favorite_color":"green","id":69,"first_name":"Lawrence","email":"lcarr1w@flickr.com","ip_address":"'59.158.212.223'","updated_at":"1988-09-13 06:07:21"}
+{"favorite_color":"green","id":70,"first_name":"Gloria","email":"gwilliams1x@mtv.com","ip_address":"'245.231.88.33'","updated_at":"1995-03-18 22:32:46"}
+{"favorite_color":"green","id":71,"first_name":"Steven","email":"ssims1y@cbslocal.com","ip_address":"'104.50.58.255'","updated_at":"2001-08-05 21:26:20"}
+{"favorite_color":"green","id":72,"first_name":"Betty","email":"bmills1z@arstechnica.com","ip_address":"'103.177.214.220'","updated_at":"1981-12-14 21:26:54"}
+{"favorite_color":"green","id":73,"first_name":"Mildred","email":"mfuller20@prnewswire.com","ip_address":"'151.158.8.130'","updated_at":"2000-04-19 10:13:55"}
+{"favorite_color":"green","id":74,"first_name":"Donald","email":"dday21@icq.com","ip_address":"'9.178.102.255'","updated_at":"1972-12-03 00:58:24"}
+{"favorite_color":"green","id":75,"first_name":"Eric","email":"ethomas22@addtoany.com","ip_address":"'85.2.241.227'","updated_at":"1992-11-01 05:59:30"}
+{"favorite_color":"green","id":76,"first_name":"Joyce","email":"jarmstrong23@sitemeter.com","ip_address":"'169.224.20.36'","updated_at":"1985-10-24 06:50:01"}
+{"favorite_color":"green","id":77,"first_name":"Maria","email":"mmartinez24@amazonaws.com","ip_address":"'143.189.167.135'","updated_at":"2005-10-05 05:17:42"}
+{"favorite_color":"green","id":78,"first_name":"Harry","email":"hburton25@youtube.com","ip_address":"'156.47.176.237'","updated_at":"1978-03-26 05:53:33"}
+{"favorite_color":"green","id":79,"first_name":"Kevin","email":"klawrence26@hao123.com","ip_address":"'79.136.183.83'","updated_at":"1994-10-12 04:38:52"}
+{"favorite_color":"green","id":80,"first_name":"David","email":"dhall27@prweb.com","ip_address":"'133.149.172.153'","updated_at":"1976-12-15 16:24:24"}
+{"favorite_color":"green","id":81,"first_name":"Kathy","email":"kperry28@twitter.com","ip_address":"'229.242.72.228'","updated_at":"1979-03-04 02:58:56"}
+{"favorite_color":"green","id":82,"first_name":"Adam","email":"aprice29@elegantthemes.com","ip_address":"'13.145.21.10'","updated_at":"1982-11-07 11:46:59"}
+{"favorite_color":"green","id":83,"first_name":"Brandon","email":"bgriffin2a@va.gov","ip_address":"'73.249.128.212'","updated_at":"2013-10-30 05:30:36"}
+{"favorite_color":"green","id":84,"first_name":"Henry","email":"hnguyen2b@discovery.com","ip_address":"'211.36.214.242'","updated_at":"1985-01-09 06:37:27"}
+{"favorite_color":"green","id":85,"first_name":"Eric","email":"esanchez2c@edublogs.org","ip_address":"'191.166.188.251'","updated_at":"2004-05-01 23:21:42"}
+{"favorite_color":"green","id":86,"first_name":"Jason","email":"jlee2d@jimdo.com","ip_address":"'193.92.16.182'","updated_at":"1973-01-08 09:05:39"}
+{"favorite_color":"green","id":87,"first_name":"Diana","email":"drichards2e@istockphoto.com","ip_address":"'19.130.175.245'","updated_at":"1994-10-05 22:50:49"}
+{"favorite_color":"green","id":88,"first_name":"Andrea","email":"awelch2f@abc.net.au","ip_address":"'94.155.233.96'","updated_at":"2002-04-26 08:41:44"}
+{"favorite_color":"green","id":89,"first_name":"Louis","email":"lwagner2g@miitbeian.gov.cn","ip_address":"'26.217.34.111'","updated_at":"2003-08-25 07:56:39"}
+{"favorite_color":"green","id":90,"first_name":"Jane","email":"jsims2h@seesaa.net","ip_address":"'43.4.220.135'","updated_at":"1987-03-20 20:39:04"}
+{"favorite_color":"green","id":91,"first_name":"Larry","email":"lgrant2i@si.edu","ip_address":"'97.126.79.34'","updated_at":"2000-09-07 20:26:19"}
+{"favorite_color":"green","id":92,"first_name":"Louis","email":"ldean2j@prnewswire.com","ip_address":"'37.148.40.127'","updated_at":"2011-09-16 20:12:14"}
+{"favorite_color":"green","id":93,"first_name":"Jennifer","email":"jcampbell2k@xing.com","ip_address":"'38.106.254.142'","updated_at":"1988-07-15 05:06:49"}
+{"favorite_color":"green","id":94,"first_name":"Wayne","email":"wcunningham2l@google.com.hk","ip_address":"'223.28.26.187'","updated_at":"2009-12-15 06:16:54"}
+{"favorite_color":"green","id":95,"first_name":"Lori","email":"lstevens2m@icq.com","ip_address":"'181.250.181.58'","updated_at":"1984-10-28 03:29:19"}
+{"favorite_color":"green","id":96,"first_name":"Judy","email":"jsimpson2n@marriott.com","ip_address":"'180.121.239.219'","updated_at":"1986-02-07 15:18:10"}
+{"favorite_color":"green","id":97,"first_name":"Phillip","email":"phoward2o@usa.gov","ip_address":"'255.247.0.175'","updated_at":"2002-12-26 08:44:45"}
+{"favorite_color":"green","id":98,"first_name":"Gloria","email":"gwalker2p@usa.gov","ip_address":"'156.140.7.128'","updated_at":"1997-10-04 07:58:58"}
+{"favorite_color":"green","id":99,"first_name":"Paul","email":"pjohnson2q@umn.edu","ip_address":"'183.59.198.197'","updated_at":"1991-11-14 12:33:55"}
+{"favorite_color":"green","id":100,"first_name":"Frank","email":"fgreene2r@blogspot.com","ip_address":"'150.143.68.121'","updated_at":"2010-06-12 23:55:39"}
\ No newline at end of file
diff --git a/tests/integration/upload_file_tests/parquet/source.parquet b/tests/integration/upload_file_tests/parquet/source.parquet
new file mode 100644
index 0000000000000000000000000000000000000000..3022faa675c8a6a7957f3dd5a239da3b8858f1b4
GIT binary patch
literal 10468
zcmcgy3wRu5xt_n>ZFkG2-Owc6v;nfAxnZ*G%B
z&dg@Bfl8wksul$wL_h`V14yx06po5`v{j(otRU#2iZ=uWK}3$8gOAEN?`(QWX?>19
z2hG!WF8}5GzW=-Z|9soE!XR7jDz7Q8>&MM?Cc~8SE{0)JYEE|Jo;GD!FK5aFow}^&n
znC0^-9wQZ>Ar*I!y1qKkDYPn^96`$GG2rUl80U7tSN`<{{Y3#F;_VAugz*w)o_C>(
znP1{zT!0%`04xMNfEQQi;z?r~OpbRJnDu87`0H_3(11o?k;4I*5
z;2fYD2m&?0xxh-G7N`U2fd-%vI1hLq&;&FCA>e!<46pzPM1Uv|16qJMkN}dvD&YM<
zD{ujDA#f4E0|L+nv;!SLC(s3S13f@5&ivS{*uX
zibq{(;fftQw(<>u#k;Ob9n2)ozHXP)ws$0S)9$hFygGH^?bl^G&bxWd-8YOsGXCIY
z_uZs+ec_1ywa@5>Ui;Gr4&0vYXZ>PWY^ZTZJ
zU{m+j8CB)3qb@p6k!IcUo+TCccK~B$v_AXR8Rai|`^#@d(7vRF`>!~i8l$`1BGG%EYIkU``vl&?@X9R{7RaK~lr#{)T6p`Yo!^ovo
zYLRHTU$Lk|aFD9m47s?Y-^eMJ>_JPrN=;c8Wm3`8L3Kq|y=Yub?Tl4IXM5?wbQ^sckCa1UbjGHbbhJEIG&5
zBkQKS)z=F~?n0S>CCGVFJ^eJ-c!8^%dZ&Z+?&bdVqG8*l*|*-7CmYuU
zI^>jL(hRxO*N?mcLhEI(@5r>aL*WgY5ku49ziSI}(9S@#vbK7;CHBCs}7fH5p`M_FG^
zQZT(d&?ZmHCiHlrzn|t!x%r^K8`({%{f)0pwr%%4Wdn)|wbZVML^@R^^0coLY9u#$
z`iw$j&^16y5}qDvHrDu7%acm&mBJZJfy7*b)GL;?nNWz~uJZIz%j*qpcOPJc;1li-
zV#PV7b32;#&he@~tVJ1v$@)DI%e^7Imgb5J?!xqm(*z>&_QIoX6>tl%t`6t`R|6mD6nE(05NLebMvh!WDW$-W~F7rOv@2o
zRVL+eBWJ>bA<@uE(KnVi49SY}X~RgXGHAQmK!)m;p~o&{8FF=hUZbWBPoHk#b6J=u
zBuhEcyJK86^mLrhP=nBMES~7-erFe^KR!&co=ANY>DoQzBq`JskV`*y3WLdDiLmG2@
z0s$!ThNdfa&UCNXfV{*^s8p9_p5`PP!kR-gNA7A$<*6x+@p)xJp`??1kEdyBPSZ4R
zhyN$2G030}`y!Pg1blg(0d0+GervX~J~tRzDQ$iZ+{GmvczjaPJA
zu6-m}Q=NfKB23H&S9>NEz5Gq`*J@oh4c>sI-7hdD0^0;l=vf|C2q^{`ic0C64cSb2
zdK;h55|zBspRu5+j67}fVj(4)Vn)%j?uRy1Ks9n!wq+BBwfM9$tr?P08t=C?#m0D5
zwfSt$lqU@}r@5~u8(POz#kN&B$4iDr6d?NTvScTLUo8XOnNG%AZt=2ddC!kL@$=f+CrMkZn
zBu#dTn!0SUlxJHCceyX}r+7IcYdYz
zKJsJVI1)&3JCmzy5>|&(DT>fzC;5DC%y8dED#9u>ZL)cOOvMSny^FMa(s?D=S*2O*
z6tCHn#2~jtaXP_!ffrMjYKV_-JjgetG#oN4cABTAHS@5ut?%4Us+zR1oT|zu7ls;U
z_Pg@R-rJbP2hXpTXn|vSMchVG^y*gZ&j=xl<9I4bwn6tA%<`E4NNJXhJuA&ccneum
zo4KAO=!Y5YnY&zEbl@z8J}wLK)=VmA!eO`=UodiZF17PkqWh8M@@DW&57)vUd7LRF
zx0Ge5s&es7XOaI2Pat0)`6CPG75myC`Se-}7ZPPI$>UUzrn(J3(d0W<5vV9_Dq~}c
z&QX4n(hb78hK!!h6=a=D@zRXjgV3yfj;vthd#`EWL>{T%v7IGc!}x=)YE_mwi62+A
zy;m|T$USSpC4)?BaWan$$q*;98N;sqdGLuIic}zGxG|olL}Xms$;8>D-gcKl4-JvDOnO7dQ51MV&pg+|*d{!(iQxRdsJI
zcMdIom(-^{$y|4K`AcQTPcplf^gBm``j%vfjl@D+xFy6UqxIgha5TiRVKhcUYzv2X
zwj~i_(b^IXv200$pW~7tj$Gi6aiL_w+u(^OLS$Jr8^;_jT3GeYy`8S$xtlpEvw?fkx&a4Vo9fu4c9MmU(f_1Y$A>km{GEJC0KG1@Z%EXB5!Ln
z76UDeZ-GqFWJzN*%0d=M9797a>RnfzKtr4jMOg9+SN&ol`9L5{{@_DX+*=!F<1L{i
z#)D=o>0KJ;SP%~CJfI}M=$_3AG#ap~dh9iA}2v!maMN76I|B!3|y&xR(_BTYhP$C?P
zLJGE74L*RurT&(9D3S2?hgnDcP)af$ipHCLzLu6yH2C%4=lO7biDP!?Dvl+9H2ZUZ
zG#pAsgS%Z&{Z)ZD_(9*n7nx+~JuWU3+#z%^k#(Q&#o#IAi#{wg-n@v3lHZeFAJp06
z-PRBRwNL_rgd^S?JscMyPxp4BxbRD19g`q0q14dlVq;`!@EuPm67|-)lHtyse(10z
z(RhE9je{}={YAk#TJlWrPfVC5J&?co3M}d-VV6Gve@S|;3I3Yo%UIYPMop2G0W1)P
zjSl2okwoJQ9(Y6PR(iE75v_fJiC5m@Yk{JLr2`BxU5N-8$n`K`;Sx_M!3nz?SI{fV
zVj-+1S<>TTlVq_A$Bc!QcQKr>gnWS&JXjR@qtM4E6ZOuwe0?GWJ#jE*$=Av_7Dh|}u
z#{~a}$9x=|i>zYUor`*J^hFVT?tr6K*842k>wy((zft-b4;M}d29pTB>8%qs3-cLv
z=AnTydMo*>FOh&iOM};X;78;SGfCqBOK)FAWElfQ4
zSr^L=Eb&0F;13V(X--tZty!)m6qonaCR~yD&MG>^aMyl}u4ck}&h;R+Kl#SYJ0949
z+*DFO^DsoX$OmVMZF_}2I37$^=homCW<<<+sNzH
z6^|~f+r7T->IZdi8G%<(U$*0^tInUf&LeMxBsn|=P?H^cwYSB1Yr(*yYkho+faF3xG8aH
z8DaRB3-cQ9z`QS3-|_W9hS~9{ww3?pL1E?cy2G!03B|0QwtdSYDC~8Wup2AqGY6in
zd6Ibo_p|S+>#Lf_@Lhecowp41Z&>jf`y}Xn;<|*2xaEKRL*I`d#=Je3-XHXXe(B?f
z>l}K2{NsP_*a*2^)yCfVD(2T-`NCiA`OJ>}kBV0=z`OY8pRUYd-l5w+b)XdT{&^o+
zID`HNFI|4sVZ87A!QHRD4f)x}^{(H5pL_Y2Kk)D1e@0h_=WP@hbKKq)0_Z4-FXT$y
z%z??jOfSW7{+H_EB{KMWcTPrbL&K^+Ubc_}y-($zZhQc8?Kpq0eIMkUe^tZC4?yqv
zYhIM!5BkBK?fFu?FMB`z_6pGd(!%vkzsKFX?N`s$fKMZNL5X92=?kt8e+cyUoO{pP
zM=*Zp>dSu83jR&n7bD+}!S{yqIr?#|C-Bm*zyAn1>MLC4(l+o!Xz*YIEVyzwgZ_uqFlu=_FpyWXUH#PO3$xd(37h*!z~_)AWF!oB3)M@}2wUr_1gI-=|(fwG#60Icw;f)1XTy
z+~!`;^Qh_)g~{X5u@jZ*MY*hml8&^6+Q%!_Utd!9wph1&7!~VN<@_7cpC*UQtGpt{1Qy_<
z6Gz-~A5-qDo%ik^Z)%zTDy+QRg)S7$XUEJb<{j&Irp}&}=eEqre8-v{LUYzsTh_n(
z%&I1^U>N*NR9RK2zPG<6{gcc7?{Sf{?w|YRQTI>Ymg@dBo?CJI6nv%5Q}NiAGKQ&a
z3M@b|QleI1m*?^tr-n1-{PLL7bgO1n^(B|rpa?ACsRp0PX}PAUy2dF(qd`Y)0&ggs
znKoX-#b+anM>ahBA{Sq_Z${57#Vtkh@=?Mk(<#<#izV(8v}Xm|;?&>JSd&9t#E@oX
z$Z6Ta@eU(3E{nD!n75l=Dv@*30&4GACW&fJX~x|$O;+vf;#vJ^EZ|Lp3}I!U6_
z^#8x4bF{a;2|^y#LX9Gw%5%2uU3^f`ZHX4`aFPy=H_uVQDNIiMj&Lfoll$Jq^dD3G
zDSV4nfKyo>Z+>sNoj5U`%JSsC_m=O8&E{0TC-=RJud_$l3Ze>Lr=MiiV?8a=OLS|q
z)BkSs7NZM45!i-Vbb3`unba&eKA$b18foO6S*WvFilIBL@lZGvu7U5>9F<~D9~;6K
zN6mI7;1dQ1FCJbsySoMUEji!$i>JE;L0AVkf2#!{Dzv2qw55gibg`|i?WF#^)7W{u
zztedx;^?Sif3G0q1}NCfv|gErx*Ex&0vHnC}B0MBX6Z;SV8YHqZ9C_2KenN0Nz
z*(rTMxmamS$%Q_XVr=iYk?z&|GAT_`dzE~;SBqs*!yBD3Ica#x>TeemF*2AD^>t_y
z@<-n%#rEEo-j2@vKs)B>Yr>-=g3~`WHWapc+xx~-+!UJ{j*U4yMn^KLSitx$8}gh=
ze^gV|RQKRmO6#&m*JE6Fj7@dt9XdzXvt~@{QS-&V_C7YHsh~f2OkOrM(iR(u493K6
zM{Yq$aedoHhu4H>*Tbq(cNcUrm~qyMb*m|D&E#k|^y0|W-ZwPVxyin`y)XGL*{7ZH
z!x2k4E?0Y>wrON^QqZ!;VZF6-U1*eaIGj)VXV(+@xjIO_6o8{li1yO~Zp3
zsk`&IoyOMZowd#3uTIRae>6KfJT)=G9n-s@(BWA+Vs{sOqicGu?qIL*n(?E2PL(5@
z(${fo+KX$pN1;>94{thdv#Df%yW?{wc)_*>_|Gxh{w-Z)MflJ$8H+x48rh`bSauVa
zoPZxVbl{(CULKmGM=go})G#zI{Uh=Xvl%*^FY5IaKWvAu40mNyv-I9`Uxb~lR1e~%
z=OiD?id{+Qb+9lxvS#{k+Hj<83_dU+YD3eAEk_^!D*0Hk+wr|2_*B=?7(3O+q>(l!
z=2XN)=BPbRCtq=l)U7&k|2J$}XMZl-`N!DQ^DJb&6c8W
zyYUN?)>V)lKZTy;;jV(HB_p%(II6<;vMDZF%pGeA(R}~7pg8f=dm>i3(UHDsI-JZE
zV|%iAXz(vCdcY998P813m(#R$ae
z7{{b!6qobS+txN%q*f$4D`oNB*|UVv*_gmnTiO|K{9XJ;=f*I0sA*^FfOByO2^oj7
zAUc;h{IE1ZKt94XEp!z3HPKn`(Z8ZN&L+6g>4%=N)Z#w4?gWv}uG8x?3L0&v=4t9a7PHyr;u)a&=q6uX-2E$6e3riJAM^di46_#hzwv(o8jjey

literal 0
HcmV?d00001

diff --git a/tests/integration/upload_file_tests/test_upload_file.py b/tests/integration/upload_file_tests/test_upload_file.py
new file mode 100644
index 000000000..a879fe2bf
--- /dev/null
+++ b/tests/integration/upload_file_tests/test_upload_file.py
@@ -0,0 +1,93 @@
+from tests.integration.base import DBTIntegrationTest, use_profile
+import datetime
+import yaml
+
+
+class TestUploadFile(DBTIntegrationTest):
+    @property
+    def schema(self):
+        return "upload_file"
+
+    @property
+    def models(self):
+        return "models"
+        
+    def perform_uploaded_table_checks(self, table_schema, table_name):
+        # Test the data types of the created table
+        query = f"""
+            select
+                column_name,
+                data_type
+            from `{table_schema}.INFORMATION_SCHEMA.COLUMNS`
+            where
+                table_name = '{table_name}'
+            order by column_name
+        """
+        results = self.run_sql(query, fetch='all')
+        self.assertTrue([row[1] for row in results] == ['STRING', 'STRING', 'STRING', 'INT64', 'STRING', 'TIMESTAMP'], 'The table should consist of columns of types: STRING, STRING, STRING, INT64, STRING, TIMESTAMP')
+        
+        # Test the values of the created table
+        query = f"""
+            select
+                count(*) row_count,
+                count(distinct id) as num_distinct_ids,
+                max(updated_at) as max_updated_at
+            from `{table_schema}.{table_name}`
+        """
+        results = self.run_sql(query, fetch='all')
+        self.assertTrue(results[0][0] == 100, 'There should be 100 rows in this table')
+        self.assertTrue(results[0][1] == 100, 'There should be 100 distinct id values in this table')
+        self.assertTrue(results[0][2] == datetime.datetime(2016, 9, 19, 14, 45, 51, tzinfo=datetime.timezone.utc), 'Maximum updated_at value should be 2016-09-19 14:45:51')
+
+    @use_profile('bigquery')
+    def test_bigquery_upload_file_csv(self):
+        # Create a table from an uploaded CSV file
+        upload_args = yaml.safe_dump({
+            'local_file_path': './csv/source.csv',
+            'database': self.default_database,
+            'table_schema': self.unique_schema(),
+            'table_name': 'TestUploadFileCSV',
+            'skip_leading_rows': 1,
+            'autodetect': True,
+            'write_disposition': 'WRITE_TRUNCATE'
+        })
+        result = self.run_dbt(['run-operation', 'upload_file', '--args', upload_args])
+        self.assertTrue(result.success)
+
+        # Check uploaded table contains expected values and schema
+        self.perform_uploaded_table_checks(self.unique_schema(), 'TestUploadFileCSV')
+
+    @use_profile('bigquery')
+    def test_bigquery_upload_file_ndjson(self):
+        # Create a table from an uploaded NDJSON file
+        upload_args = yaml.safe_dump({
+            'local_file_path': './ndjson/source.ndjson',
+            'database': self.default_database,
+            'table_schema': self.unique_schema(),
+            'table_name': 'TestUploadFileNDJSON',
+            'autodetect': True,
+            'source_format': 'NEWLINE_DELIMITED_JSON',
+            'write_disposition': 'WRITE_TRUNCATE'
+        })
+        result = self.run_dbt(['run-operation', 'upload_file', '--args', upload_args])
+        self.assertTrue(result.success)
+
+        # Check uploaded table contains expected values and schema
+        self.perform_uploaded_table_checks(self.unique_schema(), 'TestUploadFileNDJSON')
+
+    @use_profile('bigquery')
+    def test_bigquery_upload_file_parquet(self):
+        # Create a table from an uploaded parquet file
+        upload_args = yaml.safe_dump({
+            'local_file_path': './parquet/source.parquet',
+            'database': self.default_database,
+            'table_schema': self.unique_schema(),
+            'table_name': 'TestUploadFileParquet',
+            'source_format': 'PARQUET',
+            'write_disposition': 'WRITE_TRUNCATE'
+        })
+        result = self.run_dbt(['run-operation', 'upload_file', '--args', upload_args])
+        self.assertTrue(result.success)
+
+        # Check uploaded table contains expected values and schema
+        self.perform_uploaded_table_checks(self.unique_schema(), 'TestUploadFileParquet')

From 072050eb5f5c7d6e464c0b4ae26c3745af505690 Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Wed, 23 Mar 2022 17:13:05 -0400
Subject: [PATCH 433/860] Bumping version to 1.1.0b1 (#145)

* Bumping version to 1.1.0b1

* changelog update

Co-authored-by: Github Build Bot 
Co-authored-by: Matthew McKnight 
---
 .bumpversion.cfg                     |  2 +-
 CHANGELOG.md                         | 11 +++++++++--
 dbt/adapters/bigquery/__version__.py |  2 +-
 setup.py                             |  2 +-
 4 files changed, 12 insertions(+), 5 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 8a8ca719c..b0f5f6aab 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.0.0
+current_version = 1.1.0b1
 parse = (?P\d+)
 	\.(?P\d+)
 	\.(?P\d+)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index fb2d5960b..536e5a312 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,8 +1,9 @@
-## dbt-bigquery 1.1.0 (Release TBD)
+## dbt-bigquery 1.1.0b1 (March 23, 2022)
 ### Features
 - Provide a fine-grained control of the timeout and retry of BigQuery query with four new dbt profile configs: `job_creation_timeout_seconds`, `job_execution_timeout_seconds`, `job_retry_deadline_seconds`, and `job_retries` ([#45](https://github.com/dbt-labs/dbt-bigquery/issues/45), [#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
 - Adds new integration test to check against new ability to allow unique_key to be a list. [#112](https://github.com/dbt-labs/dbt-bigquery/issues/112)
 - Added upload_file macro to upload a local file to a table. [#102](https://github.com/dbt-labs/dbt-bigquery/issues/102)
+- Add logic to BigQueryConnectionManager to add fuctionality for UPDATE and SELECT statements. [#79](https://github.com/dbt-labs/dbt-bigquery/pull/79)
 
 ### Fixes
 - Fix test related to preventing coercion of boolean values (True, False) to numeric values (0, 1) in query results ([#93](https://github.com/dbt-labs/dbt-bigquery/issues/93))
@@ -11,13 +12,19 @@ This will prevent BigQuery from throwing an error since non-partitioned tables c
 - Ignore errors of the lack of permissions in `list_relations_without_caching` ([#104](https://github.com/dbt-labs/dbt-bigquery/issues/104))
 
 ### Under the hood
-- Address BigQuery API deprecation warning and simplify usage of `TableReference` and `DatasetReference` objects ([#97](https://github.com/dbt-labs/dbt-bigquery/issues/97))
+- Address BigQuery API deprecation warning and simplify usage of `TableReference` and `DatasetReference` objects ([#97](https://github.com/dbt-labs/dbt-bigquery/issues/97)),([#98](https://github.com/dbt-labs/dbt-bigquery/pull/98))
+- Add contributing.md file for adapter repo [#73](https://github.com/dbt-labs/dbt-bigquery/pull/73)
+- Add stale messaging workflow to Github Actions [#103](https://github.com/dbt-labs/dbt-bigquery/pull/103)
+- Add unique_key to go in parity with unique_key as a list chagnes made in core [#119](https://github.com/dbt-labs/dbt-bigquery/pull/119/files)
+- Adding new Enviornment variable for integration testing puproses [#116](https://github.com/dbt-labs/dbt-bigquery/pull/116)
 
 ### Contributors
 - [@hui-zheng](https://github.com/hui-zheng)([#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
 - [@oliverrmaa](https://github.com/oliverrmaa)([#109](https://github.com/dbt-labs/dbt-bigquery/pull/109))
 - [@yu-iskw](https://github.com/yu-iskw)([#108](https://github.com/dbt-labs/dbt-bigquery/pull/108))
 - [@pgoslatara](https://github.com/pgoslatara) ([#66](https://github.com/dbt-labs/dbt-bigquery/pull/121))
+- [@drewmcdonald](https://github.com/drewmcdonald)([#98](https://github.com/dbt-labs/dbt-bigquery/pull/98))
+- [@rjh336](https://github.com/rjh336)([#79](https://github.com/dbt-labs/dbt-bigquery/pull/79))
 
 ## dbt-bigquery 1.0.0 (December 3, 2021)
 
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index de43469d2..a86cb5c59 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = '1.0.0'
+version = '1.1.0b1'
diff --git a/setup.py b/setup.py
index ded7d3bcc..42e0d662f 100644
--- a/setup.py
+++ b/setup.py
@@ -52,7 +52,7 @@ def _get_dbt_core_version():
 
 
 package_name = "dbt-bigquery"
-package_version = "1.0.0"
+package_version = "1.1.0b1"
 dbt_core_version = _get_dbt_core_version()
 description = """The BigQuery adapter plugin for dbt"""
 

From edebc4728fe6fc6cac79d853387060142e7b9e06 Mon Sep 17 00:00:00 2001
From: Gerda Shank 
Date: Tue, 29 Mar 2022 15:35:10 -0400
Subject: [PATCH 434/860] Implement initial adapter tests on bigquery (#142)

---
 CHANGELOG.md                           |  5 +++
 dbt/adapters/bigquery/impl.py          | 15 ++++++++
 dev_requirements.txt                   |  3 +-
 pytest.ini                             | 10 +++++
 tests/conftest.py                      | 22 +++++++++++
 tests/functional/adapter/test_basic.py | 52 ++++++++++++++++++++++++++
 tox.ini                                |  7 ----
 7 files changed, 106 insertions(+), 8 deletions(-)
 create mode 100644 pytest.ini
 create mode 100644 tests/conftest.py
 create mode 100644 tests/functional/adapter/test_basic.py

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 536e5a312..7d17f1ebb 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,3 +1,8 @@
+## dbt-bigquery 1.1.0 (TBD)
+
+### Under the hood
+- Use dbt.tests.adapter.basic in tests (new test framework) ([#135](https://github.com/dbt-labs/dbt-bigquery/issues/135), [#142](https://github.com/dbt-labs/dbt-bigquery/pull/142))
+
 ## dbt-bigquery 1.1.0b1 (March 23, 2022)
 ### Features
 - Provide a fine-grained control of the timeout and retry of BigQuery query with four new dbt profile configs: `job_creation_timeout_seconds`, `job_execution_timeout_seconds`, `job_retry_deadline_seconds`, and `job_retries` ([#45](https://github.com/dbt-labs/dbt-bigquery/issues/45), [#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index ff983c0b8..03931aae9 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -849,3 +849,18 @@ def string_add_sql(
             raise dbt.exceptions.RuntimeException(
                 f'Got an unexpected location value of "{location}"'
             )
+
+    # This is used by the test suite
+    def run_sql_for_tests(self, sql, fetch, conn=None):
+        """ For the testing framework.
+        Run an SQL query on a bigquery adapter. No cursors, transactions,
+        etc. to worry about"""
+
+        do_fetch = fetch != 'None'
+        _, res = self.execute(sql, fetch=do_fetch)
+
+        # convert dataframe to matrix-ish repr
+        if fetch == 'one':
+            return res[0]
+        else:
+            return list(res)
diff --git a/dev_requirements.txt b/dev_requirements.txt
index 0b3983354..e33fb051e 100644
--- a/dev_requirements.txt
+++ b/dev_requirements.txt
@@ -1,6 +1,7 @@
 # install latest changes in dbt-core
 # TODO: how to automate switching from develop to version branches?
-git+https://github.com/dbt-labs/dbt.git#egg=dbt-core&subdirectory=core
+git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
+git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
 
 bumpversion
 flake8
diff --git a/pytest.ini b/pytest.ini
new file mode 100644
index 000000000..b04a6ccf3
--- /dev/null
+++ b/pytest.ini
@@ -0,0 +1,10 @@
+[pytest]
+filterwarnings =
+    ignore:.*'soft_unicode' has been renamed to 'soft_str'*:DeprecationWarning
+    ignore:unclosed file .*:ResourceWarning
+env_files =
+    test.env
+testpaths =
+    tests/unit
+    tests/integration
+    tests/functional
diff --git a/tests/conftest.py b/tests/conftest.py
new file mode 100644
index 000000000..c423f8e03
--- /dev/null
+++ b/tests/conftest.py
@@ -0,0 +1,22 @@
+import pytest
+import os
+import json
+
+# Import the fuctional fixtures as a plugin
+# Note: fixtures with session scope need to be local
+
+pytest_plugins = ["dbt.tests.fixtures.project"]
+
+# The profile dictionary, used to write out profiles.yml
+@pytest.fixture(scope="class")
+def dbt_profile_target():
+    credentials_json_str = os.getenv('BIGQUERY_TEST_SERVICE_ACCOUNT_JSON').replace("'", '')
+    credentials = json.loads(credentials_json_str)
+    project_id = credentials.get('project_id')
+    return {
+        'type': 'bigquery',
+        'method': 'service-account-json',
+        'threads': 1,
+        'project': project_id,
+        'keyfile_json': credentials,
+    }
diff --git a/tests/functional/adapter/test_basic.py b/tests/functional/adapter/test_basic.py
new file mode 100644
index 000000000..b14d95651
--- /dev/null
+++ b/tests/functional/adapter/test_basic.py
@@ -0,0 +1,52 @@
+import pytest
+
+from dbt.tests.adapter.basic.test_base import BaseSimpleMaterializations
+from dbt.tests.adapter.basic.test_singular_tests import BaseSingularTests
+from dbt.tests.adapter.basic.test_singular_tests_ephemeral import (
+    BaseSingularTestsEphemeral,
+)
+from dbt.tests.adapter.basic.test_empty import BaseEmpty
+from dbt.tests.adapter.basic.test_ephemeral import BaseEphemeral
+from dbt.tests.adapter.basic.test_incremental import BaseIncremental
+from dbt.tests.adapter.basic.test_generic_tests import BaseGenericTests
+from dbt.tests.adapter.basic.test_snapshot_check_cols import BaseSnapshotCheckCols
+from dbt.tests.adapter.basic.test_snapshot_timestamp import BaseSnapshotTimestamp
+
+
+class TestSimpleMaterializationsBigQuery(BaseSimpleMaterializations):
+    # This test requires a full-refresh to replace a table with a view
+    @pytest.fixture(scope="class")
+    def test_config(self):
+        return {"require_full_refresh": True}
+
+
+class TestSingularTestsBigQuery(BaseSingularTests):
+    pass
+
+
+class TestSingularTestsEphemeralBigQuery(BaseSingularTestsEphemeral):
+    pass
+
+
+class TestEmptyBigQuery(BaseEmpty):
+    pass
+
+
+class TestEphemeralBigQuery(BaseEphemeral):
+    pass
+
+
+class TestIncrementalBigQuery(BaseIncremental):
+    pass
+
+
+class TestGenericTestsBigQuery(BaseGenericTests):
+    pass
+
+
+class TestSnapshotCheckColsBigQuery(BaseSnapshotCheckCols):
+    pass
+
+
+class TestSnapshotTimestampBigQuery(BaseSnapshotTimestamp):
+    pass
diff --git a/tox.ini b/tox.ini
index f938e6d4f..e73a436b2 100644
--- a/tox.ini
+++ b/tox.ini
@@ -29,10 +29,3 @@ commands =
 deps =
   -rdev_requirements.txt
   -e.
-
-[pytest]
-env_files =
-    test.env
-testpaths =
-    tests/unit
-    tests/integration

From ca38a8239e8c63041efafabcf19a823af28971c7 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Fri, 8 Apr 2022 13:54:09 -0500
Subject: [PATCH 435/860] adding commit hooks without running (#147)

* adding commit hooks without running

* adding changelog entry

* adding mypy ignores

* adding final mypy ignores

* remove duplicate pip install

* adding all code changes from black, pre-commit, mypy (#148)

* adding all code changes from black, pre-commit, mypy

* adding changelog

* new mypy ignores came up
---
 .bumpversion.cfg                              |   4 +-
 .flake8                                       |  12 +
 .github/pull_request_template.md              |   2 +-
 .github/workflows/integration.yml             |   4 +-
 .github/workflows/jira-creation.yml           |   2 +-
 .github/workflows/jira-label.yml              |   3 +-
 .github/workflows/jira-transition.yml         |   2 +-
 .github/workflows/main.yml                    |  31 +-
 .github/workflows/stale.yml                   |   2 +-
 .github/workflows/version-bump.yml            |  20 +-
 .gitignore                                    |  10 +-
 .pre-commit-config.yaml                       |  53 +++
 CHANGELOG.md                                  |   2 +
 CONTRIBUTING.md                               |   4 +-
 MANIFEST.in                                   |   2 +-
 dbt/adapters/bigquery/__init__.py             |   5 +-
 dbt/adapters/bigquery/__version__.py          |   2 +-
 dbt/adapters/bigquery/column.py               |  62 ++-
 dbt/adapters/bigquery/connections.py          | 247 +++++------
 dbt/adapters/bigquery/gcloud.py               |   4 +-
 dbt/adapters/bigquery/impl.py                 | 393 ++++++++----------
 dbt/adapters/bigquery/relation.py             |  38 +-
 dbt/include/bigquery/__init__.py              |   1 +
 dbt/include/bigquery/macros/adapters.sql      |  18 +-
 .../macros/materializations/incremental.sql   |  10 +-
 dev_requirements.txt                          |   2 +
 mypy.ini                                      |   3 +
 scripts/build-dist.sh                         |   2 +-
 setup.py                                      |  61 ++-
 test.env.example                              |   3 +-
 tox.ini                                       |  11 +-
 31 files changed, 499 insertions(+), 516 deletions(-)
 create mode 100644 .flake8
 create mode 100644 .pre-commit-config.yaml
 create mode 100644 mypy.ini

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index b0f5f6aab..26bd8a191 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -4,7 +4,7 @@ parse = (?P\d+)
 	\.(?P\d+)
 	\.(?P\d+)
 	((?Pa|b|rc)(?P\d+))?
-serialize = 
+serialize =
 	{major}.{minor}.{patch}{prerelease}{num}
 	{major}.{minor}.{patch}
 commit = False
@@ -13,7 +13,7 @@ tag = False
 [bumpversion:part:prerelease]
 first_value = a
 optional_value = final
-values = 
+values =
 	a
 	b
 	rc
diff --git a/.flake8 b/.flake8
new file mode 100644
index 000000000..5e6fa8903
--- /dev/null
+++ b/.flake8
@@ -0,0 +1,12 @@
+[flake8]
+select =
+    E
+    W
+    F
+ignore =
+    W503 # makes Flake8 work like black
+    W504
+    E203 # makes Flake8 work like black
+    E741
+    E501
+exclude = tests
diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md
index 0f793f232..beedab7ad 100644
--- a/.github/pull_request_template.md
+++ b/.github/pull_request_template.md
@@ -18,4 +18,4 @@ resolves #
 - [ ] I have signed the [CLA](https://docs.getdbt.com/docs/contributor-license-agreements)
 - [ ] I have run this code in development and it appears to resolve the stated issue
 - [ ] This PR includes tests, or tests are not required/relevant for this PR
-- [ ] I have updated the `CHANGELOG.md` and added information about my change to the "dbt-bigquery next" section.
\ No newline at end of file
+- [ ] I have updated the `CHANGELOG.md` and added information about my change to the "dbt-bigquery next" section.
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 01c7dfba8..2097fded1 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -157,7 +157,7 @@ jobs:
           pip install tox
           pip --version
           tox --version
-          
+
       - name: Install dbt-core latest
         run: |
           pip install "git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core"
@@ -212,7 +212,7 @@ jobs:
 
   post-failure:
     runs-on: ubuntu-latest
-    needs: test 
+    needs: test
     if: ${{ failure() }}
 
     steps:
diff --git a/.github/workflows/jira-creation.yml b/.github/workflows/jira-creation.yml
index c84e106a7..b4016befc 100644
--- a/.github/workflows/jira-creation.yml
+++ b/.github/workflows/jira-creation.yml
@@ -13,7 +13,7 @@ name: Jira Issue Creation
 on:
   issues:
     types: [opened, labeled]
-    
+
 permissions:
   issues: write
 
diff --git a/.github/workflows/jira-label.yml b/.github/workflows/jira-label.yml
index fd533a170..3da2e3a38 100644
--- a/.github/workflows/jira-label.yml
+++ b/.github/workflows/jira-label.yml
@@ -13,7 +13,7 @@ name: Jira Label Mirroring
 on:
   issues:
     types: [labeled, unlabeled]
-    
+
 permissions:
   issues: read
 
@@ -24,4 +24,3 @@ jobs:
       JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
       JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
       JIRA_API_TOKEN: ${{ secrets.JIRA_API_TOKEN }}
-    
diff --git a/.github/workflows/jira-transition.yml b/.github/workflows/jira-transition.yml
index 71273c7a9..ed9f9cd4f 100644
--- a/.github/workflows/jira-transition.yml
+++ b/.github/workflows/jira-transition.yml
@@ -21,4 +21,4 @@ jobs:
     secrets:
       JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
       JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
-      JIRA_API_TOKEN: ${{ secrets.JIRA_API_TOKEN }}
\ No newline at end of file
+      JIRA_API_TOKEN: ${{ secrets.JIRA_API_TOKEN }}
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 23cdb4502..77e8e18e2 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -37,19 +37,10 @@ defaults:
 
 jobs:
   code-quality:
-    name: ${{ matrix.toxenv }}
+    name: code-quality
 
     runs-on: ubuntu-latest
 
-    strategy:
-      fail-fast: false
-      matrix:
-        toxenv: [flake8]
-
-    env:
-      TOXENV: ${{ matrix.toxenv }}
-      PYTEST_ADDOPTS: "-v --color=yes"
-
     steps:
       - name: Check out the repository
         uses: actions/checkout@v2
@@ -62,12 +53,13 @@ jobs:
       - name: Install python dependencies
         run: |
           pip install --user --upgrade pip
-          pip install tox
+          pip install -r dev_requirements.txt
           pip --version
-          tox --version
-
-      - name: Run tox
-        run: tox
+          pre-commit --version
+          mypy --version
+          dbt --version
+      - name: Run pre-comit hooks
+        run: pre-commit run --all-files --show-diff-on-failure
 
   unit:
     name: unit test / python ${{ matrix.python-version }}
@@ -100,11 +92,9 @@ jobs:
           pip install tox
           pip --version
           tox --version
-          
       - name: Install dbt-core latest
         run: |
           pip install "git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core"
-
       - name: Run tox
         run: tox
 
@@ -140,7 +130,6 @@ jobs:
           pip install --user --upgrade pip
           pip install --upgrade setuptools wheel twine check-wheel-contents
           pip --version
-
       - name: Build distributions
         run: ./scripts/build-dist.sh
 
@@ -150,11 +139,9 @@ jobs:
       - name: Check distribution descriptions
         run: |
           twine check dist/*
-
       - name: Check wheel contents
         run: |
           check-wheel-contents dist/*.whl --ignore W007,W008
-
       - uses: actions/upload-artifact@v2
         with:
           name: dist
@@ -184,7 +171,6 @@ jobs:
           pip install --user --upgrade pip
           pip install --upgrade wheel
           pip --version
-
       - uses: actions/download-artifact@v2
         with:
           name: dist
@@ -196,15 +182,12 @@ jobs:
       - name: Install wheel distributions
         run: |
           find ./dist/*.whl -maxdepth 1 -type f | xargs pip install --force-reinstall --find-links=dist/
-
       - name: Check wheel distributions
         run: |
           dbt --version
-
       - name: Install source distributions
         run: |
           find ./dist/*.gz -maxdepth 1 -type f | xargs pip install --force-reinstall --find-links=dist/
-
       - name: Check source distributions
         run: |
           dbt --version
diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml
index 2848ce8f7..a1ca95861 100644
--- a/.github/workflows/stale.yml
+++ b/.github/workflows/stale.yml
@@ -14,4 +14,4 @@ jobs:
           # mark issues/PRs stale when they haven't seen activity in 180 days
           days-before-stale: 180
           # ignore checking issues with the following labels
-          exempt-issue-labels: "epic, discussion"
\ No newline at end of file
+          exempt-issue-labels: "epic, discussion"
diff --git a/.github/workflows/version-bump.yml b/.github/workflows/version-bump.yml
index 4913a6e84..b0a3174df 100644
--- a/.github/workflows/version-bump.yml
+++ b/.github/workflows/version-bump.yml
@@ -1,16 +1,16 @@
 # **what?**
 # This workflow will take a version number and a dry run flag. With that
-# it will run versionbump to update the version number everywhere in the 
+# it will run versionbump to update the version number everywhere in the
 # code base and then generate an update Docker requirements file. If this
 # is a dry run, a draft PR will open with the changes. If this isn't a dry
 # run, the changes will be committed to the branch this is run on.
 
 # **why?**
-# This is to aid in releasing dbt and making sure we have updated 
+# This is to aid in releasing dbt and making sure we have updated
 # the versions and Docker requirements in all places.
 
 # **when?**
-# This is triggered either manually OR 
+# This is triggered either manually OR
 # from the repository_dispatch event "version-bump" which is sent from
 # the dbt-release repo Action
 
@@ -25,11 +25,11 @@ on:
       is_dry_run:
        description: 'Creates a draft PR to allow testing instead of committing to a branch'
        required: true
-       default: 'true'  
+       default: 'true'
   repository_dispatch:
     types: [version-bump]
 
-jobs:          
+jobs:
   bump:
     runs-on: ubuntu-latest
     steps:
@@ -57,19 +57,19 @@ jobs:
         run: |
           python3 -m venv env
           source env/bin/activate
-          pip install --upgrade pip     
-          
+          pip install --upgrade pip
+
       - name: Create PR branch
         if: ${{ steps.variables.outputs.IS_DRY_RUN  == 'true' }}
         run: |
           git checkout -b bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_$GITHUB_RUN_ID
           git push origin bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_$GITHUB_RUN_ID
           git branch --set-upstream-to=origin/bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_$GITHUB_RUN_ID bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_$GITHUB_RUN_ID
-   
+
       - name: Bumping version
         run: |
           source env/bin/activate
-          pip install -r dev_requirements.txt 
+          pip install -r dev_requirements.txt
           env/bin/bumpversion --allow-dirty --new-version ${{steps.variables.outputs.VERSION_NUMBER}} major
           git status
 
@@ -99,4 +99,4 @@ jobs:
           draft: true
           base: ${{github.ref}}
           title: 'Bumping version to ${{steps.variables.outputs.VERSION_NUMBER}}'
-          branch: 'bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_${{GITHUB.RUN_ID}}'    
+          branch: 'bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_${{GITHUB.RUN_ID}}'
diff --git a/.gitignore b/.gitignore
index 43724b61e..780d98f70 100644
--- a/.gitignore
+++ b/.gitignore
@@ -49,9 +49,7 @@ coverage.xml
 *,cover
 .hypothesis/
 test.env
-
-# Mypy
-.mypy_cache/
+*.pytest_cache/
 
 # Translations
 *.mo
@@ -66,10 +64,10 @@ docs/_build/
 # PyBuilder
 target/
 
-#Ipython Notebook
+# Ipython Notebook
 .ipynb_checkpoints
 
-#Emacs
+# Emacs
 *~
 
 # Sublime Text
@@ -78,6 +76,7 @@ target/
 # Vim
 *.sw*
 
+# Pyenv
 .python-version
 
 # Vim
@@ -90,6 +89,7 @@ venv/
 # AWS credentials
 .aws/
 
+# MacOS
 .DS_Store
 
 # vscode
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
new file mode 100644
index 000000000..9d247581b
--- /dev/null
+++ b/.pre-commit-config.yaml
@@ -0,0 +1,53 @@
+# For more on configuring pre-commit hooks (see https://pre-commit.com/)
+
+# TODO: remove global exclusion of tests when testing overhaul is complete
+exclude: '^tests/.*'
+
+default_language_version:
+  python: python3.8
+
+repos:
+- repo: https://github.com/pre-commit/pre-commit-hooks
+  rev: v3.2.0
+  hooks:
+  - id: check-yaml
+    args: [--unsafe]
+  - id: check-json
+  - id: end-of-file-fixer
+  - id: trailing-whitespace
+  - id: check-case-conflict
+- repo: https://github.com/psf/black
+  rev: 22.3.0
+  hooks:
+  - id: black
+    args:
+    - "--line-length=99"
+    - "--target-version=py38"
+  - id: black
+    alias: black-check
+    stages: [manual]
+    args:
+    - "--line-length=99"
+    - "--target-version=py38"
+    - "--check"
+    - "--diff"
+- repo: https://gitlab.com/pycqa/flake8
+  rev: 4.0.1
+  hooks:
+  - id: flake8
+  - id: flake8
+    alias: flake8-check
+    stages: [manual]
+- repo: https://github.com/pre-commit/mirrors-mypy
+  rev: v0.782
+  hooks:
+  - id: mypy
+    args: [--show-error-codes, --ignore-missing-imports]
+    files: ^dbt/adapters/.*
+    language: system
+  - id: mypy
+    alias: mypy-check
+    stages: [manual]
+    args: [--show-error-codes, --pretty, --ignore-missing-imports]
+    files: ^dbt/adapters
+    language: system
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 7d17f1ebb..987bc79c6 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -2,6 +2,8 @@
 
 ### Under the hood
 - Use dbt.tests.adapter.basic in tests (new test framework) ([#135](https://github.com/dbt-labs/dbt-bigquery/issues/135), [#142](https://github.com/dbt-labs/dbt-bigquery/pull/142))
+- Adding pre-commit and black formatter hooks ([#147](https://github.com/dbt-labs/dbt-bigquery/pull/147))
+- Adding pre-commit code changes ([#148](https://github.com/dbt-labs/dbt-bigquery/pull/148))
 
 ## dbt-bigquery 1.1.0b1 (March 23, 2022)
 ### Features
diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md
index 504710c70..aec184e46 100644
--- a/CONTRIBUTING.md
+++ b/CONTRIBUTING.md
@@ -17,7 +17,7 @@ For those wishing to contribute we highly suggest reading the [dbt-core](https:/
 Please note that all contributors to `dbt-bigquery` must sign the [Contributor License Agreement](https://docs.getdbt.com/docs/contributor-license-agreements) to have their Pull Request merged into an `dbt-bigquery` codebase. If you are unable to sign the CLA, then the `dbt-bigquery` maintainers will unfortunately be unable to merge your Pull Request. You are, however, welcome to open issues and comment on existing ones.
 
 
-## Getting the code 
+## Getting the code
 
 You will need `git` in order to download and modify the `dbt-bigquery` source code. You can find direction [here](https://github.com/git-guides/install-git) on how to install `git`.
 
@@ -93,7 +93,7 @@ Many changes will require and update to the `dbt-bigquery` docs here are some us
 
 ## Submitting a Pull Request
 
-dbt Labs provides a CI environment to test changes to the `dbt-bigquery` adapter and periodic checks against the development version of `dbt-core` through Github Actions.  
+dbt Labs provides a CI environment to test changes to the `dbt-bigquery` adapter and periodic checks against the development version of `dbt-core` through Github Actions.
 
 A `dbt-bigquery` maintainer will review your PR. They may suggest code revision for style or clarity, or request that you add unit or integration test(s). These are good things! We believe that, with a little bit of help, anyone can contribute high-quality code.
 
diff --git a/MANIFEST.in b/MANIFEST.in
index 78412d5b8..cfbc714ed 100644
--- a/MANIFEST.in
+++ b/MANIFEST.in
@@ -1 +1 @@
-recursive-include dbt/include *.sql *.yml *.md
\ No newline at end of file
+recursive-include dbt/include *.sql *.yml *.md
diff --git a/dbt/adapters/bigquery/__init__.py b/dbt/adapters/bigquery/__init__.py
index c85e79b38..b66ef278a 100644
--- a/dbt/adapters/bigquery/__init__.py
+++ b/dbt/adapters/bigquery/__init__.py
@@ -8,6 +8,5 @@
 from dbt.include import bigquery
 
 Plugin = AdapterPlugin(
-    adapter=BigQueryAdapter,
-    credentials=BigQueryCredentials,
-    include_path=bigquery.PACKAGE_PATH)
+    adapter=BigQueryAdapter, credentials=BigQueryCredentials, include_path=bigquery.PACKAGE_PATH  # type: ignore[arg-type]
+)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index a86cb5c59..56ec17a89 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = '1.1.0b1'
+version = "1.1.0b1"
diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py
index 6c8a70df1..2422fcc0b 100644
--- a/dbt/adapters/bigquery/column.py
+++ b/dbt/adapters/bigquery/column.py
@@ -5,27 +5,27 @@
 
 from google.cloud.bigquery import SchemaField
 
-Self = TypeVar('Self', bound='BigQueryColumn')
+Self = TypeVar("Self", bound="BigQueryColumn")
 
 
 @dataclass(init=False)
 class BigQueryColumn(Column):
     TYPE_LABELS = {
-        'STRING': 'STRING',
-        'TIMESTAMP': 'TIMESTAMP',
-        'FLOAT': 'FLOAT64',
-        'INTEGER': 'INT64',
-        'RECORD': 'RECORD',
+        "STRING": "STRING",
+        "TIMESTAMP": "TIMESTAMP",
+        "FLOAT": "FLOAT64",
+        "INTEGER": "INT64",
+        "RECORD": "RECORD",
     }
-    fields: List[Self]
-    mode: str
+    fields: List[Self]  # type: ignore
+    mode: str  # type: ignore
 
     def __init__(
         self,
         column: str,
         dtype: str,
         fields: Optional[Iterable[SchemaField]] = None,
-        mode: str = 'NULLABLE',
+        mode: str = "NULLABLE",
     ) -> None:
         super().__init__(column, dtype)
 
@@ -36,9 +36,7 @@ def __init__(
         self.mode = mode
 
     @classmethod
-    def wrap_subfields(
-        cls: Type[Self], fields: Iterable[SchemaField]
-    ) -> List[Self]:
+    def wrap_subfields(cls: Type[Self], fields: Iterable[SchemaField]) -> List[Self]:
         return [cls.create_from_field(field) for field in fields]
 
     @classmethod
@@ -51,20 +49,18 @@ def create_from_field(cls: Type[Self], field: SchemaField) -> Self:
         )
 
     @classmethod
-    def _flatten_recursive(
-        cls: Type[Self], col: Self, prefix: Optional[str] = None
-    ) -> List[Self]:
+    def _flatten_recursive(cls: Type[Self], col: Self, prefix: Optional[str] = None) -> List[Self]:
         if prefix is None:
-            prefix = []
+            prefix = []  # type: ignore[assignment]
 
         if len(col.fields) == 0:
-            prefixed_name = ".".join(prefix + [col.column])
+            prefixed_name = ".".join(prefix + [col.column])  # type: ignore[operator]
             new_col = cls(prefixed_name, col.dtype, col.fields, col.mode)
             return [new_col]
 
         new_fields = []
         for field in col.fields:
-            new_prefix = prefix + [col.column]
+            new_prefix = prefix + [col.column]  # type: ignore[operator]
             new_fields.extend(cls._flatten_recursive(field, new_prefix))
 
         return new_fields
@@ -74,54 +70,52 @@ def flatten(self):
 
     @property
     def quoted(self):
-        return '`{}`'.format(self.column)
+        return "`{}`".format(self.column)
 
     def literal(self, value):
         return "cast({} as {})".format(value, self.dtype)
 
     @property
     def data_type(self) -> str:
-        if self.dtype.upper() == 'RECORD':
+        if self.dtype.upper() == "RECORD":
             subcols = [
-                "{} {}".format(col.name, col.data_type) for col in self.fields
+                "{} {}".format(col.name, col.data_type) for col in self.fields  # type: ignore[attr-defined]
             ]
-            field_type = 'STRUCT<{}>'.format(", ".join(subcols))
+            field_type = "STRUCT<{}>".format(", ".join(subcols))
 
         else:
             field_type = self.dtype
 
-        if self.mode.upper() == 'REPEATED':
-            return 'ARRAY<{}>'.format(field_type)
+        if self.mode.upper() == "REPEATED":
+            return "ARRAY<{}>".format(field_type)
 
         else:
             return field_type
 
     def is_string(self) -> bool:
-        return self.dtype.lower() == 'string'
+        return self.dtype.lower() == "string"
 
     def is_integer(self) -> bool:
-        return self.dtype.lower() == 'int64'
+        return self.dtype.lower() == "int64"
 
     def is_numeric(self) -> bool:
-        return self.dtype.lower() == 'numeric'
+        return self.dtype.lower() == "numeric"
 
     def is_float(self):
-        return self.dtype.lower() == 'float64'
+        return self.dtype.lower() == "float64"
 
-    def can_expand_to(self: Self, other_column: Self) -> bool:
+    def can_expand_to(self: Self, other_column: Self) -> bool:  # type: ignore
         """returns True if both columns are strings"""
         return self.is_string() and other_column.is_string()
 
     def __repr__(self) -> str:
-        return "".format(self.name, self.data_type,
-                                                     self.mode)
+        return "".format(self.name, self.data_type, self.mode)
 
     def column_to_bq_schema(self) -> SchemaField:
-        """Convert a column to a bigquery schema object.
-        """
+        """Convert a column to a bigquery schema object."""
         kwargs = {}
         if len(self.fields) > 0:
-            fields = [field.column_to_bq_schema() for field in self.fields]
+            fields = [field.column_to_bq_schema() for field in self.fields]  # type: ignore[attr-defined]
             kwargs = {"fields": fields}
 
         return SchemaField(self.name, self.dtype, self.mode, **kwargs)
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index ad49cfd9f..05f236a55 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -15,7 +15,7 @@
 from google.auth import impersonated_credentials
 from google.oauth2 import (
     credentials as GoogleCredentials,
-    service_account as GoogleServiceAccountCredentials
+    service_account as GoogleServiceAccountCredentials,
 )
 
 from dbt.adapters.bigquery import gcloud
@@ -24,7 +24,10 @@
 from dbt.tracking import active_user
 from dbt.contracts.connection import ConnectionState, AdapterResponse
 from dbt.exceptions import (
-    FailedToConnectException, RuntimeException, DatabaseException, DbtProfileError
+    FailedToConnectException,
+    RuntimeException,
+    DatabaseException,
+    DbtProfileError,
 )
 from dbt.adapters.base import BaseConnectionManager, Credentials
 from dbt.events import AdapterLogger
@@ -36,7 +39,7 @@
 
 logger = AdapterLogger("BigQuery")
 
-BQ_QUERY_JOB_SPLIT = '-----Query Job SQL Follows-----'
+BQ_QUERY_JOB_SPLIT = "-----Query Job SQL Follows-----"
 
 WRITE_TRUNCATE = google.cloud.bigquery.job.WriteDisposition.WRITE_TRUNCATE
 
@@ -69,15 +72,15 @@ def get_bigquery_defaults(scopes=None) -> Tuple[Any, Optional[str]]:
 
 
 class Priority(StrEnum):
-    Interactive = 'interactive'
-    Batch = 'batch'
+    Interactive = "interactive"
+    Batch = "batch"
 
 
 class BigQueryConnectionMethod(StrEnum):
-    OAUTH = 'oauth'
-    SERVICE_ACCOUNT = 'service-account'
-    SERVICE_ACCOUNT_JSON = 'service-account-json'
-    OAUTH_SECRETS = 'oauth-secrets'
+    OAUTH = "oauth"
+    SERVICE_ACCOUNT = "service-account"
+    SERVICE_ACCOUNT_JSON = "service-account-json"
+    OAUTH_SECRETS = "oauth-secrets"
 
 
 @dataclass
@@ -90,7 +93,7 @@ class BigQueryCredentials(Credentials):
     method: BigQueryConnectionMethod
     # BigQuery allows an empty database / project, where it defers to the
     # environment for the project
-    database: Optional[str]
+    database: Optional[str]  # type: ignore
     execution_project: Optional[str] = None
     location: Optional[str] = None
     priority: Optional[Priority] = None
@@ -114,34 +117,44 @@ class BigQueryCredentials(Credentials):
     token_uri: Optional[str] = None
 
     scopes: Optional[Tuple[str, ...]] = (
-        'https://www.googleapis.com/auth/bigquery',
-        'https://www.googleapis.com/auth/cloud-platform',
-        'https://www.googleapis.com/auth/drive'
+        "https://www.googleapis.com/auth/bigquery",
+        "https://www.googleapis.com/auth/cloud-platform",
+        "https://www.googleapis.com/auth/drive",
     )
 
     _ALIASES = {
         # 'legacy_name': 'current_name'
-        'project': 'database',
-        'dataset': 'schema',
-        'target_project': 'target_database',
-        'target_dataset': 'target_schema',
-        'retries': 'job_retries',
-        'timeout_seconds': 'job_execution_timeout_seconds',
+        "project": "database",
+        "dataset": "schema",
+        "target_project": "target_database",
+        "target_dataset": "target_schema",
+        "retries": "job_retries",
+        "timeout_seconds": "job_execution_timeout_seconds",
     }
 
     @property
     def type(self):
-        return 'bigquery'
+        return "bigquery"
 
     @property
     def unique_field(self):
         return self.database
 
     def _connection_keys(self):
-        return ('method', 'database', 'schema', 'location', 'priority',
-                'timeout_seconds', 'maximum_bytes_billed', 'execution_project',
-                'job_retry_deadline_seconds', 'job_retries',
-                'job_creation_timeout_seconds', 'job_execution_timeout_seconds')
+        return (
+            "method",
+            "database",
+            "schema",
+            "location",
+            "priority",
+            "timeout_seconds",
+            "maximum_bytes_billed",
+            "execution_project",
+            "job_retry_deadline_seconds",
+            "job_retries",
+            "job_creation_timeout_seconds",
+            "job_execution_timeout_seconds",
+        )
 
     @classmethod
     def __pre_deserialize__(cls, d: Dict[Any, Any]) -> Dict[Any, Any]:
@@ -150,17 +163,17 @@ def __pre_deserialize__(cls, d: Dict[Any, Any]) -> Dict[Any, Any]:
         # https://github.com/dbt-labs/dbt/pull/2908#discussion_r532927436.
 
         # `database` is an alias of `project` in BigQuery
-        if 'database' not in d:
+        if "database" not in d:
             _, database = get_bigquery_defaults()
-            d['database'] = database
+            d["database"] = database
         # `execution_project` default to dataset/project
-        if 'execution_project' not in d:
-            d['execution_project'] = d['database']
+        if "execution_project" not in d:
+            d["execution_project"] = d["database"]
         return d
 
 
 class BigQueryConnectionManager(BaseConnectionManager):
-    TYPE = 'bigquery'
+    TYPE = "bigquery"
 
     QUERY_TIMEOUT = 300
     RETRIES = 1
@@ -169,7 +182,7 @@ class BigQueryConnectionManager(BaseConnectionManager):
 
     @classmethod
     def handle_error(cls, error, message):
-        error_msg = "\n".join([item['message'] for item in error.errors])
+        error_msg = "\n".join([item["message"] for item in error.errors])
         raise DatabaseException(error_msg)
 
     def clear_transaction(self):
@@ -189,12 +202,14 @@ def exception_handler(self, sql):
             self.handle_error(e, message)
 
         except google.auth.exceptions.RefreshError as e:
-            message = "Unable to generate access token, if you're using " \
-                      "impersonate_service_account, make sure your " \
-                      'initial account has the "roles/' \
-                      'iam.serviceAccountTokenCreator" role on the ' \
-                      'account you are trying to impersonate.\n\n' \
-                      f'{str(e)}'
+            message = (
+                "Unable to generate access token, if you're using "
+                "impersonate_service_account, make sure your "
+                'initial account has the "roles/'
+                'iam.serviceAccountTokenCreator" role on the '
+                "account you are trying to impersonate.\n\n"
+                f"{str(e)}"
+            )
             raise RuntimeException(message)
 
         except Exception as e:
@@ -229,7 +244,7 @@ def commit(self):
 
     def format_bytes(self, num_bytes):
         if num_bytes:
-            for unit in ['Bytes', 'KB', 'MB', 'GB', 'TB', 'PB']:
+            for unit in ["Bytes", "KB", "MB", "GB", "TB", "PB"]:
                 if abs(num_bytes) < 1024.0:
                     return f"{num_bytes:3.1f} {unit}"
                 num_bytes /= 1024.0
@@ -241,7 +256,7 @@ def format_bytes(self, num_bytes):
             return num_bytes
 
     def format_rows_number(self, rows_number):
-        for unit in ['', 'k', 'm', 'b', 't']:
+        for unit in ["", "k", "m", "b", "t"]:
             if abs(rows_number) < 1000.0:
                 return f"{rows_number:3.1f}{unit}".strip()
             rows_number /= 1000.0
@@ -273,10 +288,10 @@ def get_bigquery_credentials(cls, profile_credentials):
                 client_id=profile_credentials.client_id,
                 client_secret=profile_credentials.client_secret,
                 token_uri=profile_credentials.token_uri,
-                scopes=profile_credentials.scopes
+                scopes=profile_credentials.scopes,
             )
 
-        error = ('Invalid `method` in profile: "{}"'.format(method))
+        error = 'Invalid `method` in profile: "{}"'.format(method)
         raise FailedToConnectException(error)
 
     @classmethod
@@ -292,14 +307,13 @@ def get_impersonated_bigquery_credentials(cls, profile_credentials):
     @classmethod
     def get_bigquery_client(cls, profile_credentials):
         if profile_credentials.impersonate_service_account:
-            creds =\
-                cls.get_impersonated_bigquery_credentials(profile_credentials)
+            creds = cls.get_impersonated_bigquery_credentials(profile_credentials)
         else:
             creds = cls.get_bigquery_credentials(profile_credentials)
         execution_project = profile_credentials.execution_project
-        location = getattr(profile_credentials, 'location', None)
+        location = getattr(profile_credentials, "location", None)
 
-        info = client_info.ClientInfo(user_agent=f'dbt-{dbt_version}')
+        info = client_info.ClientInfo(user_agent=f"dbt-{dbt_version}")
         return google.cloud.bigquery.Client(
             execution_project,
             creds,
@@ -309,8 +323,8 @@ def get_bigquery_client(cls, profile_credentials):
 
     @classmethod
     def open(cls, connection):
-        if connection.state == 'open':
-            logger.debug('Connection is already open, skipping open.')
+        if connection.state == "open":
+            logger.debug("Connection is already open, skipping open.")
             return connection
 
         try:
@@ -323,16 +337,17 @@ def open(cls, connection):
             handle = cls.get_bigquery_client(connection.credentials)
 
         except Exception as e:
-            logger.debug("Got an error when attempting to create a bigquery "
-                         "client: '{}'".format(e))
+            logger.debug(
+                "Got an error when attempting to create a bigquery " "client: '{}'".format(e)
+            )
 
             connection.handle = None
-            connection.state = 'fail'
+            connection.state = "fail"
 
             raise FailedToConnectException(str(e))
 
         connection.handle = handle
-        connection.state = 'open'
+        connection.state = "open"
         return connection
 
     @classmethod
@@ -373,29 +388,30 @@ def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False):
             labels = {}
 
         if active_user:
-            labels['dbt_invocation_id'] = active_user.invocation_id
+            labels["dbt_invocation_id"] = active_user.invocation_id
 
-        job_params = {'use_legacy_sql': use_legacy_sql, 'labels': labels}
+        job_params = {"use_legacy_sql": use_legacy_sql, "labels": labels}
 
         priority = conn.credentials.priority
         if priority == Priority.Batch:
-            job_params['priority'] = google.cloud.bigquery.QueryPriority.BATCH
+            job_params["priority"] = google.cloud.bigquery.QueryPriority.BATCH
         else:
-            job_params[
-                'priority'] = google.cloud.bigquery.QueryPriority.INTERACTIVE
+            job_params["priority"] = google.cloud.bigquery.QueryPriority.INTERACTIVE
 
         maximum_bytes_billed = conn.credentials.maximum_bytes_billed
         if maximum_bytes_billed is not None and maximum_bytes_billed != 0:
-            job_params['maximum_bytes_billed'] = maximum_bytes_billed
+            job_params["maximum_bytes_billed"] = maximum_bytes_billed
 
         job_creation_timeout = self.get_job_creation_timeout_seconds(conn)
         job_execution_timeout = self.get_job_execution_timeout_seconds(conn)
 
         def fn():
             return self._query_and_results(
-                client, sql, job_params,
+                client,
+                sql,
+                job_params,
                 job_creation_timeout=job_creation_timeout,
-                job_execution_timeout=job_execution_timeout
+                job_execution_timeout=job_execution_timeout,
             )
 
         query_job, iterator = self._retry_and_handle(msg=sql, conn=conn, fn=fn)
@@ -414,84 +430,79 @@ def execute(
         else:
             table = agate_helper.empty_table()
 
-        message = 'OK'
+        message = "OK"
         code = None
         num_rows = None
         bytes_processed = None
 
-        if query_job.statement_type == 'CREATE_VIEW':
-            code = 'CREATE VIEW'
+        if query_job.statement_type == "CREATE_VIEW":
+            code = "CREATE VIEW"
 
-        elif query_job.statement_type == 'CREATE_TABLE_AS_SELECT':
+        elif query_job.statement_type == "CREATE_TABLE_AS_SELECT":
             conn = self.get_thread_connection()
             client = conn.handle
             query_table = client.get_table(query_job.destination)
-            code = 'CREATE TABLE'
+            code = "CREATE TABLE"
             num_rows = query_table.num_rows
             num_rows_formated = self.format_rows_number(num_rows)
             bytes_processed = query_job.total_bytes_processed
             processed_bytes = self.format_bytes(bytes_processed)
-            message = f'{code} ({num_rows_formated} rows, {processed_bytes} processed)'
+            message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
 
-        elif query_job.statement_type == 'SCRIPT':
-            code = 'SCRIPT'
+        elif query_job.statement_type == "SCRIPT":
+            code = "SCRIPT"
             bytes_processed = query_job.total_bytes_processed
-            message = f'{code} ({self.format_bytes(bytes_processed)} processed)'
+            message = f"{code} ({self.format_bytes(bytes_processed)} processed)"
 
-        elif query_job.statement_type in ['INSERT', 'DELETE', 'MERGE', 'UPDATE']:
+        elif query_job.statement_type in ["INSERT", "DELETE", "MERGE", "UPDATE"]:
             code = query_job.statement_type
             num_rows = query_job.num_dml_affected_rows
             num_rows_formated = self.format_rows_number(num_rows)
             bytes_processed = query_job.total_bytes_processed
             processed_bytes = self.format_bytes(bytes_processed)
-            message = f'{code} ({num_rows_formated} rows, {processed_bytes} processed)'
+            message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
 
-        elif query_job.statement_type == 'SELECT':
+        elif query_job.statement_type == "SELECT":
             conn = self.get_thread_connection()
             client = conn.handle
             # use anonymous table for num_rows
             query_table = client.get_table(query_job.destination)
-            code = 'SELECT'
+            code = "SELECT"
             num_rows = query_table.num_rows
             num_rows_formated = self.format_rows_number(num_rows)
             bytes_processed = query_job.total_bytes_processed
             processed_bytes = self.format_bytes(bytes_processed)
-            message = f'{code} ({num_rows_formated} rows, {processed_bytes} processed)'
+            message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
 
-        response = BigQueryAdapterResponse(
-            _message=message,
-            rows_affected=num_rows,
-            code=code,
-            bytes_processed=bytes_processed
+        response = BigQueryAdapterResponse(  # type: ignore[call-arg]
+            _message=message, rows_affected=num_rows, code=code, bytes_processed=bytes_processed
         )
 
         return response, table
 
     def get_partitions_metadata(self, table):
         def standard_to_legacy(table):
-            return table.project + ':' + table.dataset + '.' + table.identifier
+            return table.project + ":" + table.dataset + "." + table.identifier
 
-        legacy_sql = 'SELECT * FROM ['\
-            + standard_to_legacy(table) + '$__PARTITIONS_SUMMARY__]'
+        legacy_sql = "SELECT * FROM [" + standard_to_legacy(table) + "$__PARTITIONS_SUMMARY__]"
 
         sql = self._add_query_comment(legacy_sql)
         # auto_begin is ignored on bigquery, and only included for consistency
-        _, iterator =\
-            self.raw_execute(sql, fetch='fetch_result', use_legacy_sql=True)
+        _, iterator = self.raw_execute(sql, fetch="fetch_result", use_legacy_sql=True)
         return self.get_table_from_response(iterator)
 
     def copy_bq_table(self, source, destination, write_disposition):
         conn = self.get_thread_connection()
         client = conn.handle
 
-# -------------------------------------------------------------------------------
-#  BigQuery allows to use copy API using two different formats:
-#  1. client.copy_table(source_table_id, destination_table_id)
-#     where source_table_id = "your-project.source_dataset.source_table"
-#  2. client.copy_table(source_table_ids, destination_table_id)
-#     where source_table_ids = ["your-project.your_dataset.your_table_name", ...]
-#  Let's use uniform function call and always pass list there
-# -------------------------------------------------------------------------------
+        # -------------------------------------------------------------------------------
+        #  BigQuery allows to use copy API using two different formats:
+        #  1. client.copy_table(source_table_id, destination_table_id)
+        #     where source_table_id = "your-project.source_dataset.source_table"
+        #  2. client.copy_table(source_table_ids, destination_table_id)
+        #     where source_table_ids = ["your-project.your_dataset.your_table_name", ...]
+        #  Let's use uniform function call and always pass list there
+        # -------------------------------------------------------------------------------
         if type(source) is not list:
             source = [source]
 
@@ -505,22 +516,24 @@ def copy_bq_table(self, source, destination, write_disposition):
 
         logger.debug(
             'Copying table(s) "{}" to "{}" with disposition: "{}"',
-            ', '.join(source_ref.path for source_ref in source_ref_array),
-            destination_ref.path, write_disposition)
+            ", ".join(source_ref.path for source_ref in source_ref_array),
+            destination_ref.path,
+            write_disposition,
+        )
 
         def copy_and_results():
-            job_config = google.cloud.bigquery.CopyJobConfig(
-                write_disposition=write_disposition)
-            copy_job = client.copy_table(
-                source_ref_array, destination_ref, job_config=job_config)
+            job_config = google.cloud.bigquery.CopyJobConfig(write_disposition=write_disposition)
+            copy_job = client.copy_table(source_ref_array, destination_ref, job_config=job_config)
             iterator = copy_job.result(timeout=self.get_job_execution_timeout_seconds(conn))
             return copy_job, iterator
 
         self._retry_and_handle(
             msg='copy table "{}" to "{}"'.format(
-                ', '.join(source_ref.path for source_ref in source_ref_array),
-                destination_ref.path),
-            conn=conn, fn=copy_and_results)
+                ", ".join(source_ref.path for source_ref in source_ref_array), destination_ref.path
+            ),
+            conn=conn,
+            fn=copy_and_results,
+        )
 
     @staticmethod
     def dataset_ref(database, schema):
@@ -545,8 +558,7 @@ def drop_dataset(self, database, schema):
         def fn():
             return client.delete_dataset(dataset_ref, delete_contents=True, not_found_ok=True)
 
-        self._retry_and_handle(
-            msg='drop dataset', conn=conn, fn=fn)
+        self._retry_and_handle(msg="drop dataset", conn=conn, fn=fn)
 
     def create_dataset(self, database, schema):
         conn = self.get_thread_connection()
@@ -555,30 +567,26 @@ def create_dataset(self, database, schema):
 
         def fn():
             return client.create_dataset(dataset_ref, exists_ok=True)
-        self._retry_and_handle(msg='create dataset', conn=conn, fn=fn)
+
+        self._retry_and_handle(msg="create dataset", conn=conn, fn=fn)
 
     def _query_and_results(
-        self, client, sql, job_params,
-        job_creation_timeout=None,
-        job_execution_timeout=None
+        self, client, sql, job_params, job_creation_timeout=None, job_execution_timeout=None
     ):
         """Query the client and wait for results."""
         # Cannot reuse job_config if destination is set and ddl is used
         job_config = google.cloud.bigquery.QueryJobConfig(**job_params)
-        query_job = client.query(
-            query=sql,
-            job_config=job_config,
-            timeout=job_creation_timeout
-        )
+        query_job = client.query(query=sql, job_config=job_config, timeout=job_creation_timeout)
         iterator = query_job.result(timeout=job_execution_timeout)
 
         return query_job, iterator
 
     def _retry_and_handle(self, msg, conn, fn):
         """retry a function call within the context of exception_handler."""
+
         def reopen_conn_on_error(error):
             if isinstance(error, REOPENABLE_ERRORS):
-                logger.warning('Reopening connection after {!r}'.format(error))
+                logger.warning("Reopening connection after {!r}".format(error))
                 self.close(conn)
                 self.open(conn)
                 return
@@ -589,19 +597,20 @@ def reopen_conn_on_error(error):
                 predicate=_ErrorCounter(self.get_job_retries(conn)).count_error,
                 sleep_generator=self._retry_generator(),
                 deadline=self.get_job_retry_deadline_seconds(conn),
-                on_error=reopen_conn_on_error)
+                on_error=reopen_conn_on_error,
+            )
 
     def _retry_generator(self):
         """Generates retry intervals that exponentially back off."""
         return retry.exponential_sleep_generator(
-            initial=self.DEFAULT_INITIAL_DELAY,
-            maximum=self.DEFAULT_MAXIMUM_DELAY)
+            initial=self.DEFAULT_INITIAL_DELAY, maximum=self.DEFAULT_MAXIMUM_DELAY
+        )
 
     def _labels_from_query_comment(self, comment: str) -> Dict:
         try:
             comment_labels = json.loads(comment)
         except (TypeError, ValueError):
-            return {'query_comment': _sanitize_label(comment)}
+            return {"query_comment": _sanitize_label(comment)}
         return {
             _sanitize_label(key): _sanitize_label(str(value))
             for key, value in comment_labels.items()
@@ -621,9 +630,10 @@ def count_error(self, error):
         self.error_count += 1
         if _is_retryable(error) and self.error_count <= self.retries:
             logger.debug(
-                'Retry attempt {} of {} after error: {}'.format(
+                "Retry attempt {} of {} after error: {}".format(
                     self.error_count, self.retries, repr(error)
-                ))
+                )
+            )
             return True
         else:
             return False
@@ -634,7 +644,8 @@ def _is_retryable(error):
     if isinstance(error, RETRYABLE_ERRORS):
         return True
     elif isinstance(error, google.api_core.exceptions.Forbidden) and any(
-            e['reason'] == 'rateLimitExceeded' for e in error.errors):
+        e["reason"] == "rateLimitExceeded" for e in error.errors
+    ):
         return True
     return False
 
diff --git a/dbt/adapters/bigquery/gcloud.py b/dbt/adapters/bigquery/gcloud.py
index 28e7e1a74..eb418e93b 100644
--- a/dbt/adapters/bigquery/gcloud.py
+++ b/dbt/adapters/bigquery/gcloud.py
@@ -14,7 +14,7 @@
 
 def gcloud_installed():
     try:
-        run_cmd('.', ['gcloud', '--version'])
+        run_cmd(".", ["gcloud", "--version"])
         return True
     except OSError as e:
         logger.debug(e)
@@ -23,6 +23,6 @@ def gcloud_installed():
 
 def setup_default_credentials():
     if gcloud_installed():
-        run_cmd('.', ["gcloud", "auth", "application-default", "login"])
+        run_cmd(".", ["gcloud", "auth", "application-default", "login"])
     else:
         raise dbt.exceptions.RuntimeException(NOT_INSTALLED_MSG)
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 03931aae9..0fc5fc1cc 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -6,10 +6,8 @@
 import dbt.exceptions
 import dbt.clients.agate_helper
 
-from dbt import ui
-from dbt.adapters.base import (
-    BaseAdapter, available, RelationType, SchemaSearchMap, AdapterConfig
-)
+from dbt import ui  # type: ignore
+from dbt.adapters.base import BaseAdapter, available, RelationType, SchemaSearchMap, AdapterConfig
 from dbt.adapters.bigquery.relation import BigQueryRelation
 from dbt.adapters.bigquery import BigQueryColumn
 from dbt.adapters.bigquery import BigQueryConnectionManager
@@ -38,9 +36,7 @@
 
 def sql_escape(string):
     if not isinstance(string, str):
-        dbt.exceptions.raise_compiler_exception(
-            f'cannot escape a non-string: {string}'
-        )
+        dbt.exceptions.raise_compiler_exception(f"cannot escape a non-string: {string}")
 
     return json.dumps(string)[1:-1]
 
@@ -48,25 +44,24 @@ def sql_escape(string):
 @dataclass
 class PartitionConfig(dbtClassMixin):
     field: str
-    data_type: str = 'date'
-    granularity: str = 'day'
+    data_type: str = "date"
+    granularity: str = "day"
     range: Optional[Dict[str, Any]] = None
 
     def render(self, alias: Optional[str] = None):
         column: str = self.field
         if alias:
-            column = f'{alias}.{self.field}'
+            column = f"{alias}.{self.field}"
 
-        if self.data_type.lower() == 'int64' or (
-            self.data_type.lower() == 'date' and
-            self.granularity.lower() == 'day'
+        if self.data_type.lower() == "int64" or (
+            self.data_type.lower() == "date" and self.granularity.lower() == "day"
         ):
             return column
         else:
-            return f'{self.data_type}_trunc({column}, {self.granularity})'
+            return f"{self.data_type}_trunc({column}, {self.granularity})"
 
     @classmethod
-    def parse(cls, raw_partition_by) -> Optional['PartitionConfig']:
+    def parse(cls, raw_partition_by) -> Optional["PartitionConfig"]:  # type: ignore [return]
         if raw_partition_by is None:
             return None
         try:
@@ -74,13 +69,11 @@ def parse(cls, raw_partition_by) -> Optional['PartitionConfig']:
             return cls.from_dict(raw_partition_by)
         except ValidationError as exc:
             msg = dbt.exceptions.validator_error_message(exc)
-            dbt.exceptions.raise_compiler_error(
-                f'Could not parse partition config: {msg}'
-            )
+            dbt.exceptions.raise_compiler_error(f"Could not parse partition config: {msg}")
         except TypeError:
             dbt.exceptions.raise_compiler_error(
-                f'Invalid partition_by config:\n'
-                f'  Got: {raw_partition_by}\n'
+                f"Invalid partition_by config:\n"
+                f"  Got: {raw_partition_by}\n"
                 f'  Expected a dictionary with "field" and "data_type" keys'
             )
 
@@ -91,16 +84,12 @@ class GrantTarget(dbtClassMixin):
     project: str
 
     def render(self):
-        return f'{self.project}.{self.dataset}'
+        return f"{self.project}.{self.dataset}"
 
 
 def _stub_relation(*args, **kwargs):
     return BigQueryRelation.create(
-        database='',
-        schema='',
-        identifier='',
-        quote_policy={},
-        type=BigQueryRelation.Table
+        database="", schema="", identifier="", quote_policy={}, type=BigQueryRelation.Table
     )
 
 
@@ -121,9 +110,9 @@ class BigqueryConfig(AdapterConfig):
 class BigQueryAdapter(BaseAdapter):
 
     RELATION_TYPES = {
-        'TABLE': RelationType.Table,
-        'VIEW': RelationType.View,
-        'EXTERNAL': RelationType.External
+        "TABLE": RelationType.Table,
+        "VIEW": RelationType.View,
+        "EXTERNAL": RelationType.External,
     }
 
     Relation = BigQueryRelation
@@ -138,14 +127,14 @@ class BigQueryAdapter(BaseAdapter):
 
     @classmethod
     def date_function(cls) -> str:
-        return 'CURRENT_TIMESTAMP()'
+        return "CURRENT_TIMESTAMP()"
 
     @classmethod
     def is_cancelable(cls) -> bool:
         return False
 
     def drop_relation(self, relation: BigQueryRelation) -> None:
-        is_cached = self._schema_is_cached(relation.database, relation.schema)
+        is_cached = self._schema_is_cached(relation.database, relation.schema)  # type: ignore[arg-type]
         if is_cached:
             self.cache_dropped(relation)
 
@@ -156,7 +145,7 @@ def drop_relation(self, relation: BigQueryRelation) -> None:
 
     def truncate_relation(self, relation: BigQueryRelation) -> None:
         raise dbt.exceptions.NotImplementedException(
-            '`truncate` is not implemented for this adapter!'
+            "`truncate` is not implemented for this adapter!"
         )
 
     def rename_relation(
@@ -168,11 +157,13 @@ def rename_relation(
 
         from_table_ref = self.get_table_ref_from_relation(from_relation)
         from_table = client.get_table(from_table_ref)
-        if from_table.table_type == "VIEW" or \
-                from_relation.type == RelationType.View or \
-                to_relation.type == RelationType.View:
+        if (
+            from_table.table_type == "VIEW"
+            or from_relation.type == RelationType.View
+            or to_relation.type == RelationType.View
+        ):
             raise dbt.exceptions.RuntimeException(
-                'Renaming of views is not currently supported in BigQuery'
+                "Renaming of views is not currently supported in BigQuery"
             )
 
         to_table_ref = self.get_table_ref_from_relation(to_relation)
@@ -185,18 +176,16 @@ def rename_relation(
     def list_schemas(self, database: str) -> List[str]:
         # the database string we get here is potentially quoted. Strip that off
         # for the API call.
-        database = database.strip('`')
+        database = database.strip("`")
         conn = self.connections.get_thread_connection()
         client = conn.handle
 
         def query_schemas():
             # this is similar to how we have to deal with listing tables
-            all_datasets = client.list_datasets(project=database,
-                                                max_results=10000)
+            all_datasets = client.list_datasets(project=database, max_results=10000)
             return [ds.dataset_id for ds in all_datasets]
 
-        return self.connections._retry_and_handle(
-            msg='list dataset', conn=conn, fn=query_schemas)
+        return self.connections._retry_and_handle(msg="list dataset", conn=conn, fn=query_schemas)
 
     @available.parse(lambda *a, **k: False)
     def check_schema_exists(self, database: str, schema: str) -> bool:
@@ -217,14 +206,10 @@ def check_schema_exists(self, database: str, schema: str) -> bool:
             return False
         return True
 
-    def get_columns_in_relation(
-        self, relation: BigQueryRelation
-    ) -> List[BigQueryColumn]:
+    def get_columns_in_relation(self, relation: BigQueryRelation) -> List[BigQueryColumn]:
         try:
             table = self.connections.get_bq_table(
-                database=relation.database,
-                schema=relation.schema,
-                identifier=relation.identifier
+                database=relation.database, schema=relation.schema, identifier=relation.identifier
             )
             return self._get_dbt_columns_from_bq_table(table)
 
@@ -232,9 +217,7 @@ def get_columns_in_relation(
             logger.debug("get_columns_in_relation error: {}".format(e))
             return []
 
-    def expand_column_types(
-        self, goal: BigQueryRelation, current: BigQueryRelation
-    ) -> None:
+    def expand_column_types(self, goal: BigQueryRelation, current: BigQueryRelation) -> None:  # type: ignore[override]
         # This is a no-op on BigQuery
         pass
 
@@ -265,7 +248,8 @@ def list_relations_without_caching(
             # see: https://github.com/dbt-labs/dbt/issues/726
             # TODO: cache the list of relations up front, and then we
             #       won't need to do this
-            max_results=100000)
+            max_results=100000,
+        )
 
         # This will 404 if the dataset does not exist. This behavior mirrors
         # the implementation of list_relations for other adapters
@@ -274,20 +258,14 @@ def list_relations_without_caching(
         except google.api_core.exceptions.NotFound:
             return []
         except google.api_core.exceptions.Forbidden as exc:
-            logger.debug('list_relations_without_caching error: {}'.format(str(exc)))
+            logger.debug("list_relations_without_caching error: {}".format(str(exc)))
             return []
 
-    def get_relation(
-        self, database: str, schema: str, identifier: str
-    ) -> BigQueryRelation:
+    def get_relation(self, database: str, schema: str, identifier: str) -> BigQueryRelation:
         if self._schema_is_cached(database, schema):
             # if it's in the cache, use the parent's model of going through
             # the relations cache and picking out the relation
-            return super().get_relation(
-                database=database,
-                schema=schema,
-                identifier=identifier
-            )
+            return super().get_relation(database=database, schema=schema, identifier=identifier)
 
         try:
             table = self.connections.get_bq_table(database, schema, identifier)
@@ -310,29 +288,23 @@ def drop_schema(self, relation: BigQueryRelation) -> None:
 
     @classmethod
     def quote(cls, identifier: str) -> str:
-        return '`{}`'.format(identifier)
+        return "`{}`".format(identifier)
 
     @classmethod
     def convert_text_type(cls, agate_table: agate.Table, col_idx: int) -> str:
         return "string"
 
     @classmethod
-    def convert_number_type(
-        cls, agate_table: agate.Table, col_idx: int
-    ) -> str:
+    def convert_number_type(cls, agate_table: agate.Table, col_idx: int) -> str:
         decimals = agate_table.aggregate(agate.MaxPrecision(col_idx))
         return "float64" if decimals else "int64"
 
     @classmethod
-    def convert_boolean_type(
-        cls, agate_table: agate.Table, col_idx: int
-    ) -> str:
+    def convert_boolean_type(cls, agate_table: agate.Table, col_idx: int) -> str:
         return "bool"
 
     @classmethod
-    def convert_datetime_type(
-        cls, agate_table: agate.Table, col_idx: int
-    ) -> str:
+    def convert_datetime_type(cls, agate_table: agate.Table, col_idx: int) -> str:
         return "datetime"
 
     @classmethod
@@ -346,14 +318,14 @@ def convert_time_type(cls, agate_table: agate.Table, col_idx: int) -> str:
     ###
     # Implementation details
     ###
-    def _make_match_kwargs(
-        self, database: str, schema: str, identifier: str
-    ) -> Dict[str, str]:
-        return filter_null_values({
-            'database': database,
-            'identifier': identifier,
-            'schema': schema,
-        })
+    def _make_match_kwargs(self, database: str, schema: str, identifier: str) -> Dict[str, str]:
+        return filter_null_values(
+            {
+                "database": database,
+                "identifier": identifier,
+                "schema": schema,
+            }
+        )
 
     def _get_dbt_columns_from_bq_table(self, table) -> List[BigQueryColumn]:
         "Translates BQ SchemaField dicts into dbt BigQueryColumn objects"
@@ -362,8 +334,7 @@ def _get_dbt_columns_from_bq_table(self, table) -> List[BigQueryColumn]:
         for col in table.schema:
             # BigQuery returns type labels that are not valid type specifiers
             dtype = self.Column.translate_type(col.field_type)
-            column = self.Column(
-                col.name, dtype, col.fields, col.mode)
+            column = self.Column(col.name, dtype, col.fields, col.mode)
             columns.append(column)
 
         return columns
@@ -371,8 +342,7 @@ def _get_dbt_columns_from_bq_table(self, table) -> List[BigQueryColumn]:
     def _agate_to_schema(
         self, agate_table: agate.Table, column_override: Dict[str, str]
     ) -> List[SchemaField]:
-        """Convert agate.Table with column names to a list of bigquery schemas.
-        """
+        """Convert agate.Table with column names to a list of bigquery schemas."""
         bq_schema = []
         for idx, col_name in enumerate(agate_table.column_names):
             inferred_type = self.convert_agate_type(agate_table, idx)
@@ -381,17 +351,14 @@ def _agate_to_schema(
         return bq_schema
 
     def _materialize_as_view(self, model: Dict[str, Any]) -> str:
-        model_database = model.get('database')
-        model_schema = model.get('schema')
-        model_alias = model.get('alias')
-        model_sql = model.get('compiled_sql')
+        model_database = model.get("database")
+        model_schema = model.get("schema")
+        model_alias = model.get("alias")
+        model_sql = model.get("compiled_sql")
 
         logger.debug("Model SQL ({}):\n{}".format(model_alias, model_sql))
         self.connections.create_view(
-            database=model_database,
-            schema=model_schema,
-            table_name=model_alias,
-            sql=model_sql
+            database=model_database, schema=model_schema, table_name=model_alias, sql=model_sql
         )
         return "CREATE VIEW"
 
@@ -401,9 +368,9 @@ def _materialize_as_table(
         model_sql: str,
         decorator: Optional[str] = None,
     ) -> str:
-        model_database = model.get('database')
-        model_schema = model.get('schema')
-        model_alias = model.get('alias')
+        model_database = model.get("database")
+        model_schema = model.get("schema")
+        model_alias = model.get("alias")
 
         if decorator is None:
             table_name = model_alias
@@ -412,28 +379,25 @@ def _materialize_as_table(
 
         logger.debug("Model SQL ({}):\n{}".format(table_name, model_sql))
         self.connections.create_table(
-            database=model_database,
-            schema=model_schema,
-            table_name=table_name,
-            sql=model_sql
+            database=model_database, schema=model_schema, table_name=table_name, sql=model_sql
         )
 
         return "CREATE TABLE"
 
-    @available.parse(lambda *a, **k: '')
+    @available.parse(lambda *a, **k: "")
     def copy_table(self, source, destination, materialization):
-        if materialization == 'incremental':
+        if materialization == "incremental":
             write_disposition = WRITE_APPEND
-        elif materialization == 'table':
+        elif materialization == "table":
             write_disposition = WRITE_TRUNCATE
         else:
             dbt.exceptions.raise_compiler_error(
                 'Copy table materialization must be "copy" or "table", but '
                 f"config.get('copy_materialization', 'table') was "
-                f'{materialization}')
+                f"{materialization}"
+            )
 
-        self.connections.copy_bq_table(
-            source, destination, write_disposition)
+        self.connections.copy_bq_table(source, destination, write_disposition)
 
         return "COPY TABLE with materialization: {}".format(materialization)
 
@@ -441,18 +405,16 @@ def copy_table(self, source, destination, materialization):
     def poll_until_job_completes(cls, job, timeout):
         retry_count = timeout
 
-        while retry_count > 0 and job.state != 'DONE':
+        while retry_count > 0 and job.state != "DONE":
             retry_count -= 1
             time.sleep(1)
             job.reload()
 
-        if job.state != 'DONE':
+        if job.state != "DONE":
             raise dbt.exceptions.RuntimeException("BigQuery Timeout Exceeded")
 
         elif job.error_result:
-            message = '\n'.join(
-                error['message'].strip() for error in job.errors
-            )
+            message = "\n".join(error["message"].strip() for error in job.errors)
             raise dbt.exceptions.RuntimeException(message)
 
     def _bq_table_to_relation(self, bq_table):
@@ -463,13 +425,8 @@ def _bq_table_to_relation(self, bq_table):
             database=bq_table.project,
             schema=bq_table.dataset_id,
             identifier=bq_table.table_id,
-            quote_policy={
-                'schema': True,
-                'identifier': True
-            },
-            type=self.RELATION_TYPES.get(
-                bq_table.table_type, RelationType.External
-            ),
+            quote_policy={"schema": True, "identifier": True},
+            type=self.RELATION_TYPES.get(bq_table.table_type, RelationType.External),
         )
 
     @classmethod
@@ -479,21 +436,19 @@ def warning_on_hooks(hook_type):
         logger.info(warn_msg)
 
     @available
-    def add_query(self, sql, auto_begin=True, bindings=None,
-                  abridge_sql_log=False):
-        if self.nice_connection_name() in ['on-run-start', 'on-run-end']:
+    def add_query(self, sql, auto_begin=True, bindings=None, abridge_sql_log=False):
+        if self.nice_connection_name() in ["on-run-start", "on-run-end"]:
             self.warning_on_hooks(self.nice_connection_name())
         else:
             raise dbt.exceptions.NotImplementedException(
-                '`add_query` is not implemented for this adapter!')
+                "`add_query` is not implemented for this adapter!"
+            )
 
     ###
     # Special bigquery adapter methods
     ###
 
-    def _partitions_match(
-        self, table, conf_partition: Optional[PartitionConfig]
-    ) -> bool:
+    def _partitions_match(self, table, conf_partition: Optional[PartitionConfig]) -> bool:
         """
         Check if the actual and configured partitions for a table are a match.
         BigQuery tables can be replaced if:
@@ -502,23 +457,27 @@ def _partitions_match(
 
         If there is a mismatch, then the table cannot be replaced directly.
         """
-        is_partitioned = (table.range_partitioning or table.time_partitioning)
+        is_partitioned = table.range_partitioning or table.time_partitioning
 
         if not is_partitioned and not conf_partition:
             return True
         elif conf_partition and table.time_partitioning is not None:
             table_field = table.time_partitioning.field.lower()
             table_granularity = table.partitioning_type.lower()
-            return table_field == conf_partition.field.lower() \
+            return (
+                table_field == conf_partition.field.lower()
                 and table_granularity == conf_partition.granularity.lower()
+            )
         elif conf_partition and table.range_partitioning is not None:
             dest_part = table.range_partitioning
             conf_part = conf_partition.range or {}
 
-            return dest_part.field == conf_partition.field \
-                and dest_part.range_.start == conf_part.get('start') \
-                and dest_part.range_.end == conf_part.get('end') \
-                and dest_part.range_.interval == conf_part.get('interval')
+            return (
+                dest_part.field == conf_partition.field
+                and dest_part.range_.start == conf_part.get("start")
+                and dest_part.range_.end == conf_part.get("end")
+                and dest_part.range_.interval == conf_part.get("interval")
+            )
         else:
             return False
 
@@ -535,10 +494,7 @@ def _clusters_match(self, table, conf_cluster) -> bool:
 
     @available.parse(lambda *a, **k: True)
     def is_replaceable(
-        self,
-        relation,
-        conf_partition: Optional[PartitionConfig],
-        conf_cluster
+        self, relation, conf_partition: Optional[PartitionConfig], conf_cluster
     ) -> bool:
         """
         Check if a given partition and clustering column spec for a table
@@ -552,22 +508,20 @@ def is_replaceable(
 
         try:
             table = self.connections.get_bq_table(
-                database=relation.database,
-                schema=relation.schema,
-                identifier=relation.identifier
+                database=relation.database, schema=relation.schema, identifier=relation.identifier
             )
         except google.cloud.exceptions.NotFound:
             return True
 
-        return all((
-            self._partitions_match(table, conf_partition),
-            self._clusters_match(table, conf_cluster)
-        ))
+        return all(
+            (
+                self._partitions_match(table, conf_partition),
+                self._clusters_match(table, conf_cluster),
+            )
+        )
 
     @available
-    def parse_partition_by(
-        self, raw_partition_by: Any
-    ) -> Optional[PartitionConfig]:
+    def parse_partition_by(self, raw_partition_by: Any) -> Optional[PartitionConfig]:
         """
         dbt v0.16.0 expects `partition_by` to be a dictionary where previously
         it was a string. Check the type of `partition_by`, raise error
@@ -576,11 +530,9 @@ def parse_partition_by(
         return PartitionConfig.parse(raw_partition_by)
 
     def get_table_ref_from_relation(self, relation):
-        return self.connections.table_ref(
-            relation.database, relation.schema, relation.identifier
-        )
+        return self.connections.table_ref(relation.database, relation.schema, relation.identifier)
 
-    def _update_column_dict(self, bq_column_dict, dbt_columns, parent=''):
+    def _update_column_dict(self, bq_column_dict, dbt_columns, parent=""):
         """
         Helper function to recursively traverse the schema of a table in the
         update_column_descriptions function below.
@@ -589,28 +541,24 @@ def _update_column_dict(self, bq_column_dict, dbt_columns, parent=''):
         function of a SchemaField object.
         """
         if parent:
-            dotted_column_name = '{}.{}'.format(parent, bq_column_dict['name'])
+            dotted_column_name = "{}.{}".format(parent, bq_column_dict["name"])
         else:
-            dotted_column_name = bq_column_dict['name']
+            dotted_column_name = bq_column_dict["name"]
 
         if dotted_column_name in dbt_columns:
             column_config = dbt_columns[dotted_column_name]
-            bq_column_dict['description'] = column_config.get('description')
-            if column_config.get('policy_tags'):
-                bq_column_dict['policyTags'] = {
-                    'names': column_config.get('policy_tags')
-                }
+            bq_column_dict["description"] = column_config.get("description")
+            if column_config.get("policy_tags"):
+                bq_column_dict["policyTags"] = {"names": column_config.get("policy_tags")}
 
         new_fields = []
-        for child_col_dict in bq_column_dict.get('fields', list()):
+        for child_col_dict in bq_column_dict.get("fields", list()):
             new_child_column_dict = self._update_column_dict(
-                child_col_dict,
-                dbt_columns,
-                parent=dotted_column_name
+                child_col_dict, dbt_columns, parent=dotted_column_name
             )
             new_fields.append(new_child_column_dict)
 
-        bq_column_dict['fields'] = new_fields
+        bq_column_dict["fields"] = new_fields
 
         return bq_column_dict
 
@@ -626,14 +574,11 @@ def update_columns(self, relation, columns):
         new_schema = []
         for bq_column in table.schema:
             bq_column_dict = bq_column.to_api_repr()
-            new_bq_column_dict = self._update_column_dict(
-                bq_column_dict,
-                columns
-            )
+            new_bq_column_dict = self._update_column_dict(bq_column_dict, columns)
             new_schema.append(SchemaField.from_api_repr(new_bq_column_dict))
 
         new_table = google.cloud.bigquery.Table(table_ref, schema=new_schema)
-        conn.handle.update_table(new_table, ['schema'])
+        conn.handle.update_table(new_table, ["schema"])
 
     @available.parse_none
     def update_table_description(
@@ -645,13 +590,12 @@ def update_table_description(
         table_ref = self.connections.table_ref(database, schema, identifier)
         table = client.get_table(table_ref)
         table.description = description
-        client.update_table(table, ['description'])
+        client.update_table(table, ["description"])
 
     @available.parse_none
     def alter_table_add_columns(self, relation, columns):
 
-        logger.debug('Adding columns ({}) to table {}".'.format(
-                     columns, relation))
+        logger.debug('Adding columns ({}) to table {}".'.format(columns, relation))
 
         conn = self.connections.get_thread_connection()
         client = conn.handle
@@ -663,11 +607,10 @@ def alter_table_add_columns(self, relation, columns):
         new_schema = table.schema + new_columns
 
         new_table = google.cloud.bigquery.Table(table_ref, schema=new_schema)
-        client.update_table(new_table, ['schema'])
+        client.update_table(new_table, ["schema"])
 
     @available.parse_none
-    def load_dataframe(self, database, schema, table_name, agate_table,
-                       column_override):
+    def load_dataframe(self, database, schema, table_name, agate_table, column_override):
         bq_schema = self._agate_to_schema(agate_table, column_override)
         conn = self.connections.get_thread_connection()
         client = conn.handle
@@ -679,43 +622,40 @@ def load_dataframe(self, database, schema, table_name, agate_table,
         load_config.schema = bq_schema
 
         with open(agate_table.original_abspath, "rb") as f:
-            job = client.load_table_from_file(f, table_ref, rewind=True,
-                                              job_config=load_config)
+            job = client.load_table_from_file(f, table_ref, rewind=True, job_config=load_config)
 
         timeout = self.connections.get_job_execution_timeout_seconds(conn)
         with self.connections.exception_handler("LOAD TABLE"):
             self.poll_until_job_completes(job, timeout)
 
     @available.parse_none
-    def upload_file(self, local_file_path: str, database: str, table_schema: str,
-                    table_name: str, **kwargs) -> None:
+    def upload_file(
+        self, local_file_path: str, database: str, table_schema: str, table_name: str, **kwargs
+    ) -> None:
         conn = self.connections.get_thread_connection()
         client = conn.handle
 
         table_ref = self.connections.table_ref(database, table_schema, table_name)
 
         load_config = google.cloud.bigquery.LoadJobConfig()
-        for k, v in kwargs['kwargs'].items():
+        for k, v in kwargs["kwargs"].items():
             if k == "schema":
                 setattr(load_config, k, json.loads(v))
             else:
                 setattr(load_config, k, v)
 
         with open(local_file_path, "rb") as f:
-            job = client.load_table_from_file(f, table_ref, rewind=True,
-                                              job_config=load_config)
+            job = client.load_table_from_file(f, table_ref, rewind=True, job_config=load_config)
 
         timeout = self.connections.get_job_execution_timeout_seconds(conn)
         with self.connections.exception_handler("LOAD TABLE"):
             self.poll_until_job_completes(job, timeout)
 
     @classmethod
-    def _catalog_filter_table(
-        cls, table: agate.Table, manifest: Manifest
-    ) -> agate.Table:
-        table = table.rename(column_names={
-            col.name: col.name.replace('__', ':') for col in table.columns
-        })
+    def _catalog_filter_table(cls, table: agate.Table, manifest: Manifest) -> agate.Table:
+        table = table.rename(
+            column_names={col.name: col.name.replace("__", ":") for col in table.columns}
+        )
         return super()._catalog_filter_table(table, manifest)
 
     def _get_catalog_schemas(self, manifest: Manifest) -> SchemaSearchMap:
@@ -726,13 +666,14 @@ def _get_catalog_schemas(self, manifest: Manifest) -> SchemaSearchMap:
         for candidate, schemas in candidates.items():
             database = candidate.database
             if database not in db_schemas:
-                db_schemas[database] = set(self.list_schemas(database))
-            if candidate.schema in db_schemas[database]:
+                db_schemas[database] = set(self.list_schemas(database))  # type: ignore[index]
+            if candidate.schema in db_schemas[database]:  # type: ignore[index]
                 result[candidate] = schemas
             else:
                 logger.debug(
-                    'Skipping catalog for {}.{} - schema does not exist'
-                    .format(database, candidate.schema)
+                    "Skipping catalog for {}.{} - schema does not exist".format(
+                        database, candidate.schema
+                    )
                 )
         return result
 
@@ -742,19 +683,19 @@ def get_common_options(
     ) -> Dict[str, Any]:
         opts = {}
 
-        if (config.get('hours_to_expiration') is not None) and (not temporary):
-            expiration = (
-                'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL '
-                '{} hour)').format(config.get('hours_to_expiration'))
-            opts['expiration_timestamp'] = expiration
+        if (config.get("hours_to_expiration") is not None) and (not temporary):
+            expiration = ("TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL " "{} hour)").format(
+                config.get("hours_to_expiration")
+            )
+            opts["expiration_timestamp"] = expiration
 
-        if config.persist_relation_docs() and 'description' in node:
-            description = sql_escape(node['description'])
-            opts['description'] = '"""{}"""'.format(description)
+        if config.persist_relation_docs() and "description" in node:  # type: ignore[attr-defined]
+            description = sql_escape(node["description"])
+            opts["description"] = '"""{}"""'.format(description)
 
-        if config.get('labels'):
-            labels = config.get('labels', {})
-            opts['labels'] = list(labels.items())
+        if config.get("labels"):
+            labels = config.get("labels", {})
+            opts["labels"] = list(labels.items())  # type: ignore[assignment]
 
         return opts
 
@@ -764,30 +705,28 @@ def get_table_options(
     ) -> Dict[str, Any]:
         opts = self.get_common_options(config, node, temporary)
 
-        if config.get('kms_key_name') is not None:
-            opts['kms_key_name'] = "'{}'".format(config.get('kms_key_name'))
+        if config.get("kms_key_name") is not None:
+            opts["kms_key_name"] = "'{}'".format(config.get("kms_key_name"))
 
         if temporary:
-            expiration = 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)'
-            opts['expiration_timestamp'] = expiration
+            expiration = "TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)"
+            opts["expiration_timestamp"] = expiration
         else:
             # It doesn't apply the `require_partition_filter` option for a temporary table
             # so that we avoid the error by not specifying a partition with a temporary table
             # in the incremental model.
-            if config.get('require_partition_filter') is not None and \
-                    config.get('partition_by') is not None:
-                opts['require_partition_filter'] = config.get(
-                    'require_partition_filter')
-            if config.get('partition_expiration_days') is not None:
-                opts['partition_expiration_days'] = config.get(
-                    'partition_expiration_days')
+            if (
+                config.get("require_partition_filter") is not None
+                and config.get("partition_by") is not None
+            ):
+                opts["require_partition_filter"] = config.get("require_partition_filter")
+            if config.get("partition_expiration_days") is not None:
+                opts["partition_expiration_days"] = config.get("partition_expiration_days")
 
         return opts
 
     @available.parse(lambda *a, **k: {})
-    def get_view_options(
-        self, config: Dict[str, Any], node: Dict[str, Any]
-    ) -> Dict[str, Any]:
+    def get_view_options(self, config: Dict[str, Any], node: Dict[str, Any]) -> Dict[str, Any]:
         opts = self.get_common_options(config, node)
         return opts
 
@@ -804,27 +743,26 @@ def grant_access_to(self, entity, entity_type, role, grant_target_dict):
         dataset_ref = self.connections.dataset_ref(grant_target.project, grant_target.dataset)
         dataset = client.get_dataset(dataset_ref)
 
-        if entity_type == 'view':
+        if entity_type == "view":
             entity = self.get_table_ref_from_relation(entity).to_api_repr()
 
         access_entry = AccessEntry(role, entity_type, entity)
         access_entries = dataset.access_entries
 
         if access_entry in access_entries:
-            logger.debug(f"Access entry {access_entry} "
-                         f"already exists in dataset")
+            logger.debug(f"Access entry {access_entry} " f"already exists in dataset")
             return
 
         access_entries.append(AccessEntry(role, entity_type, entity))
         dataset.access_entries = access_entries
-        client.update_dataset(dataset, ['access_entries'])
+        client.update_dataset(dataset, ["access_entries"])
 
-    def get_rows_different_sql(
+    def get_rows_different_sql(  # type: ignore[override]
         self,
         relation_a: BigQueryRelation,
         relation_b: BigQueryRelation,
         column_names: Optional[List[str]] = None,
-        except_operator='EXCEPT DISTINCT'
+        except_operator="EXCEPT DISTINCT",
     ) -> str:
         return super().get_rows_different_sql(
             relation_a=relation_a,
@@ -833,17 +771,18 @@ def get_rows_different_sql(
             except_operator=except_operator,
         )
 
-    def timestamp_add_sql(
-        self, add_to: str, number: int = 1, interval: str = 'hour'
-    ) -> str:
-        return f'timestamp_add({add_to}, interval {number} {interval})'
+    def timestamp_add_sql(self, add_to: str, number: int = 1, interval: str = "hour") -> str:
+        return f"timestamp_add({add_to}, interval {number} {interval})"
 
     def string_add_sql(
-        self, add_to: str, value: str, location='append',
+        self,
+        add_to: str,
+        value: str,
+        location="append",
     ) -> str:
-        if location == 'append':
+        if location == "append":
             return f"concat({add_to}, '{value}')"
-        elif location == 'prepend':
+        elif location == "prepend":
             return f"concat('{value}', {add_to})"
         else:
             raise dbt.exceptions.RuntimeException(
@@ -852,15 +791,15 @@ def string_add_sql(
 
     # This is used by the test suite
     def run_sql_for_tests(self, sql, fetch, conn=None):
-        """ For the testing framework.
+        """For the testing framework.
         Run an SQL query on a bigquery adapter. No cursors, transactions,
         etc. to worry about"""
 
-        do_fetch = fetch != 'None'
+        do_fetch = fetch != "None"
         _, res = self.execute(sql, fetch=do_fetch)
 
         # convert dataframe to matrix-ish repr
-        if fetch == 'one':
+        if fetch == "one":
             return res[0]
         else:
             return list(res)
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index 08f2c8f06..8156e360d 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -1,19 +1,17 @@
 from dataclasses import dataclass
 from typing import Optional
 
-from dbt.adapters.base.relation import (
-    BaseRelation, ComponentName, InformationSchema
-)
+from dbt.adapters.base.relation import BaseRelation, ComponentName, InformationSchema
 from dbt.utils import filter_null_values
 from typing import TypeVar
 
 
-Self = TypeVar('Self', bound='BigQueryRelation')
+Self = TypeVar("Self", bound="BigQueryRelation")
 
 
 @dataclass(frozen=True, eq=False, repr=False)
 class BigQueryRelation(BaseRelation):
-    quote_character: str = '`'
+    quote_character: str = "`"
 
     def matches(
         self,
@@ -21,11 +19,13 @@ def matches(
         schema: Optional[str] = None,
         identifier: Optional[str] = None,
     ) -> bool:
-        search = filter_null_values({
-            ComponentName.Database: database,
-            ComponentName.Schema: schema,
-            ComponentName.Identifier: identifier
-        })
+        search = filter_null_values(
+            {
+                ComponentName.Database: database,
+                ComponentName.Schema: schema,
+                ComponentName.Identifier: identifier,
+            }
+        )
 
         if not search:
             # nothing was passed in
@@ -45,24 +45,22 @@ def project(self):
     def dataset(self):
         return self.schema
 
-    def information_schema(
-        self, identifier: Optional[str] = None
-    ) -> 'BigQueryInformationSchema':
+    def information_schema(self, identifier: Optional[str] = None) -> "BigQueryInformationSchema":
         return BigQueryInformationSchema.from_relation(self, identifier)
 
 
 @dataclass(frozen=True, eq=False, repr=False)
 class BigQueryInformationSchema(InformationSchema):
-    quote_character: str = '`'
+    quote_character: str = "`"
 
     @classmethod
     def get_include_policy(cls, relation, information_schema_view):
         schema = True
-        if information_schema_view in ('SCHEMATA', 'SCHEMATA_OPTIONS', None):
+        if information_schema_view in ("SCHEMATA", "SCHEMATA_OPTIONS", None):
             schema = False
 
         identifier = True
-        if information_schema_view == '__TABLES__':
+        if information_schema_view == "__TABLES__":
             identifier = False
 
         return relation.include_policy.replace(
@@ -71,10 +69,10 @@ def get_include_policy(cls, relation, information_schema_view):
         )
 
     def replace(self, **kwargs):
-        if 'information_schema_view' in kwargs:
-            view = kwargs['information_schema_view']
+        if "information_schema_view" in kwargs:
+            view = kwargs["information_schema_view"]
             # we also need to update the include policy, unless the caller did
             # in which case it's their problem
-            if 'include_policy' not in kwargs:
-                kwargs['include_policy'] = self.get_include_policy(self, view)
+            if "include_policy" not in kwargs:
+                kwargs["include_policy"] = self.get_include_policy(self, view)
         return super().replace(**kwargs)
diff --git a/dbt/include/bigquery/__init__.py b/dbt/include/bigquery/__init__.py
index 564a3d1e8..b177e5d49 100644
--- a/dbt/include/bigquery/__init__.py
+++ b/dbt/include/bigquery/__init__.py
@@ -1,2 +1,3 @@
 import os
+
 PACKAGE_PATH = os.path.dirname(__file__)
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index fa30922fe..f5a732d4d 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -136,14 +136,14 @@
 {% endmacro %}
 
 {% macro bigquery__alter_relation_add_columns(relation, add_columns) %}
-  
+
   {% set sql -%}
-     
+
      alter {{ relation.type }} {{ relation }}
         {% for column in add_columns %}
           add column {{ column.name }} {{ column.data_type }}{{ ',' if not loop.last }}
         {% endfor %}
-  
+
   {%- endset -%}
 
   {{ return(run_query(sql)) }}
@@ -151,17 +151,17 @@
 {% endmacro %}
 
 {% macro bigquery__alter_relation_drop_columns(relation, drop_columns) %}
-  
+
   {% set sql -%}
-     
+
      alter {{ relation.type }} {{ relation }}
 
         {% for column in drop_columns %}
           drop column {{ column.name }}{{ ',' if not loop.last }}
         {% endfor %}
-  
+
   {%- endset -%}
-  
+
   {{ return(run_query(sql)) }}
 
 {% endmacro %}
@@ -198,11 +198,11 @@
 {% macro bigquery__test_unique(model, column_name) %}
 
 with dbt_test__target as (
-  
+
   select {{ column_name }} as unique_field
   from {{ model }}
   where {{ column_name }} is not null
-  
+
 )
 
 select
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index 56811234f..f9d36aead 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -6,7 +6,7 @@
       select max({{ partition_by.field }}) from {{ this }}
       where {{ partition_by.field }} is not null
     );
-  
+
   {% endif %}
 
 {% endmacro %}
@@ -66,7 +66,7 @@
       {# have we already created the temp table to check for schema changes? #}
       {% if not tmp_relation_exists %}
         {{ declare_dbt_max_partition(this, partition_by, sql) }}
-        
+
         -- 1. create a temp table
         {{ create_table_as(True, tmp_relation, sql) }}
       {% else %}
@@ -156,12 +156,12 @@
 
   {% if existing_relation is none %}
       {% set build_sql = create_table_as(False, target_relation, sql) %}
-  
+
   {% elif existing_relation.is_view %}
       {#-- There's no way to atomically replace a view with a table on BQ --#}
       {{ adapter.drop_relation(existing_relation) }}
       {% set build_sql = create_table_as(False, target_relation, sql) %}
-  
+
   {% elif full_refresh_mode %}
       {#-- If the partition/cluster config has changed, then we must drop and recreate --#}
       {% if not adapter.is_replaceable(existing_relation, partition_by, cluster_by) %}
@@ -169,7 +169,7 @@
           {{ adapter.drop_relation(existing_relation) }}
       {% endif %}
       {% set build_sql = create_table_as(False, target_relation, sql) %}
-  
+
   {% else %}
     {% set tmp_relation_exists = false %}
     {% if on_schema_change != 'ignore' %} {# Check first, since otherwise we may not build a temp table #}
diff --git a/dev_requirements.txt b/dev_requirements.txt
index e33fb051e..ff7b6522b 100644
--- a/dev_requirements.txt
+++ b/dev_requirements.txt
@@ -3,6 +3,7 @@
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
 
+black==22.3.0
 bumpversion
 flake8
 flaky
@@ -10,6 +11,7 @@ freezegun==1.1.0
 ipdb
 mypy==0.782
 pip-tools
+pre-commit
 pytest
 pytest-dotenv
 pytest-logbook
diff --git a/mypy.ini b/mypy.ini
new file mode 100644
index 000000000..51fada1b1
--- /dev/null
+++ b/mypy.ini
@@ -0,0 +1,3 @@
+[mypy]
+mypy_path = ./third-party-stubs
+namespace_packages = True
diff --git a/scripts/build-dist.sh b/scripts/build-dist.sh
index 65e6dbc97..3c3808399 100755
--- a/scripts/build-dist.sh
+++ b/scripts/build-dist.sh
@@ -1,4 +1,4 @@
-#!/bin/bash 
+#!/bin/bash
 
 set -eo pipefail
 
diff --git a/setup.py b/setup.py
index 42e0d662f..3d1a95766 100644
--- a/setup.py
+++ b/setup.py
@@ -5,41 +5,39 @@
 
 # require python 3.7 or newer
 if sys.version_info < (3, 7):
-    print('Error: dbt does not support this version of Python.')
-    print('Please upgrade to Python 3.7 or higher.')
+    print("Error: dbt does not support this version of Python.")
+    print("Please upgrade to Python 3.7 or higher.")
     sys.exit(1)
 
 
 # require version of setuptools that supports find_namespace_packages
 from setuptools import setup
+
 try:
     from setuptools import find_namespace_packages
 except ImportError:
     # the user has a downlevel version of setuptools.
-    print('Error: dbt requires setuptools v40.1.0 or higher.')
-    print('Please upgrade setuptools with "pip install --upgrade setuptools" '
-          'and try again')
+    print("Error: dbt requires setuptools v40.1.0 or higher.")
+    print('Please upgrade setuptools with "pip install --upgrade setuptools" ' "and try again")
     sys.exit(1)
 
 
 # pull long description from README
 this_directory = os.path.abspath(os.path.dirname(__file__))
-with open(os.path.join(this_directory, 'README.md')) as f:
+with open(os.path.join(this_directory, "README.md")) as f:
     long_description = f.read()
 
 
 # get this package's version from dbt/adapters//__version__.py
 def _get_plugin_version_dict():
-    _version_path = os.path.join(
-        this_directory, 'dbt', 'adapters', 'bigquery', '__version__.py'
-    )
-    _semver = r'''(?P\d+)\.(?P\d+)\.(?P\d+)'''
-    _pre = r'''((?Pa|b|rc)(?P
\d+))?'''
-    _version_pattern = fr'''version\s*=\s*["']{_semver}{_pre}["']'''
+    _version_path = os.path.join(this_directory, "dbt", "adapters", "bigquery", "__version__.py")
+    _semver = r"""(?P\d+)\.(?P\d+)\.(?P\d+)"""
+    _pre = r"""((?Pa|b|rc)(?P
\d+))?"""
+    _version_pattern = rf"""version\s*=\s*["']{_semver}{_pre}["']"""
     with open(_version_path) as f:
         match = re.search(_version_pattern, f.read().strip())
         if match is None:
-            raise ValueError(f'invalid version at {_version_path}')
+            raise ValueError(f"invalid version at {_version_path}")
         return match.groupdict()
 
 
@@ -47,7 +45,7 @@ def _get_plugin_version_dict():
 def _get_dbt_core_version():
     parts = _get_plugin_version_dict()
     minor = "{major}.{minor}.0".format(**parts)
-    pre = (parts["prekind"]+"1" if parts["prekind"] else "")
+    pre = parts["prekind"] + "1" if parts["prekind"] else ""
     return f"{minor}{pre}"
 
 
@@ -61,33 +59,30 @@ def _get_dbt_core_version():
     version=package_version,
     description=description,
     long_description=long_description,
-    long_description_content_type='text/markdown',
+    long_description_content_type="text/markdown",
     author="dbt Labs",
     author_email="info@dbtlabs.com",
     url="https://github.com/dbt-labs/dbt-bigquery",
-    packages=find_namespace_packages(include=['dbt', 'dbt.*']),
+    packages=find_namespace_packages(include=["dbt", "dbt.*"]),
     include_package_data=True,
     install_requires=[
-        'dbt-core~={}'.format(dbt_core_version),
-        'protobuf>=3.13.0,<4',
-        'google-cloud-core>=1.3.0,<3',
-        'google-cloud-bigquery>=1.25.0,<3',
-        'google-api-core>=1.16.0,<3',
-        'googleapis-common-protos>=1.6.0,<2',
+        "dbt-core~={}".format(dbt_core_version),
+        "protobuf>=3.13.0,<4",
+        "google-cloud-core>=1.3.0,<3",
+        "google-cloud-bigquery>=1.25.0,<3",
+        "google-api-core>=1.16.0,<3",
+        "googleapis-common-protos>=1.6.0,<2",
     ],
     zip_safe=False,
     classifiers=[
-        'Development Status :: 5 - Production/Stable',
-
-        'License :: OSI Approved :: Apache Software License',
-
-        'Operating System :: Microsoft :: Windows',
-        'Operating System :: MacOS :: MacOS X',
-        'Operating System :: POSIX :: Linux',
-
-        'Programming Language :: Python :: 3.7',
-        'Programming Language :: Python :: 3.8',
-        'Programming Language :: Python :: 3.9',
+        "Development Status :: 5 - Production/Stable",
+        "License :: OSI Approved :: Apache Software License",
+        "Operating System :: Microsoft :: Windows",
+        "Operating System :: MacOS :: MacOS X",
+        "Operating System :: POSIX :: Linux",
+        "Programming Language :: Python :: 3.7",
+        "Programming Language :: Python :: 3.8",
+        "Programming Language :: Python :: 3.9",
     ],
     python_requires=">=3.7",
 )
diff --git a/test.env.example b/test.env.example
index a69a35fe7..031968c60 100644
--- a/test.env.example
+++ b/test.env.example
@@ -1,3 +1,4 @@
 BIGQUERY_TEST_DATABASE=
 BIGQUERY_TEST_ALT_DATABASE=
-BIGQUERY_TEST_SERVICE_ACCOUNT_JSON='{}'
\ No newline at end of file
+BIGQUERY_TEST_NO_ACCESS_DATABASE=
+BIGQUERY_TEST_SERVICE_ACCOUNT_JSON='{}'
diff --git a/tox.ini b/tox.ini
index e73a436b2..4b23d97e2 100644
--- a/tox.ini
+++ b/tox.ini
@@ -1,15 +1,6 @@
 [tox]
 skipsdist = True
-envlist = py37,py38,py39,flake8
-
-[testenv:flake8]
-description = flake8 code checks
-basepython = python3.8
-skip_install = true
-commands = flake8 --select=E,W,F --ignore=W504,E741 --max-line-length 99 \
-  dbt
-deps =
-  -rdev_requirements.txt
+envlist = py37,py38,py39
 
 [testenv:{unit,py37,py38,py39,py}]
 description = unit testing

From 2d26484fa027b836872ca12146dbb7d3729255c3 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Fri, 8 Apr 2022 14:23:06 -0500
Subject: [PATCH 436/860] init push of convert unique id tests (#150)

* init push of convert unique id tests

* adding tox command for new functional tests

* add changelog and removed old tests

* changing pointer back to core main

* removing changelog entry (per disucssion these prs don't need one)

* trying to trigger code-quality checks
---
 .../adapter/test_incremental_unique_id.py     |   6 +
 .../duplicated_unary_unique_key_list.sql      |  17 -
 .../models/empty_str_unique_key.sql           |  14 -
 .../models/empty_unique_key_list.sql          |  12 -
 .../models/expected/one_str__overwrite.sql    |  21 --
 .../unique_key_list__inplace_overwrite.sql    |  21 --
 .../models/no_unique_key.sql                  |  13 -
 .../nontyped_trinary_unique_key_list.sql      |  19 --
 .../models/not_found_unique_key.sql           |  14 -
 .../models/not_found_unique_key_list.sql      |   8 -
 .../models/str_unique_key.sql                 |  17 -
 .../models/trinary_unique_key_list.sql        |  19 --
 .../models/unary_unique_key_list.sql          |  17 -
 .../seeds/add_new_rows.sql                    |   9 -
 .../seeds/duplicate_insert.sql                |   5 -
 .../incremental_unique_id_test/seeds/seed.csv |   7 -
 .../test_incremental_unique_id.py             | 296 ------------------
 tox.ini                                       |   1 +
 18 files changed, 7 insertions(+), 509 deletions(-)
 create mode 100644 tests/functional/adapter/test_incremental_unique_id.py
 delete mode 100644 tests/integration/incremental_unique_id_test/models/duplicated_unary_unique_key_list.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/models/empty_str_unique_key.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/models/empty_unique_key_list.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/models/expected/one_str__overwrite.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/models/expected/unique_key_list__inplace_overwrite.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/models/no_unique_key.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/models/nontyped_trinary_unique_key_list.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/models/not_found_unique_key.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/models/not_found_unique_key_list.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/models/str_unique_key.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/models/trinary_unique_key_list.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/models/unary_unique_key_list.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/seeds/add_new_rows.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/seeds/duplicate_insert.sql
 delete mode 100644 tests/integration/incremental_unique_id_test/seeds/seed.csv
 delete mode 100644 tests/integration/incremental_unique_id_test/test_incremental_unique_id.py

diff --git a/tests/functional/adapter/test_incremental_unique_id.py b/tests/functional/adapter/test_incremental_unique_id.py
new file mode 100644
index 000000000..a485ba9a7
--- /dev/null
+++ b/tests/functional/adapter/test_incremental_unique_id.py
@@ -0,0 +1,6 @@
+import pytest
+from dbt.tests.adapter.incremental.test_incremental_unique_id import BaseIncrementalUniqueKey
+
+
+class TestUniqueKeyBigQuery(BaseIncrementalUniqueKey):
+    pass
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/duplicated_unary_unique_key_list.sql b/tests/integration/incremental_unique_id_test/models/duplicated_unary_unique_key_list.sql
deleted file mode 100644
index 7290b6c43..000000000
--- a/tests/integration/incremental_unique_id_test/models/duplicated_unary_unique_key_list.sql
+++ /dev/null
@@ -1,17 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key=['state', 'state']
-    )
-}}
-
-select
-    state as state,
-    county as county,
-    city as city,
-    last_visit_date as last_visit_date
-from {{ ref('seed') }}
-
-{% if is_incremental() %}
-    where last_visit_date > (select max(last_visit_date) from {{ this }})
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/empty_str_unique_key.sql b/tests/integration/incremental_unique_id_test/models/empty_str_unique_key.sql
deleted file mode 100644
index 5260e177c..000000000
--- a/tests/integration/incremental_unique_id_test/models/empty_str_unique_key.sql
+++ /dev/null
@@ -1,14 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key=''
-    )
-}}
-
-select
-    *
-from {{ ref('seed') }}
-
-{% if is_incremental() %}
-    where last_visit_date > (select max(last_visit_date) from {{ this }})
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/empty_unique_key_list.sql b/tests/integration/incremental_unique_id_test/models/empty_unique_key_list.sql
deleted file mode 100644
index c582d532c..000000000
--- a/tests/integration/incremental_unique_id_test/models/empty_unique_key_list.sql
+++ /dev/null
@@ -1,12 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key=[]
-    )
-}}
-
-select * from {{ ref('seed') }}
-
-{% if is_incremental() %}
-    where last_visit_date > (select max(last_visit_date) from {{ this }})
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/expected/one_str__overwrite.sql b/tests/integration/incremental_unique_id_test/models/expected/one_str__overwrite.sql
deleted file mode 100644
index baa37c601..000000000
--- a/tests/integration/incremental_unique_id_test/models/expected/one_str__overwrite.sql
+++ /dev/null
@@ -1,21 +0,0 @@
-{{
-    config(
-        materialized='table'
-    )
-}}
-
-select
-    'CT' as state,
-    'Hartford' as county,
-    'Hartford' as city,
-    cast('2022-02-14' as date) as last_visit_date
-union all
-select 'MA','Suffolk','Boston','2020-02-12'
-union all
-select 'NJ','Mercer','Trenton','2022-01-01'
-union all
-select 'NY','Kings','Brooklyn','2021-04-02'
-union all
-select 'NY','New York','Manhattan','2021-04-01'
-union all
-select 'PA','Philadelphia','Philadelphia','2021-05-21'
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/expected/unique_key_list__inplace_overwrite.sql b/tests/integration/incremental_unique_id_test/models/expected/unique_key_list__inplace_overwrite.sql
deleted file mode 100644
index baa37c601..000000000
--- a/tests/integration/incremental_unique_id_test/models/expected/unique_key_list__inplace_overwrite.sql
+++ /dev/null
@@ -1,21 +0,0 @@
-{{
-    config(
-        materialized='table'
-    )
-}}
-
-select
-    'CT' as state,
-    'Hartford' as county,
-    'Hartford' as city,
-    cast('2022-02-14' as date) as last_visit_date
-union all
-select 'MA','Suffolk','Boston','2020-02-12'
-union all
-select 'NJ','Mercer','Trenton','2022-01-01'
-union all
-select 'NY','Kings','Brooklyn','2021-04-02'
-union all
-select 'NY','New York','Manhattan','2021-04-01'
-union all
-select 'PA','Philadelphia','Philadelphia','2021-05-21'
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/no_unique_key.sql b/tests/integration/incremental_unique_id_test/models/no_unique_key.sql
deleted file mode 100644
index 44a63e75c..000000000
--- a/tests/integration/incremental_unique_id_test/models/no_unique_key.sql
+++ /dev/null
@@ -1,13 +0,0 @@
-{{
-    config(
-        materialized='incremental'
-    )
-}}
-
-select
-    *
-from {{ ref('seed') }}
-
-{% if is_incremental() %}
-    where last_visit_date > (select max(last_visit_date) from {{ this }})
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/nontyped_trinary_unique_key_list.sql b/tests/integration/incremental_unique_id_test/models/nontyped_trinary_unique_key_list.sql
deleted file mode 100644
index 52b4509f0..000000000
--- a/tests/integration/incremental_unique_id_test/models/nontyped_trinary_unique_key_list.sql
+++ /dev/null
@@ -1,19 +0,0 @@
--- for comparing against auto-typed seeds
-
-{{
-    config(
-        materialized='incremental',
-        unique_key=['state', 'county', 'city']
-    )
-}}
-
-select
-    state as state,
-    county as county,
-    city as city,
-    last_visit_date as last_visit_date
-from {{ ref('seed') }}
-
-{% if is_incremental() %}
-    where last_visit_date > (select max(last_visit_date) from {{ this }})
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/not_found_unique_key.sql b/tests/integration/incremental_unique_id_test/models/not_found_unique_key.sql
deleted file mode 100644
index d247aa341..000000000
--- a/tests/integration/incremental_unique_id_test/models/not_found_unique_key.sql
+++ /dev/null
@@ -1,14 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key='thisisnotacolumn'
-    )
-}}
-
-select
-    *
-from {{ ref('seed') }}
-
-{% if is_incremental() %}
-    where last_visit_date > (select max(last_visit_date) from {{ this }})
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/not_found_unique_key_list.sql b/tests/integration/incremental_unique_id_test/models/not_found_unique_key_list.sql
deleted file mode 100644
index f1462a48f..000000000
--- a/tests/integration/incremental_unique_id_test/models/not_found_unique_key_list.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key=['state', 'thisisnotacolumn']
-    )
-}}
-
-select * from {{ ref('seed') }}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/str_unique_key.sql b/tests/integration/incremental_unique_id_test/models/str_unique_key.sql
deleted file mode 100644
index 2f9fc2987..000000000
--- a/tests/integration/incremental_unique_id_test/models/str_unique_key.sql
+++ /dev/null
@@ -1,17 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key='state'
-    )
-}}
-
-select
-    state as state,
-    county as county,
-    city as city,
-    last_visit_date as last_visit_date
-from {{ ref('seed') }}
-
-{% if is_incremental() %}
-    where last_visit_date > (select max(last_visit_date) from {{ this }})
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/trinary_unique_key_list.sql b/tests/integration/incremental_unique_id_test/models/trinary_unique_key_list.sql
deleted file mode 100644
index 0359546bf..000000000
--- a/tests/integration/incremental_unique_id_test/models/trinary_unique_key_list.sql
+++ /dev/null
@@ -1,19 +0,0 @@
--- types needed to compare against expected model reliably
-
-{{
-    config(
-        materialized='incremental',
-        unique_key=['state', 'county', 'city']
-    )
-}}
-
-select
-    state as state,
-    county as county,
-    city as city,
-    last_visit_date as last_visit_date
-from {{ ref('seed') }}
-
-{% if is_incremental() %}
-    where last_visit_date > (select max(last_visit_date) from {{ this }})
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/models/unary_unique_key_list.sql b/tests/integration/incremental_unique_id_test/models/unary_unique_key_list.sql
deleted file mode 100644
index 7f5875f85..000000000
--- a/tests/integration/incremental_unique_id_test/models/unary_unique_key_list.sql
+++ /dev/null
@@ -1,17 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key=['state']
-    )
-}}
-
-select
-    state as state,
-    county as county,
-    city as city,
-    last_visit_date as last_visit_date
-from {{ ref('seed') }}
-
-{% if is_incremental() %}
-    where last_visit_date > (select max(last_visit_date) from {{ this }})
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/seeds/add_new_rows.sql b/tests/integration/incremental_unique_id_test/seeds/add_new_rows.sql
deleted file mode 100644
index 6d515ca3a..000000000
--- a/tests/integration/incremental_unique_id_test/seeds/add_new_rows.sql
+++ /dev/null
@@ -1,9 +0,0 @@
--- insert two new rows, both of which should be in incremental model
---   with any unique columns
-insert into {schema}.seed
-    (state, county, city, last_visit_date)
-values ('WA','King','Seattle','2022-02-01');
-
-insert into {schema}.seed
-    (state, county, city, last_visit_date)
-values ('CA','Los Angeles','Los Angeles','2022-02-01');
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/seeds/duplicate_insert.sql b/tests/integration/incremental_unique_id_test/seeds/duplicate_insert.sql
deleted file mode 100644
index bd73cee86..000000000
--- a/tests/integration/incremental_unique_id_test/seeds/duplicate_insert.sql
+++ /dev/null
@@ -1,5 +0,0 @@
--- insert new row, which should not be in incremental model
---  with primary or first three columns unique
-insert into {schema}.seed
-    (state, county, city, last_visit_date)
-values ('CT','Hartford','Hartford','2022-02-14');
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/seeds/seed.csv b/tests/integration/incremental_unique_id_test/seeds/seed.csv
deleted file mode 100644
index b988827fb..000000000
--- a/tests/integration/incremental_unique_id_test/seeds/seed.csv
+++ /dev/null
@@ -1,7 +0,0 @@
-state,county,city,last_visit_date
-CT,Hartford,Hartford,2020-09-23
-MA,Suffolk,Boston,2020-02-12
-NJ,Mercer,Trenton,2022-01-01
-NY,Kings,Brooklyn,2021-04-02
-NY,New York,Manhattan,2021-04-01
-PA,Philadelphia,Philadelphia,2021-05-21
\ No newline at end of file
diff --git a/tests/integration/incremental_unique_id_test/test_incremental_unique_id.py b/tests/integration/incremental_unique_id_test/test_incremental_unique_id.py
deleted file mode 100644
index a21d84327..000000000
--- a/tests/integration/incremental_unique_id_test/test_incremental_unique_id.py
+++ /dev/null
@@ -1,296 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-from dbt.contracts.results import RunStatus
-from collections import namedtuple
-from pathlib import Path
-
-
-TestResults = namedtuple(
-    'TestResults',
-    ['seed_count', 'model_count', 'seed_rows', 'inc_test_model_count',
-     'opt_model_count', 'relation'],
-)
-
-
-class TestIncrementalUniqueKey(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return 'incremental_unique_key'
-
-    @property
-    def models(self):
-        return 'models'
-
-    def update_incremental_model(self, incremental_model):
-        '''update incremental model after the seed table has been updated'''
-        model_result_set = self.run_dbt(['run', '--select', incremental_model])
-        return len(model_result_set)
-
-    def setup_test(self, seed, incremental_model, update_sql_file):
-        '''build a test case and return values for assertions'''
-        
-        # Idempotently create some number of seeds and incremental models
-        seed_count = len(self.run_dbt(
-            ['seed', '--select', seed, '--full-refresh']
-        ))
-        model_count = len(self.run_dbt(
-            ['run', '--select', incremental_model, '--full-refresh']
-        ))
-        
-         # Upate seed and return new row count
-        row_count_query = 'select * from {}.{}'.format(
-            self.unique_schema(),
-            seed
-        )
-        self.run_sql_file(Path('seeds') / Path(update_sql_file + '.sql'))
-        seed_rows = len(self.run_sql(row_count_query, fetch='all'))
-
-        inc_test_model_count = self.update_incremental_model(
-            incremental_model=incremental_model
-        )
-
-        return (seed_count, model_count, seed_rows, inc_test_model_count)
-
-    def test_scenario_correctness(self, expected_fields, test_case_fields):
-        '''Invoke assertions to verify correct build functionality'''
-        # 1. test seed(s) should build afresh
-        self.assertEqual(
-            expected_fields.seed_count, test_case_fields.seed_count
-        )
-        # 2. test model(s) should build afresh
-        self.assertEqual(
-            expected_fields.model_count, test_case_fields.model_count
-        )
-        # 3. seeds should have intended row counts post update
-        self.assertEqual(
-            expected_fields.seed_rows, test_case_fields.seed_rows
-        )
-        # 4. incremental test model(s) should be updated
-        self.assertEqual(
-            expected_fields.inc_test_model_count,
-            test_case_fields.inc_test_model_count
-        )
-        # 5. extra incremental model(s) should be built; optional since
-        #   comparison may be between an incremental model and seed
-        if (expected_fields.opt_model_count and
-            test_case_fields.opt_model_count):
-            self.assertEqual(
-                expected_fields.opt_model_count,
-                test_case_fields.opt_model_count
-            )
-        # 6. result table should match intended result set (itself a relation)
-        self.assertTablesEqual(
-            expected_fields.relation, test_case_fields.relation
-        )
-
-    def stub_expected_fields(
-        self, relation, seed_rows, opt_model_count=None
-    ):
-        return TestResults(
-            seed_count=1, model_count=1, seed_rows=seed_rows,
-            inc_test_model_count=1, opt_model_count=opt_model_count,
-            relation=relation
-        )
-
-    def fail_to_build_inc_missing_unique_key_column(self, incremental_model_name):
-        '''should pass back error state when trying build an incremental
-           model whose unique key or keylist includes a column missing
-           from the incremental model'''
-        seed_count = len(self.run_dbt(
-            ['seed', '--select', 'seed', '--full-refresh']
-        ))
-        # unique keys are not applied on first run, so two are needed
-        self.run_dbt(
-            ['run', '--select', incremental_model_name, '--full-refresh'],
-            expect_pass=True
-        )
-        run_result = self.run_dbt(
-            ['run', '--select', incremental_model_name],
-            expect_pass=False
-        ).results[0]
-
-        return run_result.status, run_result.message
-
-
-class TestNoIncrementalUniqueKey(TestIncrementalUniqueKey):
-    @use_profile('bigquery')
-    def test__bigquery_no_unique_keys(self):
-        '''with no unique keys, seed and model should match'''
-        seed='seed'
-        seed_rows=8
-        incremental_model='no_unique_key'
-        update_sql_file='add_new_rows'
-
-        expected_fields = self.stub_expected_fields(
-            relation=seed, seed_rows=seed_rows
-        )
-        test_case_fields = TestResults(
-            *self.setup_test(seed, incremental_model, update_sql_file),
-            opt_model_count=None, relation=incremental_model
-        )
-
-        self.test_scenario_correctness(expected_fields, test_case_fields)
-
-
-class TestIncrementalStrUniqueKey(TestIncrementalUniqueKey):
-    @use_profile('bigquery')
-    def test__bigquery_empty_str_unique_key(self):
-        '''with empty string for unique key, seed and model should match'''
-        seed='seed'
-        seed_rows=8
-        incremental_model='empty_str_unique_key'
-        update_sql_file='add_new_rows'
-
-        expected_fields = self.stub_expected_fields(
-            relation=seed, seed_rows=seed_rows
-        )
-        test_case_fields = TestResults(
-            *self.setup_test(seed, incremental_model, update_sql_file),
-            opt_model_count=None, relation=incremental_model
-        )
-
-        self.test_scenario_correctness(expected_fields, test_case_fields)
-
-    @use_profile('bigquery')
-    def test__bigquery_one_unique_key(self):
-        '''with one unique key, model will overwrite existing row'''
-        seed='seed'
-        seed_rows=7
-        incremental_model='str_unique_key'
-        update_sql_file='duplicate_insert'
-        expected_model='one_str__overwrite'
-
-        expected_fields = self.stub_expected_fields(
-            relation=expected_model, seed_rows=seed_rows, opt_model_count=1
-        )
-        test_case_fields = TestResults(
-            *self.setup_test(seed, incremental_model, update_sql_file),
-            opt_model_count=self.update_incremental_model(expected_model),
-            relation=incremental_model
-        )
-
-        self.test_scenario_correctness(expected_fields, test_case_fields)
-
-    @use_profile('bigquery')
-    def test__bigquery_bad_unique_key(self):
-        '''expect compilation error from unique key not being a column'''
-
-        err_msg = "Name thisisnotacolumn not found inside DBT_INTERNAL_SOURCE"
-
-        (status, exc) = self.fail_to_build_inc_missing_unique_key_column(
-            incremental_model_name='not_found_unique_key'
-        )
-
-        self.assertEqual(status, RunStatus.Error)
-        self.assertTrue(err_msg in exc)
-
-
-class TestIncrementalListUniqueKey(TestIncrementalUniqueKey):
-    @use_profile('bigquery')
-    def test__bigquery_empty_unique_key_list(self):
-        '''with no unique keys, seed and model should match'''
-        seed='seed'
-        seed_rows=8
-        incremental_model='empty_unique_key_list'
-        update_sql_file='add_new_rows'
-
-        expected_fields = self.stub_expected_fields(
-            relation=seed, seed_rows=seed_rows
-        )
-        test_case_fields = TestResults(
-            *self.setup_test(seed, incremental_model, update_sql_file),
-            opt_model_count=None, relation=incremental_model
-        )
-
-        self.test_scenario_correctness(expected_fields, test_case_fields)
-
-    @use_profile('bigquery')
-    def test__bigquery_unary_unique_key_list(self):
-        '''with one unique key, model will overwrite existing row'''
-        seed='seed'
-        seed_rows=7
-        incremental_model='unary_unique_key_list'
-        update_sql_file='duplicate_insert'
-        expected_model='unique_key_list__inplace_overwrite'
-
-        expected_fields = self.stub_expected_fields(
-            relation=expected_model, seed_rows=seed_rows, opt_model_count=1
-        )
-        test_case_fields = TestResults(
-            *self.setup_test(seed, incremental_model, update_sql_file),
-            opt_model_count=self.update_incremental_model(expected_model),
-            relation=incremental_model
-        )
-
-        self.test_scenario_correctness(expected_fields, test_case_fields)
-
-    @use_profile('bigquery')
-    def test__bigquery_duplicated_unary_unique_key_list(self):
-        '''with two of the same unique key, model will overwrite existing row'''
-        seed='seed'
-        seed_rows=7
-        incremental_model='duplicated_unary_unique_key_list'
-        update_sql_file='duplicate_insert'
-        expected_model='unique_key_list__inplace_overwrite'
-
-        expected_fields = self.stub_expected_fields(
-            relation=expected_model, seed_rows=seed_rows, opt_model_count=1
-        )
-        test_case_fields = TestResults(
-            *self.setup_test(seed, incremental_model, update_sql_file),
-            opt_model_count=self.update_incremental_model(expected_model),
-            relation=incremental_model
-        )
-
-        self.test_scenario_correctness(expected_fields, test_case_fields)
-
-    @use_profile('bigquery')
-    def test__bigquery_trinary_unique_key_list(self):
-        '''with three unique keys, model will overwrite existing row'''
-        seed='seed'
-        seed_rows=7
-        incremental_model='trinary_unique_key_list'
-        update_sql_file='duplicate_insert'
-        expected_model='unique_key_list__inplace_overwrite'
-
-        expected_fields = self.stub_expected_fields(
-            relation=expected_model, seed_rows=seed_rows, opt_model_count=1
-        )
-        test_case_fields = TestResults(
-            *self.setup_test(seed, incremental_model, update_sql_file),
-            opt_model_count=self.update_incremental_model(expected_model),
-            relation=incremental_model
-        )
-
-        self.test_scenario_correctness(expected_fields, test_case_fields)
-
-    @use_profile('bigquery')
-    def test__bigquery_trinary_unique_key_list_no_update(self):
-        '''even with three unique keys, adding distinct rows to seed does not
-           cause seed and model to diverge'''
-        seed='seed'
-        seed_rows=8
-        incremental_model='nontyped_trinary_unique_key_list'
-        update_sql_file='add_new_rows'
-
-        expected_fields = self.stub_expected_fields(
-            relation=seed, seed_rows=seed_rows
-        )
-        test_case_fields = TestResults(
-            *self.setup_test(seed, incremental_model, update_sql_file),
-            opt_model_count=None, relation=incremental_model
-        )
-
-        self.test_scenario_correctness(expected_fields, test_case_fields)
-
-    @use_profile('bigquery')
-    def test__bigquery_bad_unique_key_list(self):
-        '''expect compilation error from unique key not being a column'''
-
-        err_msg = "Name thisisnotacolumn not found inside DBT_INTERNAL_SOURCE"
-
-        (status, exc) = self.fail_to_build_inc_missing_unique_key_column(
-            incremental_model_name='not_found_unique_key_list'
-        )
-
-        self.assertEqual(status, RunStatus.Error)
-        self.assertTrue(err_msg in exc)
diff --git a/tox.ini b/tox.ini
index 4b23d97e2..fad7d7e34 100644
--- a/tox.ini
+++ b/tox.ini
@@ -17,6 +17,7 @@ skip_install = true
 passenv = DBT_* BIGQUERY_TEST_* PYTEST_ADDOPTS
 commands =
   bigquery: {envpython} -m pytest {posargs} -m profile_bigquery tests/integration
+  bigquery: {envpython} -m pytest {posargs} tests/functional
 deps =
   -rdev_requirements.txt
   -e.

From 3d6986984fffb21d82d3365e7cc5d72545f602e6 Mon Sep 17 00:00:00 2001
From: Chenyu Li 
Date: Fri, 8 Apr 2022 15:02:24 -0600
Subject: [PATCH 437/860] convert adapter method test (#155)

---
 tests/functional/adapter/test_basic.py        |   4 +
 .../bigquery-models/renamed_model.sql         |   1 -
 .../bigquery-models/sources.yml               |   6 -
 .../macros/rename_named_relation.sql          |   6 -
 .../adapter_methods_test/models/expected.sql  |   3 -
 .../adapter_methods_test/models/model.sql     |  19 ----
 .../adapter_methods_test/models/upstream.sql  |   1 -
 .../adapter_methods_test/seed_bq.sql          |  32 ------
 .../test_adapter_methods.py                   | 104 ------------------
 .../tests/get_columns_in_relation.sql         |   7 --
 10 files changed, 4 insertions(+), 179 deletions(-)
 delete mode 100644 tests/integration/adapter_methods_test/bigquery-models/renamed_model.sql
 delete mode 100644 tests/integration/adapter_methods_test/bigquery-models/sources.yml
 delete mode 100644 tests/integration/adapter_methods_test/macros/rename_named_relation.sql
 delete mode 100644 tests/integration/adapter_methods_test/models/expected.sql
 delete mode 100644 tests/integration/adapter_methods_test/models/model.sql
 delete mode 100644 tests/integration/adapter_methods_test/models/upstream.sql
 delete mode 100644 tests/integration/adapter_methods_test/seed_bq.sql
 delete mode 100644 tests/integration/adapter_methods_test/test_adapter_methods.py
 delete mode 100644 tests/integration/adapter_methods_test/tests/get_columns_in_relation.sql

diff --git a/tests/functional/adapter/test_basic.py b/tests/functional/adapter/test_basic.py
index b14d95651..fceef2dee 100644
--- a/tests/functional/adapter/test_basic.py
+++ b/tests/functional/adapter/test_basic.py
@@ -11,6 +11,7 @@
 from dbt.tests.adapter.basic.test_generic_tests import BaseGenericTests
 from dbt.tests.adapter.basic.test_snapshot_check_cols import BaseSnapshotCheckCols
 from dbt.tests.adapter.basic.test_snapshot_timestamp import BaseSnapshotTimestamp
+from dbt.tests.adapter.basic.test_adapter_methods import BaseAdapterMethod
 
 
 class TestSimpleMaterializationsBigQuery(BaseSimpleMaterializations):
@@ -50,3 +51,6 @@ class TestSnapshotCheckColsBigQuery(BaseSnapshotCheckCols):
 
 class TestSnapshotTimestampBigQuery(BaseSnapshotTimestamp):
     pass
+
+class TestBaseAdapterMethodBigQuery(BaseAdapterMethod):
+    pass
diff --git a/tests/integration/adapter_methods_test/bigquery-models/renamed_model.sql b/tests/integration/adapter_methods_test/bigquery-models/renamed_model.sql
deleted file mode 100644
index 3c030d920..000000000
--- a/tests/integration/adapter_methods_test/bigquery-models/renamed_model.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ source('test_source', 'renamed_seed') }}
\ No newline at end of file
diff --git a/tests/integration/adapter_methods_test/bigquery-models/sources.yml b/tests/integration/adapter_methods_test/bigquery-models/sources.yml
deleted file mode 100644
index cb74cde9c..000000000
--- a/tests/integration/adapter_methods_test/bigquery-models/sources.yml
+++ /dev/null
@@ -1,6 +0,0 @@
-version: 2
-sources:
-  - name: test_source
-    schema: "{{ target.schema }}"
-    tables:
-      - name: renamed_seed
diff --git a/tests/integration/adapter_methods_test/macros/rename_named_relation.sql b/tests/integration/adapter_methods_test/macros/rename_named_relation.sql
deleted file mode 100644
index 253e1e0ad..000000000
--- a/tests/integration/adapter_methods_test/macros/rename_named_relation.sql
+++ /dev/null
@@ -1,6 +0,0 @@
--- Macro to rename a relation
-{% macro rename_named_relation(from_name, to_name) %}
-{%- set from_relation = api.Relation.create(database=target.database, schema=target.schema, identifier=from_name, type='table') -%}
-{%- set to_relation = api.Relation.create(database=target.database, schema=target.schema, identifier=to_name, type='table') -%}
-{% do adapter.rename_relation(from_relation, to_relation) %}
-{% endmacro %}
\ No newline at end of file
diff --git a/tests/integration/adapter_methods_test/models/expected.sql b/tests/integration/adapter_methods_test/models/expected.sql
deleted file mode 100644
index ddebb5f5d..000000000
--- a/tests/integration/adapter_methods_test/models/expected.sql
+++ /dev/null
@@ -1,3 +0,0 @@
--- make sure this runs after 'model'
--- {{ ref('model') }}
-select 2 as id
diff --git a/tests/integration/adapter_methods_test/models/model.sql b/tests/integration/adapter_methods_test/models/model.sql
deleted file mode 100644
index bcf11d81c..000000000
--- a/tests/integration/adapter_methods_test/models/model.sql
+++ /dev/null
@@ -1,19 +0,0 @@
-
-{% set upstream = ref('upstream') %}
-
-{% if execute %}
-    {# don't ever do any of this #}
-    {%- do adapter.drop_schema(upstream) -%}
-    {% set existing = adapter.get_relation(upstream.database, upstream.schema, upstream.identifier) %}
-    {% if existing is not none %}
-        {% do exceptions.raise_compiler_error('expected ' ~ ' to not exist, but it did') %}
-    {% endif %}
-
-    {%- do adapter.create_schema(upstream) -%}
-
-    {% set sql = create_view_as(upstream, 'select 2 as id') %}
-    {% do run_query(sql) %}
-{% endif %}
-
-
-select * from {{ upstream }}
diff --git a/tests/integration/adapter_methods_test/models/upstream.sql b/tests/integration/adapter_methods_test/models/upstream.sql
deleted file mode 100644
index 43258a714..000000000
--- a/tests/integration/adapter_methods_test/models/upstream.sql
+++ /dev/null
@@ -1 +0,0 @@
-select 1 as id
diff --git a/tests/integration/adapter_methods_test/seed_bq.sql b/tests/integration/adapter_methods_test/seed_bq.sql
deleted file mode 100644
index 71a9a78c6..000000000
--- a/tests/integration/adapter_methods_test/seed_bq.sql
+++ /dev/null
@@ -1,32 +0,0 @@
-create table {database}.{schema}.seed (
-	id INT64,
-	first_name STRING,
-	last_name STRING,
-	email STRING,
-	gender STRING,
-	ip_address STRING,
-	updated_at TIMESTAMP
-);
-
--- seed inserts
-insert {database}.{schema}.seed (id, first_name, last_name, email, gender, ip_address, updated_at) values
-(1, 'Judith', 'Kennedy', 'jkennedy0@phpbb.com', 'Female', '54.60.24.128', '2015-12-24 12:19:28'),
-(2, 'Arthur', 'Kelly', 'akelly1@eepurl.com', 'Male', '62.56.24.215', '2015-10-28 16:22:15'),
-(3, 'Rachel', 'Moreno', 'rmoreno2@msu.edu', 'Female', '31.222.249.23', '2016-04-05 02:05:30'),
-(4, 'Ralph', 'Turner', 'rturner3@hp.com', 'Male', '157.83.76.114', '2016-08-08 00:06:51'),
-(5, 'Laura', 'Gonzales', 'lgonzales4@howstuffworks.com', 'Female', '30.54.105.168', '2016-09-01 08:25:38'),
-(6, 'Katherine', 'Lopez', 'klopez5@yahoo.co.jp', 'Female', '169.138.46.89', '2016-08-30 18:52:11'),
-(7, 'Jeremy', 'Hamilton', 'jhamilton6@mozilla.org', 'Male', '231.189.13.133', '2016-07-17 02:09:46'),
-(8, 'Heather', 'Rose', 'hrose7@goodreads.com', 'Female', '87.165.201.65', '2015-12-29 22:03:56'),
-(9, 'Gregory', 'Kelly', 'gkelly8@trellian.com', 'Male', '154.209.99.7', '2016-03-24 21:18:16'),
-(10, 'Rachel', 'Lopez', 'rlopez9@themeforest.net', 'Female', '237.165.82.71', '2016-08-20 15:44:49'),
-(11, 'Donna', 'Welch', 'dwelcha@shutterfly.com', 'Female', '103.33.110.138', '2016-02-27 01:41:48'),
-(12, 'Russell', 'Lawrence', 'rlawrenceb@qq.com', 'Male', '189.115.73.4', '2016-06-11 03:07:09'),
-(13, 'Michelle', 'Montgomery', 'mmontgomeryc@scientificamerican.com', 'Female', '243.220.95.82', '2016-06-18 16:27:19'),
-(14, 'Walter', 'Castillo', 'wcastillod@pagesperso-orange.fr', 'Male', '71.159.238.196', '2016-10-06 01:55:44'),
-(15, 'Robin', 'Mills', 'rmillse@vkontakte.ru', 'Female', '172.190.5.50', '2016-10-31 11:41:21'),
-(16, 'Raymond', 'Holmes', 'rholmesf@usgs.gov', 'Male', '148.153.166.95', '2016-10-03 08:16:38'),
-(17, 'Gary', 'Bishop', 'gbishopg@plala.or.jp', 'Male', '161.108.182.13', '2016-08-29 19:35:20'),
-(18, 'Anna', 'Riley', 'arileyh@nasa.gov', 'Female', '253.31.108.22', '2015-12-11 04:34:27'),
-(19, 'Sarah', 'Knight', 'sknighti@foxnews.com', 'Female', '222.220.3.177', '2016-09-26 00:49:06'),
-(20, 'Phyllis', 'Fox', null, 'Female', '163.191.232.95', '2016-08-21 10:35:19');
diff --git a/tests/integration/adapter_methods_test/test_adapter_methods.py b/tests/integration/adapter_methods_test/test_adapter_methods.py
deleted file mode 100644
index cc5137dea..000000000
--- a/tests/integration/adapter_methods_test/test_adapter_methods.py
+++ /dev/null
@@ -1,104 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-import yaml
-
-
-class TestBaseCaching(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "adapter_methods_caching"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'test-paths': ['tests']
-        }
-
-    @use_profile('bigquery')
-    def test_bigquery_adapter_methods(self):
-        self.run_dbt(['compile'])  # trigger any compile-time issues
-        self.run_dbt()
-        self.assertTablesEqual('model', 'expected')
-
-
-class TestRenameRelation(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "adapter_methods_rename_relation"
-
-    @property
-    def models(self):
-        return 'bigquery-models'
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'model-paths': ['models']
-        }
-
-    @use_profile('bigquery')
-    def test_bigquery_adapter_methods(self):
-        self.run_dbt(['compile'])  # trigger any compile-time issues
-        self.run_sql_file("seed_bq.sql")
-        self.run_dbt(['seed'])
-        rename_relation_args = yaml.safe_dump({
-            'from_name': 'seed',
-            'to_name': 'renamed_seed',
-        })
-        self.run_dbt(['run-operation', 'rename_named_relation', '--args', rename_relation_args])
-        self.run_dbt()
-
-
-class TestGrantAccess(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "adapter_methods_grant_access"
-
-    @property
-    def models(self):
-        return 'bigquery-models'
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'model-paths': ['models']
-        }
-
-    @use_profile('bigquery')
-    def test_bigquery_adapter_methods(self):
-        from dbt.adapters.bigquery import GrantTarget
-        from google.cloud.bigquery import AccessEntry
-
-        self.run_dbt(['compile'])  # trigger any compile-time issues
-        self.run_sql_file("seed_bq.sql")
-        self.run_dbt(['seed'])
-
-        ae_role = "READER"
-        ae_entity = "user@email.com"
-        ae_entity_type = "userByEmail"
-        ae_grant_target_dict = {
-            'project': self.default_database,
-            'dataset': self.unique_schema()
-        }
-        self.adapter.grant_access_to(ae_entity, ae_entity_type, ae_role, ae_grant_target_dict)
-
-        conn = self.adapter.connections.get_thread_connection()
-        client = conn.handle
-
-        grant_target = GrantTarget.from_dict(ae_grant_target_dict)
-        dataset_ref = self.adapter.connections.dataset_ref(
-            grant_target.project, grant_target.dataset
-        )
-        dataset = client.get_dataset(dataset_ref)
-
-        expected_access_entry = AccessEntry(ae_role, ae_entity_type, ae_entity)
-        self.assertTrue(expected_access_entry in dataset.access_entries)
-
-        unexpected_access_entry = AccessEntry(ae_role, ae_entity_type, "unexpected@email.com")
-        self.assertFalse(unexpected_access_entry in dataset.access_entries)
diff --git a/tests/integration/adapter_methods_test/tests/get_columns_in_relation.sql b/tests/integration/adapter_methods_test/tests/get_columns_in_relation.sql
deleted file mode 100644
index 6507b52ec..000000000
--- a/tests/integration/adapter_methods_test/tests/get_columns_in_relation.sql
+++ /dev/null
@@ -1,7 +0,0 @@
-{% set columns = adapter.get_columns_in_relation(ref('model')) %}
-{% set limit_query = 0 %}
-{% if (columns | length) == 0 %}
-	{% set limit_query = 1 %}
-{% endif %}
-
-select 1 as id limit {{ limit_query }}

From e70d873bcc06a13dbe28c9baacfba074554c36e2 Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Wed, 13 Apr 2022 11:27:20 -0400
Subject: [PATCH 438/860] Bumping version to 1.2.0a1 (#156)

* Bumping version to 1.2.0a1

* Remove extra space

* Skip test-build if alpha

* fix whitespace

* Bumping manifest schema

Co-authored-by: Github Build Bot 
Co-authored-by: leahwicz <60146280+leahwicz@users.noreply.github.com>
Co-authored-by: Jeremy Cohen 
---
 .bumpversion.cfg                                  |  2 +-
 .github/workflows/main.yml                        | 15 +++++++++++++++
 dbt/adapters/bigquery/__version__.py              |  2 +-
 setup.py                                          |  2 +-
 .../docs_generate_tests/test_docs_generate.py     |  6 +++---
 5 files changed, 21 insertions(+), 6 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 26bd8a191..78087456a 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.1.0b1
+current_version = 1.2.0a1
 parse = (?P\d+)
 	\.(?P\d+)
 	\.(?P\d+)
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 77e8e18e2..bc30363c1 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -114,6 +114,9 @@ jobs:
 
     runs-on: ubuntu-latest
 
+    outputs:
+      is_alpha: ${{ steps.check-is-alpha.outputs.is_alpha }}
+
     steps:
       - name: Check out the repository
         uses: actions/checkout@v2
@@ -130,6 +133,7 @@ jobs:
           pip install --user --upgrade pip
           pip install --upgrade setuptools wheel twine check-wheel-contents
           pip --version
+
       - name: Build distributions
         run: ./scripts/build-dist.sh
 
@@ -139,9 +143,18 @@ jobs:
       - name: Check distribution descriptions
         run: |
           twine check dist/*
+
       - name: Check wheel contents
         run: |
           check-wheel-contents dist/*.whl --ignore W007,W008
+
+      - name: Check if this is an alpha version
+        id: check-is-alpha
+        run: |
+          export is_alpha=0
+          if [[ "$(ls -lh dist/)" == *"a1"* ]]; then export is_alpha=1; fi
+          echo "::set-output name=is_alpha::$is_alpha"
+
       - uses: actions/upload-artifact@v2
         with:
           name: dist
@@ -150,6 +163,8 @@ jobs:
   test-build:
     name: verify packages / python ${{ matrix.python-version }} / ${{ matrix.os }}
 
+    if: needs.build.outputs.is_alpha == 0
+
     needs: build
 
     runs-on: ${{ matrix.os }}
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 56ec17a89..a6b977228 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.1.0b1"
+version = "1.2.0a1"
diff --git a/setup.py b/setup.py
index 3d1a95766..461e9746b 100644
--- a/setup.py
+++ b/setup.py
@@ -50,7 +50,7 @@ def _get_dbt_core_version():
 
 
 package_name = "dbt-bigquery"
-package_version = "1.1.0b1"
+package_version = "1.2.0a1"
 dbt_core_version = _get_dbt_core_version()
 description = """The BigQuery adapter plugin for dbt"""
 
diff --git a/tests/integration/docs_generate_tests/test_docs_generate.py b/tests/integration/docs_generate_tests/test_docs_generate.py
index 1ec52c3a2..0f62c025f 100644
--- a/tests/integration/docs_generate_tests/test_docs_generate.py
+++ b/tests/integration/docs_generate_tests/test_docs_generate.py
@@ -717,7 +717,7 @@ def expected_seeded_manifest(self, model_database=None, quote_model=False):
         )
 
         return {
-            'dbt_schema_version': 'https://schemas.getdbt.com/dbt/manifest/v4.json',
+            'dbt_schema_version': 'https://schemas.getdbt.com/dbt/manifest/v5.json',
             'dbt_version': dbt.version.__version__,
             'nodes': {
                 'model.test.model': {
@@ -1306,7 +1306,7 @@ def expected_bigquery_complex_manifest(self):
         my_schema_name = self.unique_schema()
 
         return {
-            'dbt_schema_version': 'https://schemas.getdbt.com/dbt/manifest/v4.json',
+            'dbt_schema_version': 'https://schemas.getdbt.com/dbt/manifest/v5.json',
             'dbt_version': dbt.version.__version__,
             'nodes': {
                 'model.test.clustered': {
@@ -1828,7 +1828,7 @@ def verify_manifest(self, expected_manifest):
             elif key == 'metadata':
                 metadata = manifest['metadata']
                 self.verify_metadata(
-                    metadata, 'https://schemas.getdbt.com/dbt/manifest/v4.json')
+                    metadata, 'https://schemas.getdbt.com/dbt/manifest/v5.json')
                 assert 'project_id' in metadata and metadata[
                     'project_id'] == '098f6bcd4621d373cade4e832627b4f6'
                 assert 'send_anonymous_usage_stats' in metadata and metadata[

From 8f7208c071c1834819ce9252ba68d6c7b94bf02c Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Thu, 14 Apr 2022 15:26:00 +0200
Subject: [PATCH 439/860] Set job_execution_timeout_seconds default to None
 (#159)

* Set job_execution_timeout_seconds default to None

* Pass old default of 300s in previous call sites

* Clearer factoring
---
 CHANGELOG.md                         | 7 ++++++-
 dbt/adapters/bigquery/connections.py | 7 ++++---
 dbt/adapters/bigquery/impl.py        | 4 ++--
 3 files changed, 12 insertions(+), 6 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 987bc79c6..528cd08b1 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,9 @@
-## dbt-bigquery 1.1.0 (TBD)
+## dbt-bigquery 1.1.0 (Release TBD)
+
+### Fixes
+- Restore default behavior for query timeout. Set default `job_execution_timeout` to `None` by default. Keep 300 seconds as query timeout where previously used.
+
+## dbt-bigquery 1.1.0rc1 (April 13, 2022)
 
 ### Under the hood
 - Use dbt.tests.adapter.basic in tests (new test framework) ([#135](https://github.com/dbt-labs/dbt-bigquery/issues/135), [#142](https://github.com/dbt-labs/dbt-bigquery/pull/142))
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 05f236a55..5a5b83044 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -103,7 +103,7 @@ class BigQueryCredentials(Credentials):
     job_retry_deadline_seconds: Optional[int] = None
     job_retries: Optional[int] = 1
     job_creation_timeout_seconds: Optional[int] = None
-    job_execution_timeout_seconds: Optional[int] = 300
+    job_execution_timeout_seconds: Optional[int] = None
 
     # Keyfile json creds
     keyfile: Optional[str] = None
@@ -301,7 +301,7 @@ def get_impersonated_bigquery_credentials(cls, profile_credentials):
             source_credentials=source_credentials,
             target_principal=profile_credentials.impersonate_service_account,
             target_scopes=list(profile_credentials.scopes),
-            lifetime=profile_credentials.job_execution_timeout_seconds,
+            lifetime=(profile_credentials.job_execution_timeout_seconds or 300),
         )
 
     @classmethod
@@ -524,7 +524,8 @@ def copy_bq_table(self, source, destination, write_disposition):
         def copy_and_results():
             job_config = google.cloud.bigquery.CopyJobConfig(write_disposition=write_disposition)
             copy_job = client.copy_table(source_ref_array, destination_ref, job_config=job_config)
-            iterator = copy_job.result(timeout=self.get_job_execution_timeout_seconds(conn))
+            timeout = self.get_job_execution_timeout_seconds(conn) or 300
+            iterator = copy_job.result(timeout=timeout)
             return copy_job, iterator
 
         self._retry_and_handle(
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 0fc5fc1cc..50ca3c6e1 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -624,7 +624,7 @@ def load_dataframe(self, database, schema, table_name, agate_table, column_overr
         with open(agate_table.original_abspath, "rb") as f:
             job = client.load_table_from_file(f, table_ref, rewind=True, job_config=load_config)
 
-        timeout = self.connections.get_job_execution_timeout_seconds(conn)
+        timeout = self.connections.get_job_execution_timeout_seconds(conn) or 300
         with self.connections.exception_handler("LOAD TABLE"):
             self.poll_until_job_completes(job, timeout)
 
@@ -647,7 +647,7 @@ def upload_file(
         with open(local_file_path, "rb") as f:
             job = client.load_table_from_file(f, table_ref, rewind=True, job_config=load_config)
 
-        timeout = self.connections.get_job_execution_timeout_seconds(conn)
+        timeout = self.connections.get_job_execution_timeout_seconds(conn) or 300
         with self.connections.exception_handler("LOAD TABLE"):
             self.poll_until_job_completes(job, timeout)
 

From 5253bf1579653c3f335de0f2d9e5558a748e7b8b Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Mon, 25 Apr 2022 11:13:04 +0200
Subject: [PATCH 440/860] Add oauth test target as default (#166)

---
 tests/conftest.py | 30 +++++++++++++++++++++++++++---
 tox.ini           |  2 +-
 2 files changed, 28 insertions(+), 4 deletions(-)

diff --git a/tests/conftest.py b/tests/conftest.py
index c423f8e03..4bbdb00e0 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -7,9 +7,33 @@
 
 pytest_plugins = ["dbt.tests.fixtures.project"]
 
-# The profile dictionary, used to write out profiles.yml
-@pytest.fixture(scope="class")
-def dbt_profile_target():
+
+def pytest_addoption(parser):
+    parser.addoption("--profile", action="store", default="oauth", type=str)
+
+
+@pytest.fixture(scope="session")
+def dbt_profile_target(request):
+    profile_type = request.config.getoption("--profile")
+    if profile_type == "oauth":
+        target = oauth_target()
+    elif profile_type == "service_account":
+        target = service_account_target()
+    else:
+        raise ValueError(f"Invalid profile type '{profile_type}'")
+    return target
+
+
+def oauth_target():
+    return {
+        'type': 'bigquery',
+        'method': 'oauth',
+        'threads': 1,
+        # project isn't needed if you configure a default, via 'gcloud config set project'
+    }
+
+
+def service_account_target():
     credentials_json_str = os.getenv('BIGQUERY_TEST_SERVICE_ACCOUNT_JSON').replace("'", '')
     credentials = json.loads(credentials_json_str)
     project_id = credentials.get('project_id')
diff --git a/tox.ini b/tox.ini
index fad7d7e34..8ddd4795c 100644
--- a/tox.ini
+++ b/tox.ini
@@ -17,7 +17,7 @@ skip_install = true
 passenv = DBT_* BIGQUERY_TEST_* PYTEST_ADDOPTS
 commands =
   bigquery: {envpython} -m pytest {posargs} -m profile_bigquery tests/integration
-  bigquery: {envpython} -m pytest {posargs} tests/functional
+  bigquery: {envpython} -m pytest {posargs} tests/functional --profile service_account
 deps =
   -rdev_requirements.txt
   -e.

From 8ef193a7ebe477f6fbbb01f8996851838f6bd441 Mon Sep 17 00:00:00 2001
From: Chenyu Li 
Date: Wed, 27 Apr 2022 09:11:58 -0600
Subject: [PATCH 441/860] fix validate connection failing (#168)

* fix validate connection failing

* fix code check

* more format
---
 dbt/adapters/bigquery/connections.py   | 22 +++++++++++++++++-----
 tests/functional/adapter/test_basic.py |  4 ++++
 2 files changed, 21 insertions(+), 5 deletions(-)

diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 5a5b83044..a445796b5 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -380,8 +380,11 @@ def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False):
         client = conn.handle
 
         fire_event(SQLQuery(conn_name=conn.name, sql=sql))
-
-        if self.profile.query_comment and self.profile.query_comment.job_label:
+        if (
+            hasattr(self.profile, "query_comment")
+            and self.profile.query_comment
+            and self.profile.query_comment.job_label
+        ):
             query_comment = self.query_header.comment.query_comment
             labels = self._labels_from_query_comment(query_comment)
         else:
@@ -475,7 +478,10 @@ def execute(
             message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
 
         response = BigQueryAdapterResponse(  # type: ignore[call-arg]
-            _message=message, rows_affected=num_rows, code=code, bytes_processed=bytes_processed
+            _message=message,
+            rows_affected=num_rows,
+            code=code,
+            bytes_processed=bytes_processed,
         )
 
         return response, table
@@ -530,7 +536,8 @@ def copy_and_results():
 
         self._retry_and_handle(
             msg='copy table "{}" to "{}"'.format(
-                ", ".join(source_ref.path for source_ref in source_ref_array), destination_ref.path
+                ", ".join(source_ref.path for source_ref in source_ref_array),
+                destination_ref.path,
             ),
             conn=conn,
             fn=copy_and_results,
@@ -572,7 +579,12 @@ def fn():
         self._retry_and_handle(msg="create dataset", conn=conn, fn=fn)
 
     def _query_and_results(
-        self, client, sql, job_params, job_creation_timeout=None, job_execution_timeout=None
+        self,
+        client,
+        sql,
+        job_params,
+        job_creation_timeout=None,
+        job_execution_timeout=None,
     ):
         """Query the client and wait for results."""
         # Cannot reuse job_config if destination is set and ddl is used
diff --git a/tests/functional/adapter/test_basic.py b/tests/functional/adapter/test_basic.py
index fceef2dee..2c07a6e98 100644
--- a/tests/functional/adapter/test_basic.py
+++ b/tests/functional/adapter/test_basic.py
@@ -12,6 +12,7 @@
 from dbt.tests.adapter.basic.test_snapshot_check_cols import BaseSnapshotCheckCols
 from dbt.tests.adapter.basic.test_snapshot_timestamp import BaseSnapshotTimestamp
 from dbt.tests.adapter.basic.test_adapter_methods import BaseAdapterMethod
+from dbt.tests.adapter.basic.test_validate_connection import BaseValidateConnection
 
 
 class TestSimpleMaterializationsBigQuery(BaseSimpleMaterializations):
@@ -54,3 +55,6 @@ class TestSnapshotTimestampBigQuery(BaseSnapshotTimestamp):
 
 class TestBaseAdapterMethodBigQuery(BaseAdapterMethod):
     pass
+
+class TestBigQueryValidateConnection(BaseValidateConnection):
+    pass
\ No newline at end of file

From 3bbd76b328f25ba919a99d40c08eed37fc8c1b28 Mon Sep 17 00:00:00 2001
From: robomill <84311571+robomill@users.noreply.github.com>
Date: Tue, 3 May 2022 21:31:57 +0200
Subject: [PATCH 442/860] incorrect parameter is passed to the
 bq_insert_overwrite macro call in the bq_generate_incremental_build_sql macro
 causing two tmp tables to be created (#172)

---
 dbt/include/bigquery/macros/materializations/incremental.sql | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index f9d36aead..b6d387890 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -109,7 +109,7 @@
     {% endif %}
 
     {% set build_sql = bq_insert_overwrite(
-        tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, on_schema_change
+        tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
     ) %}
 
   {% else %} {# strategy == 'merge' #}

From 15f35b4ff7658fff0783b40fb7b0347344692f4c Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 5 May 2022 13:05:03 -0500
Subject: [PATCH 443/860] fixing filename in contributing.md (#149)

* fixing filename in contributing.md

* rename dev_requirements.txt -> dev-requirements.txt to match dbt-core

* catch all refs to dev_requirements for rename
---
 .github/workflows/main.yml                   | 2 +-
 .github/workflows/version-bump.yml           | 2 +-
 dev_requirements.txt => dev-requirements.txt | 0
 tox.ini                                      | 4 ++--
 4 files changed, 4 insertions(+), 4 deletions(-)
 rename dev_requirements.txt => dev-requirements.txt (100%)

diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index bc30363c1..d2ed07654 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -53,7 +53,7 @@ jobs:
       - name: Install python dependencies
         run: |
           pip install --user --upgrade pip
-          pip install -r dev_requirements.txt
+          pip install -r dev-requirements.txt
           pip --version
           pre-commit --version
           mypy --version
diff --git a/.github/workflows/version-bump.yml b/.github/workflows/version-bump.yml
index b0a3174df..dbd36786a 100644
--- a/.github/workflows/version-bump.yml
+++ b/.github/workflows/version-bump.yml
@@ -69,7 +69,7 @@ jobs:
       - name: Bumping version
         run: |
           source env/bin/activate
-          pip install -r dev_requirements.txt
+          pip install -r dev-requirements.txt
           env/bin/bumpversion --allow-dirty --new-version ${{steps.variables.outputs.VERSION_NUMBER}} major
           git status
 
diff --git a/dev_requirements.txt b/dev-requirements.txt
similarity index 100%
rename from dev_requirements.txt
rename to dev-requirements.txt
diff --git a/tox.ini b/tox.ini
index 8ddd4795c..a5da0d0a6 100644
--- a/tox.ini
+++ b/tox.ini
@@ -8,7 +8,7 @@ skip_install = true
 passenv = DBT_* PYTEST_ADDOPTS
 commands = {envpython} -m pytest {posargs} tests/unit
 deps =
-  -rdev_requirements.txt
+  -rdev-requirements.txt
   -e.
 
 [testenv:{integration,py37,py38,py39,py}-{bigquery}]
@@ -19,5 +19,5 @@ commands =
   bigquery: {envpython} -m pytest {posargs} -m profile_bigquery tests/integration
   bigquery: {envpython} -m pytest {posargs} tests/functional --profile service_account
 deps =
-  -rdev_requirements.txt
+  -rdev-requirements.txt
   -e.

From 8aaa6ab6da182aa8e831f631feb12af5ec76d9ce Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 11 May 2022 11:20:26 -0500
Subject: [PATCH 444/860] init push or pytest conversion of override_database
 for bigquery (#165)

* init push or pytest conversion of override_database for bigquery

* tests are currently passing, need to implement some test changes overs (commented out tests)

* implemented a vew versions of the check_realations_equal test on test_bigquery_database_override test working on getting the rest to work for first test

* trying to grab local env to test against need to find way to grab information from configs

* attempting to move over to using check_relations_equal_with_relations

* ordering of models

* post pair update

* debugged test, had passing locally

* adding change to conftest to see if it passes tests in ci/cd

* removing conftest change

* trying ci/cd after some changes

* trying to take into account CT-604, remove old test, remove BIGQUERY_TEST_DATABASE env

* moving override_database out of adapter folder up one
---
 dbt/adapters/bigquery/connections.py          |   3 +
 test.env.example                              |   1 -
 .../test_override_database/fixtures.py        |  71 +++++++
 .../test_override_database.py                 | 158 ++++++++++++++++
 .../models/subfolder/view_3.sql               |   1 -
 .../models/subfolder/view_4.sql               |   5 -
 .../override_database_test/models/view_1.sql  |   7 -
 .../override_database_test/models/view_2.sql  |   6 -
 .../override_database_test/seeds/seed.csv     |   6 -
 .../test_override_database.py                 | 176 ------------------
 10 files changed, 232 insertions(+), 202 deletions(-)
 create mode 100644 tests/functional/test_override_database/fixtures.py
 create mode 100644 tests/functional/test_override_database/test_override_database.py
 delete mode 100644 tests/integration/override_database_test/models/subfolder/view_3.sql
 delete mode 100644 tests/integration/override_database_test/models/subfolder/view_4.sql
 delete mode 100644 tests/integration/override_database_test/models/view_1.sql
 delete mode 100644 tests/integration/override_database_test/models/view_2.sql
 delete mode 100644 tests/integration/override_database_test/seeds/seed.csv
 delete mode 100644 tests/integration/override_database_test/test_override_database.py

diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index a445796b5..454e84d7d 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -555,6 +555,9 @@ def table_ref(database, schema, table_name):
     def get_bq_table(self, database, schema, identifier):
         """Get a bigquery table for a schema/model."""
         conn = self.get_thread_connection()
+        # backwards compatibility: fill in with defaults if not specified
+        database = database or conn.credentials.database
+        schema = schema or conn.credentials.schema
         table_ref = self.table_ref(database, schema, identifier)
         return conn.handle.get_table(table_ref)
 
diff --git a/test.env.example b/test.env.example
index 031968c60..2065e4393 100644
--- a/test.env.example
+++ b/test.env.example
@@ -1,4 +1,3 @@
-BIGQUERY_TEST_DATABASE=
 BIGQUERY_TEST_ALT_DATABASE=
 BIGQUERY_TEST_NO_ACCESS_DATABASE=
 BIGQUERY_TEST_SERVICE_ACCOUNT_JSON='{}'
diff --git a/tests/functional/test_override_database/fixtures.py b/tests/functional/test_override_database/fixtures.py
new file mode 100644
index 000000000..315a74e8f
--- /dev/null
+++ b/tests/functional/test_override_database/fixtures.py
@@ -0,0 +1,71 @@
+import pytest
+from dbt.tests.fixtures.project import write_project_files
+
+
+models__view_2_sql = """
+{%- if target.type == 'bigquery' -%}
+  {{ config(project=var('alternate_db')) }}
+{%- else -%}
+  {{ config(database=var('alternate_db')) }}
+{%- endif -%}
+select * from {{ ref('seed') }}
+
+"""
+
+models__view_1_sql = """
+{#
+	We are running against a database that must be quoted.
+	These calls ensure that we trigger an error if we're failing to quote at parse-time
+#}
+{% do adapter.already_exists(this.schema, this.table) %}
+{% do adapter.get_relation(this.database, this.schema, this.table) %}
+select * from {{ ref('seed') }}
+
+"""
+
+models__subfolder__view_4_sql = """
+{{
+    config(database=var('alternate_db'))
+}}
+
+select * from {{ ref('seed') }}
+
+"""
+
+models__subfolder__view_3_sql = """
+select * from {{ ref('seed') }}
+
+"""
+
+seeds__seed_csv = """id,name
+1,a
+2,b
+3,c
+4,d
+5,e
+"""
+
+@pytest.fixture(scope="class")
+def models():
+    return {
+      "view_2.sql": models__view_2_sql,
+      "view_1.sql": models__view_1_sql,
+      "subfolder":
+        {
+          "view_4.sql": models__subfolder__view_4_sql,
+          "view_3.sql": models__subfolder__view_3_sql,
+          },
+    }
+
+@pytest.fixture(scope="class")
+def seeds():
+    return {
+      "seed.csv": seeds__seed_csv
+      }
+
+@pytest.fixture(scope="class")
+def project_files(project_root, models, seeds,):
+    write_project_files(project_root, "models", models)
+    write_project_files(project_root, "seeds", seeds)
+
+
diff --git a/tests/functional/test_override_database/test_override_database.py b/tests/functional/test_override_database/test_override_database.py
new file mode 100644
index 000000000..8ce179056
--- /dev/null
+++ b/tests/functional/test_override_database/test_override_database.py
@@ -0,0 +1,158 @@
+import pytest
+from dbt.tests.util import run_dbt, check_relations_equal, check_relations_equal_with_relations
+from tests.functional.test_override_database.fixtures import (
+    models,
+    seeds,
+    project_files
+)
+import os
+
+
+
+
+class BaseOverrideDatabase:
+    @pytest.fixture(scope="class")
+    def model_path(self):
+        return "models"
+
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            "config-version": 2,
+            "seed-paths": ["seeds"],
+            "vars": {
+                "alternate_db": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+            },
+            "quoting": {
+                "database": True,
+            },
+            "seeds": {
+                "quote_columns": False,
+            }
+        }
+
+
+class TestModelOverrideBigQuery(BaseOverrideDatabase):
+    def run_database_override(self, project):
+        run_dbt(["seed"])
+        assert len(run_dbt(["run"])) == 4
+        check_relations_equal_with_relations(project.adapter, [
+            project.adapter.Relation.create(schema=project.test_schema, identifier="seed"),
+            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_2"),
+            project.adapter.Relation.create(schema=project.test_schema, identifier="view_1"),
+            project.adapter.Relation.create(schema=project.test_schema, identifier="view_3"),
+            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_4")
+        ])
+
+
+    def test_bigquery_database_override(self, project):
+        self.run_database_override(project)
+
+
+class BaseTestProjectModelOverrideBigQuery(BaseOverrideDatabase):
+
+    def run_database_override(self, project):
+        run_dbt(["seed"])
+        assert len(run_dbt(["run"])) == 4
+        self.assertExpectedRelations(project)
+
+    def assertExpectedRelations(self, project):
+        check_relations_equal_with_relations(project.adapter, [
+            project.adapter.Relation.create(schema=project.test_schema, identifier="seed"),
+            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_2"),
+            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_1"),
+            project.adapter.Relation.create(schema=project.test_schema, identifier="view_3"),
+            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_4")
+        ])
+
+
+class TestProjectModelOverrideBigQuery(BaseTestProjectModelOverrideBigQuery):
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            "config-version": 2,
+            "vars": {
+                "alternate_db": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+            },
+            "models": {
+                "database": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+                "test": {
+                    "subfolder": {
+                        "database": "{{ target.database }}"
+                    }
+                }
+            },
+            "seed-paths": ["seeds"],
+            "vars": {
+                "alternate_db": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+            },
+            "quoting": {
+                "database": True,
+            },
+            "seeds": {
+                "quote_columns": False,
+            }
+        }
+
+    def test_bigquery_database_override(self, project):
+        self.run_database_override(project)
+
+
+class TestProjectModelAliasOverrideBigQuery(BaseTestProjectModelOverrideBigQuery):
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            "config-version": 2,
+            "vars": {
+                "alternate_db": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+            },
+            "models": {
+                "project": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+                "test": {
+                    "subfolder": {
+                        "project": "{{ target.database }}"
+                    }
+                }
+            },
+            "seed-paths": ["seeds"],
+            "vars": {
+                "alternate_db": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+            },
+            "quoting": {
+                "database": True,
+            },
+            "seeds": {
+                "quote_columns": False,
+            }
+        }
+
+    def test_bigquery_project_override(self, project):
+        self.run_database_override(project)
+
+
+class TestProjectSeedOverrideBigQuery(BaseOverrideDatabase):
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            "config-version": 2,
+            "seed-paths": ["seeds"],
+            "vars": {
+                "alternate_db": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+            },
+            "seeds": {
+                "database": os.getenv("BIGQUERY_TEST_ALT_DATABASE")
+            }
+        }
+    def run_database_override(self, project):
+        run_dbt(["seed"])
+        assert len(run_dbt(["run"])) == 4
+        check_relations_equal_with_relations(project.adapter, [
+            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="seed"),
+            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_2"),
+            project.adapter.Relation.create(schema=project.test_schema, identifier="view_1"),
+            project.adapter.Relation.create(schema=project.test_schema, identifier="view_3"),
+            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_4")
+        ])
+
+    def test_bigquery_database_override(self, project):
+        self.run_database_override(project)
diff --git a/tests/integration/override_database_test/models/subfolder/view_3.sql b/tests/integration/override_database_test/models/subfolder/view_3.sql
deleted file mode 100644
index 4b91aa0f2..000000000
--- a/tests/integration/override_database_test/models/subfolder/view_3.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('seed') }}
diff --git a/tests/integration/override_database_test/models/subfolder/view_4.sql b/tests/integration/override_database_test/models/subfolder/view_4.sql
deleted file mode 100644
index efa1268fa..000000000
--- a/tests/integration/override_database_test/models/subfolder/view_4.sql
+++ /dev/null
@@ -1,5 +0,0 @@
-{{
-    config(database=var('alternate_db'))
-}}
-
-select * from {{ ref('seed') }}
diff --git a/tests/integration/override_database_test/models/view_1.sql b/tests/integration/override_database_test/models/view_1.sql
deleted file mode 100644
index a43f04646..000000000
--- a/tests/integration/override_database_test/models/view_1.sql
+++ /dev/null
@@ -1,7 +0,0 @@
-{#
-	We are running against a database that must be quoted.
-	These calls ensure that we trigger an error if we're failing to quote at parse-time
-#}
-{% do adapter.already_exists(this.schema, this.table) %}
-{% do adapter.get_relation(this.database, this.schema, this.table) %}
-select * from {{ ref('seed') }}
diff --git a/tests/integration/override_database_test/models/view_2.sql b/tests/integration/override_database_test/models/view_2.sql
deleted file mode 100644
index 9ac6bdad6..000000000
--- a/tests/integration/override_database_test/models/view_2.sql
+++ /dev/null
@@ -1,6 +0,0 @@
-{%- if target.type == 'bigquery' -%}
-  {{ config(project=var('alternate_db')) }}
-{%- else -%}
-  {{ config(database=var('alternate_db')) }}
-{%- endif -%}
-select * from {{ ref('seed') }}
diff --git a/tests/integration/override_database_test/seeds/seed.csv b/tests/integration/override_database_test/seeds/seed.csv
deleted file mode 100644
index ae9125c16..000000000
--- a/tests/integration/override_database_test/seeds/seed.csv
+++ /dev/null
@@ -1,6 +0,0 @@
-id,name
-1,a
-2,b
-3,c
-4,d
-5,e
diff --git a/tests/integration/override_database_test/test_override_database.py b/tests/integration/override_database_test/test_override_database.py
deleted file mode 100644
index 9b12b3e79..000000000
--- a/tests/integration/override_database_test/test_override_database.py
+++ /dev/null
@@ -1,176 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-import os
-
-
-class BaseOverrideDatabase(DBTIntegrationTest):
-    setup_alternate_db = True
-    @property
-    def schema(self):
-        return "override_database"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def alternative_database(self):
-        return super().alternative_database
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds'],
-            'vars': {
-                'alternate_db': self.alternative_database,
-            },
-            'quoting': {
-                'database': True,
-            },
-            'seeds': {
-                'quote_columns': False,
-            }
-        }
-
-
-class TestModelOverride(BaseOverrideDatabase):
-    def run_database_override(self):
-        func = lambda x: x
-
-        self.run_dbt(['seed'])
-
-        self.assertEqual(len(self.run_dbt(['run'])), 4)
-        self.assertManyRelationsEqual([
-            (func('seed'), self.unique_schema(), self.default_database),
-            (func('view_2'), self.unique_schema(), self.alternative_database),
-            (func('view_1'), self.unique_schema(), self.default_database),
-            (func('view_3'), self.unique_schema(), self.default_database),
-            (func('view_4'), self.unique_schema(), self.alternative_database),
-        ])
-
-    @use_profile('bigquery')
-    def test_bigquery_database_override(self):
-        self.run_database_override()
-
-
-class BaseTestProjectModelOverride(BaseOverrideDatabase):
-    # this is janky, but I really want to access self.default_database in
-    # project_config
-    @property
-    def default_database(self):
-        target = self._profile_config['test']['target']
-        profile = self._profile_config['test']['outputs'][target]
-        for key in ['database', 'project', 'dbname']:
-            if key in profile:
-                database = profile[key]
-                return database
-        assert False, 'No profile database found!'
-
-    def run_database_override(self):
-        self.run_dbt(['seed'])
-        self.assertEqual(len(self.run_dbt(['run'])), 4)
-        self.assertExpectedRelations()
-
-    def assertExpectedRelations(self):
-        func = lambda x: x
-
-        self.assertManyRelationsEqual([
-            (func('seed'), self.unique_schema(), self.default_database),
-            (func('view_2'), self.unique_schema(), self.alternative_database),
-            (func('view_1'), self.unique_schema(), self.alternative_database),
-            (func('view_3'), self.unique_schema(), self.default_database),
-            (func('view_4'), self.unique_schema(), self.alternative_database),
-        ])
-
-
-class TestProjectModelOverride(BaseTestProjectModelOverride):
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'vars': {
-                'alternate_db': self.alternative_database,
-            },
-            'models': {
-                'database': self.alternative_database,
-                'test': {
-                    'subfolder': {
-                        'database': self.default_database,
-                    }
-                }
-            },
-            'seed-paths': ['seeds'],
-            'vars': {
-                'alternate_db': self.alternative_database,
-            },
-            'quoting': {
-                'database': True,
-            },
-            'seeds': {
-                'quote_columns': False,
-            }
-        }
-
-    @use_profile('bigquery')
-    def test_bigquery_database_override(self):
-        self.run_database_override()
-
-
-class TestProjectModelAliasOverride(BaseTestProjectModelOverride):
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'vars': {
-                'alternate_db': self.alternative_database,
-            },
-            'models': {
-                'project': self.alternative_database,
-                'test': {
-                    'subfolder': {
-                        'project': self.default_database,
-                    }
-                }
-            },
-            'seed-paths': ['seeds'],
-            'vars': {
-                'alternate_db': self.alternative_database,
-            },
-            'quoting': {
-                'database': True,
-            },
-            'seeds': {
-                'quote_columns': False,
-            }
-        }
-
-    @use_profile('bigquery')
-    def test_bigquery_project_override(self):
-        self.run_database_override()
-
-
-class TestProjectSeedOverride(BaseOverrideDatabase):
-    def run_database_override(self):
-        func = lambda x: x
-
-        self.use_default_project({
-            'config-version': 2,
-            'seeds': {
-                'database': self.alternative_database
-            },
-        })
-        self.run_dbt(['seed'])
-
-        self.assertEqual(len(self.run_dbt(['run'])), 4)
-        self.assertManyRelationsEqual([
-            (func('seed'), self.unique_schema(), self.alternative_database),
-            (func('view_2'), self.unique_schema(), self.alternative_database),
-            (func('view_1'), self.unique_schema(), self.default_database),
-            (func('view_3'), self.unique_schema(), self.default_database),
-            (func('view_4'), self.unique_schema(), self.alternative_database),
-        ])
-
-    @use_profile('bigquery')
-    def test_bigquery_database_override(self):
-        self.run_database_override()

From 824052240a14d78b74f7e33a17dc8529d6326199 Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Wed, 18 May 2022 17:45:03 -0400
Subject: [PATCH 445/860] Adding Python 3.10 to test matrix  (#177)

* Updating to test python 3.10

* Fixing mypy issues from upgrade

* Extending path in init

* Adding newline

* Updating changelog

* Adding response stubs
---
 .github/scripts/integration-test-matrix.js |  2 +-
 .github/workflows/main.yml                 |  4 +-
 .pre-commit-config.yaml                    |  2 +-
 CHANGELOG.md                               |  7 +-
 dbt/__init__.py                            |  3 +
 dbt/adapters/bigquery/impl.py              |  2 +-
 dev-requirements.txt                       |  3 +-
 mypy.ini                                   |  3 +-
 setup.py                                   |  1 +
 tests/integration/base.py                  |  8 ++
 third-party-stubs/agate/__init__.pyi       | 89 ++++++++++++++++++++++
 third-party-stubs/agate/data_types.pyi     | 71 +++++++++++++++++
 tox.ini                                    |  6 +-
 13 files changed, 190 insertions(+), 11 deletions(-)
 create mode 100644 dbt/__init__.py
 create mode 100644 third-party-stubs/agate/__init__.pyi
 create mode 100644 third-party-stubs/agate/data_types.pyi

diff --git a/.github/scripts/integration-test-matrix.js b/.github/scripts/integration-test-matrix.js
index 6a33653a2..1e3bb0f0d 100644
--- a/.github/scripts/integration-test-matrix.js
+++ b/.github/scripts/integration-test-matrix.js
@@ -1,6 +1,6 @@
 module.exports = ({ context }) => {
   const defaultPythonVersion = "3.8";
-  const supportedPythonVersions = ["3.7", "3.8", "3.9"];
+  const supportedPythonVersions = ["3.7", "3.8", "3.9", "3.10"];
   const supportedAdapters = ["bigquery"];
 
   // if PR, generate matrix based on files changed and PR labels
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index d2ed07654..2e961dfa7 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -69,7 +69,7 @@ jobs:
     strategy:
       fail-fast: false
       matrix:
-        python-version: [3.7, 3.8] # TODO: support unit testing for python 3.9 (https://github.com/dbt-labs/dbt/issues/3689)
+        python-version: ['3.7', '3.8', '3.9', '3.10']
 
     env:
       TOXENV: "unit"
@@ -173,7 +173,7 @@ jobs:
       fail-fast: false
       matrix:
         os: [ubuntu-latest, macos-latest, windows-latest]
-        python-version: [3.7, 3.8, 3.9]
+        python-version: ['3.7', '3.8', '3.9', '3.10']
 
     steps:
       - name: Set up Python ${{ matrix.python-version }}
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index 9d247581b..cf4ddf68a 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -39,7 +39,7 @@ repos:
     alias: flake8-check
     stages: [manual]
 - repo: https://github.com/pre-commit/mirrors-mypy
-  rev: v0.782
+  rev: v0.942
   hooks:
   - id: mypy
     args: [--show-error-codes, --ignore-missing-imports]
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 528cd08b1..be153ce60 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,9 @@
-## dbt-bigquery 1.1.0 (Release TBD)
+## dbt-bigquery 1.2.0 (Release TBD)
+- Adding Python 3.10 testing and enabling mypy ([#177](https://github.com/dbt-labs/dbt-bigquery/pull/177))
+
+## dbt-bigquery 1.1.0 (April 28, 2022)
+
+## dbt-bigquery 1.1.0rc2 (April 20, 2022)
 
 ### Fixes
 - Restore default behavior for query timeout. Set default `job_execution_timeout` to `None` by default. Keep 300 seconds as query timeout where previously used.
diff --git a/dbt/__init__.py b/dbt/__init__.py
new file mode 100644
index 000000000..b36383a61
--- /dev/null
+++ b/dbt/__init__.py
@@ -0,0 +1,3 @@
+from pkgutil import extend_path
+
+__path__ = extend_path(__path__, __name__)
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 50ca3c6e1..344ffb22e 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -296,7 +296,7 @@ def convert_text_type(cls, agate_table: agate.Table, col_idx: int) -> str:
 
     @classmethod
     def convert_number_type(cls, agate_table: agate.Table, col_idx: int) -> str:
-        decimals = agate_table.aggregate(agate.MaxPrecision(col_idx))
+        decimals = agate_table.aggregate(agate.MaxPrecision(col_idx))  # type: ignore[attr-defined]
         return "float64" if decimals else "int64"
 
     @classmethod
diff --git a/dev-requirements.txt b/dev-requirements.txt
index ff7b6522b..09cf7f7c2 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -9,7 +9,7 @@ flake8
 flaky
 freezegun==1.1.0
 ipdb
-mypy==0.782
+mypy==0.942
 pip-tools
 pre-commit
 pytest
@@ -19,5 +19,6 @@ pytest-csv
 pytest-xdist
 pytz
 tox>=3.13
+types-requests
 twine
 wheel
diff --git a/mypy.ini b/mypy.ini
index 51fada1b1..b111482fc 100644
--- a/mypy.ini
+++ b/mypy.ini
@@ -1,3 +1,4 @@
 [mypy]
-mypy_path = ./third-party-stubs
+mypy_path = third-party-stubs/
 namespace_packages = True
+exclude = third-party-stubs/*
diff --git a/setup.py b/setup.py
index 461e9746b..0adf95f3b 100644
--- a/setup.py
+++ b/setup.py
@@ -83,6 +83,7 @@ def _get_dbt_core_version():
         "Programming Language :: Python :: 3.7",
         "Programming Language :: Python :: 3.8",
         "Programming Language :: Python :: 3.9",
+        "Programming Language :: Python :: 3.10",
     ],
     python_requires=">=3.7",
 )
diff --git a/tests/integration/base.py b/tests/integration/base.py
index 243ac92d8..ec6769e8e 100644
--- a/tests/integration/base.py
+++ b/tests/integration/base.py
@@ -7,6 +7,7 @@
 import tempfile
 import traceback
 import unittest
+import warnings
 from contextlib import contextmanager
 from datetime import datetime
 from functools import wraps
@@ -230,6 +231,13 @@ def _generate_test_root_dir(self):
         return normalize(tempfile.mkdtemp(prefix='dbt-int-test-'))
 
     def setUp(self):
+        # Logbook warnings are ignored so we don't have to fork logbook to support python 3.10. 
+        # This _only_ works for tests in `test/integration`.
+        warnings.filterwarnings(
+            "ignore",
+            category=DeprecationWarning,
+            module="logbook"
+        )
         self.dbt_core_install_root = os.path.dirname(dbt.__file__)
         log_manager.reset_handlers()
         self.initial_dir = INITIAL_ROOT
diff --git a/third-party-stubs/agate/__init__.pyi b/third-party-stubs/agate/__init__.pyi
new file mode 100644
index 000000000..c773cc7d7
--- /dev/null
+++ b/third-party-stubs/agate/__init__.pyi
@@ -0,0 +1,89 @@
+from collections.abc import Sequence
+
+from typing import Any, Optional, Callable, Iterable, Dict, Union
+
+from . import data_types as data_types
+from .data_types import (
+    Text as Text,
+    Number as Number,
+    Boolean as Boolean,
+    DateTime as DateTime,
+    Date as Date,
+    TimeDelta as TimeDelta,
+)
+
+class MappedSequence(Sequence):
+    def __init__(self, values: Any, keys: Optional[Any] = ...) -> None: ...
+    def __unicode__(self): ...
+    def __getitem__(self, key: Any): ...
+    def __setitem__(self, key: Any, value: Any) -> None: ...
+    def __iter__(self): ...
+    def __len__(self): ...
+    def __eq__(self, other: Any): ...
+    def __ne__(self, other: Any): ...
+    def __contains__(self, value: Any): ...
+    def keys(self): ...
+    def values(self): ...
+    def items(self): ...
+    def get(self, key: Any, default: Optional[Any] = ...): ...
+    def dict(self): ...
+
+class Row(MappedSequence): ...
+
+class Table:
+    def __init__(
+        self,
+        rows: Any,
+        column_names: Optional[Any] = ...,
+        column_types: Optional[Any] = ...,
+        row_names: Optional[Any] = ...,
+        _is_fork: bool = ...,
+    ) -> None: ...
+    def __len__(self): ...
+    def __iter__(self): ...
+    def __getitem__(self, key: Any): ...
+    @property
+    def column_types(self): ...
+    @property
+    def column_names(self): ...
+    @property
+    def row_names(self): ...
+    @property
+    def columns(self): ...
+    @property
+    def rows(self): ...
+    def print_csv(self, **kwargs: Any) -> None: ...
+    def print_json(self, **kwargs: Any) -> None: ...
+    def where(self, test: Callable[[Row], bool]) -> "Table": ...
+    def select(self, key: Union[Iterable[str], str]) -> "Table": ...
+    # these definitions are much narrower than what's actually accepted
+    @classmethod
+    def from_object(
+        cls, obj: Iterable[Dict[str, Any]], *, column_types: Optional["TypeTester"] = None
+    ) -> "Table": ...
+    @classmethod
+    def from_csv(
+        cls, path: Iterable[str], *, column_types: Optional["TypeTester"] = None
+    ) -> "Table": ...
+    @classmethod
+    def merge(cls, tables: Iterable["Table"]) -> "Table": ...
+    def rename(
+        self,
+        column_names: Optional[Iterable[str]] = None,
+        row_names: Optional[Any] = None,
+        slug_columns: bool = False,
+        slug_rows: bool = False,
+        **kwargs: Any,
+    ) -> "Table": ...
+
+class TypeTester:
+    def __init__(
+        self, force: Any = ..., limit: Optional[Any] = ..., types: Optional[Any] = ...
+    ) -> None: ...
+    def run(self, rows: Any, column_names: Any): ...
+
+class MaxPrecision:
+    def __init__(self, column_name: Any) -> None: ...
+
+# this is not strictly true, but it's all we care about.
+def aggregate(self, aggregations: MaxPrecision) -> int: ...
diff --git a/third-party-stubs/agate/data_types.pyi b/third-party-stubs/agate/data_types.pyi
new file mode 100644
index 000000000..8114f7b55
--- /dev/null
+++ b/third-party-stubs/agate/data_types.pyi
@@ -0,0 +1,71 @@
+from typing import Any, Optional
+
+DEFAULT_NULL_VALUES: Any
+
+class DataType:
+    null_values: Any = ...
+    def __init__(self, null_values: Any = ...) -> None: ...
+    def test(self, d: Any): ...
+    def cast(self, d: Any) -> None: ...
+    def csvify(self, d: Any): ...
+    def jsonify(self, d: Any): ...
+
+DEFAULT_TRUE_VALUES: Any
+DEFAULT_FALSE_VALUES: Any
+
+class Boolean(DataType):
+    true_values: Any = ...
+    false_values: Any = ...
+    def __init__(
+        self, true_values: Any = ..., false_values: Any = ..., null_values: Any = ...
+    ) -> None: ...
+    def cast(self, d: Any): ...
+    def jsonify(self, d: Any): ...
+
+ZERO_DT: Any
+
+class Date(DataType):
+    date_format: Any = ...
+    parser: Any = ...
+    def __init__(self, date_format: Optional[Any] = ..., **kwargs: Any) -> None: ...
+    def cast(self, d: Any): ...
+    def csvify(self, d: Any): ...
+    def jsonify(self, d: Any): ...
+
+class DateTime(DataType):
+    datetime_format: Any = ...
+    timezone: Any = ...
+    def __init__(
+        self, datetime_format: Optional[Any] = ..., timezone: Optional[Any] = ..., **kwargs: Any
+    ) -> None: ...
+    def cast(self, d: Any): ...
+    def csvify(self, d: Any): ...
+    def jsonify(self, d: Any): ...
+
+DEFAULT_CURRENCY_SYMBOLS: Any
+POSITIVE: Any
+NEGATIVE: Any
+
+class Number(DataType):
+    locale: Any = ...
+    currency_symbols: Any = ...
+    group_symbol: Any = ...
+    decimal_symbol: Any = ...
+    def __init__(
+        self,
+        locale: str = ...,
+        group_symbol: Optional[Any] = ...,
+        decimal_symbol: Optional[Any] = ...,
+        currency_symbols: Any = ...,
+        **kwargs: Any,
+    ) -> None: ...
+    def cast(self, d: Any): ...
+    def jsonify(self, d: Any): ...
+
+class TimeDelta(DataType):
+    def cast(self, d: Any): ...
+
+class Text(DataType):
+    cast_nulls: Any = ...
+    def __init__(self, cast_nulls: bool = ..., **kwargs: Any) -> None: ...
+    def cast(self, d: Any): ...
diff --git a/tox.ini b/tox.ini
index a5da0d0a6..348be15af 100644
--- a/tox.ini
+++ b/tox.ini
@@ -1,8 +1,8 @@
 [tox]
 skipsdist = True
-envlist = py37,py38,py39
+envlist = py37,py38,py39,py310
 
-[testenv:{unit,py37,py38,py39,py}]
+[testenv:{unit,py37,py38,py39,py310,py}]
 description = unit testing
 skip_install = true
 passenv = DBT_* PYTEST_ADDOPTS
@@ -11,7 +11,7 @@ deps =
   -rdev-requirements.txt
   -e.
 
-[testenv:{integration,py37,py38,py39,py}-{bigquery}]
+[testenv:{integration,py37,py38,py39,py310,py}-{bigquery}]
 description = adapter plugin integration testing
 skip_install = true
 passenv = DBT_* BIGQUERY_TEST_* PYTEST_ADDOPTS

From 69d1bf211199202642e0b3965b00f70f1d77e7b2 Mon Sep 17 00:00:00 2001
From: Gerda Shank 
Date: Mon, 23 May 2022 18:39:06 -0400
Subject: [PATCH 446/860] Implement TestDocsGenerateBigQuery test (#190)

* Implement TestDocsGenerateBigQuery test

* Remove legacy docs_generate tests

* Add -vv to get info on failing test
---
 tests/functional/adapter/expected_stats.py    |   62 +
 tests/functional/adapter/test_basic.py        |   23 +-
 .../assets/lorem-ipsum.txt                    |    1 -
 .../bq_models/clustered.sql                   |    9 -
 .../bq_models/multi_clustered.sql             |    9 -
 .../bq_models/nested_table.sql                |   15 -
 .../bq_models/nested_view.sql                 |    7 -
 .../docs_generate_tests/bq_models/schema.yml  |   44 -
 .../bq_models_noschema/disabled.sql           |    2 -
 .../bq_models_noschema/model.sql              |    2 -
 .../cross_db_models/model.sql                 |    8 -
 .../cross_db_models/schema.yml                |   21 -
 .../fail_macros/failure.sql                   |    3 -
 .../docs_generate_tests/macros/dummy_test.sql |    9 -
 .../docs_generate_tests/macros/macro.md       |    8 -
 .../docs_generate_tests/macros/schema.yml     |   10 -
 .../docs_generate_tests/models/model.sql      |    8 -
 .../docs_generate_tests/models/readme.md      |    1 -
 .../docs_generate_tests/models/schema.yml     |   82 -
 .../models/second_model.sql                   |    8 -
 .../docs_generate_tests/ref_models/docs.md    |   31 -
 .../ref_models/ephemeral_copy.sql             |    7 -
 .../ref_models/ephemeral_summary.sql          |    9 -
 .../docs_generate_tests/ref_models/schema.yml |   48 -
 .../ref_models/view_summary.sql               |    8 -
 .../docs_generate_tests/seed/schema.yml       |   15 -
 .../docs_generate_tests/seed/seed.csv         |    2 -
 .../snapshot/snapshot_seed.sql                |   11 -
 .../docs_generate_tests/test_docs_generate.py | 1988 -----------------
 .../trivial_models/model.sql                  |    1 -
 tox.ini                                       |    2 +-
 31 files changed, 85 insertions(+), 2369 deletions(-)
 create mode 100644 tests/functional/adapter/expected_stats.py
 delete mode 100644 tests/integration/docs_generate_tests/assets/lorem-ipsum.txt
 delete mode 100644 tests/integration/docs_generate_tests/bq_models/clustered.sql
 delete mode 100644 tests/integration/docs_generate_tests/bq_models/multi_clustered.sql
 delete mode 100644 tests/integration/docs_generate_tests/bq_models/nested_table.sql
 delete mode 100644 tests/integration/docs_generate_tests/bq_models/nested_view.sql
 delete mode 100644 tests/integration/docs_generate_tests/bq_models/schema.yml
 delete mode 100644 tests/integration/docs_generate_tests/bq_models_noschema/disabled.sql
 delete mode 100644 tests/integration/docs_generate_tests/bq_models_noschema/model.sql
 delete mode 100644 tests/integration/docs_generate_tests/cross_db_models/model.sql
 delete mode 100644 tests/integration/docs_generate_tests/cross_db_models/schema.yml
 delete mode 100644 tests/integration/docs_generate_tests/fail_macros/failure.sql
 delete mode 100644 tests/integration/docs_generate_tests/macros/dummy_test.sql
 delete mode 100644 tests/integration/docs_generate_tests/macros/macro.md
 delete mode 100644 tests/integration/docs_generate_tests/macros/schema.yml
 delete mode 100644 tests/integration/docs_generate_tests/models/model.sql
 delete mode 100644 tests/integration/docs_generate_tests/models/readme.md
 delete mode 100644 tests/integration/docs_generate_tests/models/schema.yml
 delete mode 100644 tests/integration/docs_generate_tests/models/second_model.sql
 delete mode 100644 tests/integration/docs_generate_tests/ref_models/docs.md
 delete mode 100644 tests/integration/docs_generate_tests/ref_models/ephemeral_copy.sql
 delete mode 100644 tests/integration/docs_generate_tests/ref_models/ephemeral_summary.sql
 delete mode 100644 tests/integration/docs_generate_tests/ref_models/schema.yml
 delete mode 100644 tests/integration/docs_generate_tests/ref_models/view_summary.sql
 delete mode 100644 tests/integration/docs_generate_tests/seed/schema.yml
 delete mode 100644 tests/integration/docs_generate_tests/seed/seed.csv
 delete mode 100644 tests/integration/docs_generate_tests/snapshot/snapshot_seed.sql
 delete mode 100644 tests/integration/docs_generate_tests/test_docs_generate.py
 delete mode 100644 tests/integration/docs_generate_tests/trivial_models/model.sql

diff --git a/tests/functional/adapter/expected_stats.py b/tests/functional/adapter/expected_stats.py
new file mode 100644
index 000000000..f4a1e022d
--- /dev/null
+++ b/tests/functional/adapter/expected_stats.py
@@ -0,0 +1,62 @@
+from dbt.tests.util import AnyString, AnyFloat
+
+
+def bigquery_stats(is_table, partition=None, cluster=None):
+    stats = {}
+
+    if is_table:
+        stats.update(
+            {
+                "num_bytes": {
+                    "id": "num_bytes",
+                    "label": AnyString(),
+                    "value": AnyFloat(),
+                    "description": AnyString(),
+                    "include": True,
+                },
+                "num_rows": {
+                    "id": "num_rows",
+                    "label": AnyString(),
+                    "value": AnyFloat(),
+                    "description": AnyString(),
+                    "include": True,
+                },
+            }
+        )
+
+    if partition is not None:
+        stats.update(
+            {
+                "partitioning_type": {
+                    "id": "partitioning_type",
+                    "label": AnyString(),
+                    "value": partition,
+                    "description": AnyString(),
+                    "include": True,
+                }
+            }
+        )
+
+    if cluster is not None:
+        stats.update(
+            {
+                "clustering_fields": {
+                    "id": "clustering_fields",
+                    "label": AnyString(),
+                    "value": cluster,
+                    "description": AnyString(),
+                    "include": True,
+                }
+            }
+        )
+
+    has_stats = {
+        "id": "has_stats",
+        "label": "Has Stats?",
+        "value": bool(stats),
+        "description": "Indicates whether there are statistics for this table",
+        "include": False,
+    }
+    stats["has_stats"] = has_stats
+
+    return stats
diff --git a/tests/functional/adapter/test_basic.py b/tests/functional/adapter/test_basic.py
index 2c07a6e98..f95f043cf 100644
--- a/tests/functional/adapter/test_basic.py
+++ b/tests/functional/adapter/test_basic.py
@@ -13,6 +13,9 @@
 from dbt.tests.adapter.basic.test_snapshot_timestamp import BaseSnapshotTimestamp
 from dbt.tests.adapter.basic.test_adapter_methods import BaseAdapterMethod
 from dbt.tests.adapter.basic.test_validate_connection import BaseValidateConnection
+from dbt.tests.adapter.basic.test_docs_generate import BaseDocsGenerate
+from dbt.tests.adapter.basic.expected_catalog import base_expected_catalog
+from tests.functional.adapter.expected_stats import bigquery_stats
 
 
 class TestSimpleMaterializationsBigQuery(BaseSimpleMaterializations):
@@ -53,8 +56,26 @@ class TestSnapshotCheckColsBigQuery(BaseSnapshotCheckCols):
 class TestSnapshotTimestampBigQuery(BaseSnapshotTimestamp):
     pass
 
+
 class TestBaseAdapterMethodBigQuery(BaseAdapterMethod):
     pass
 
+
 class TestBigQueryValidateConnection(BaseValidateConnection):
-    pass
\ No newline at end of file
+    pass
+
+
+class TestDocsGenerateBigQuery(BaseDocsGenerate):
+    @pytest.fixture(scope="class")
+    def expected_catalog(self, project):
+        return base_expected_catalog(
+            project,
+            role=None,
+            id_type="INT64",
+            text_type="STRING",
+            time_type="DATETIME",
+            view_type="view",
+            table_type="table",
+            model_stats=bigquery_stats(False),
+            seed_stats=bigquery_stats(True),
+        )
diff --git a/tests/integration/docs_generate_tests/assets/lorem-ipsum.txt b/tests/integration/docs_generate_tests/assets/lorem-ipsum.txt
deleted file mode 100644
index cee7a927c..000000000
--- a/tests/integration/docs_generate_tests/assets/lorem-ipsum.txt
+++ /dev/null
@@ -1 +0,0 @@
-Lorem ipsum dolor sit amet, consectetur adipisicing elit, sed do eiusmod tempor incididunt ut labore et dolore magna aliqua. Ut enim ad minim veniam, quis nostrud exercitation ullamco laboris nisi ut aliquip ex ea commodo consequat. Duis aute irure dolor in reprehenderit in voluptate velit esse cillum dolore eu fugiat nulla pariatur. Excepteur sint occaecat cupidatat non proident, sunt in culpa qui officia deserunt mollit anim id est laborum.
diff --git a/tests/integration/docs_generate_tests/bq_models/clustered.sql b/tests/integration/docs_generate_tests/bq_models/clustered.sql
deleted file mode 100644
index 744d2ecb2..000000000
--- a/tests/integration/docs_generate_tests/bq_models/clustered.sql
+++ /dev/null
@@ -1,9 +0,0 @@
-{{
-    config(
-        materialized='table',
-        partition_by={'field': 'updated_at', 'data_type': 'date'},
-        cluster_by=['first_name']
-    )
-}}
-
-select id,first_name,email,ip_address,DATE(updated_at) as updated_at from {{ ref('seed') }}
diff --git a/tests/integration/docs_generate_tests/bq_models/multi_clustered.sql b/tests/integration/docs_generate_tests/bq_models/multi_clustered.sql
deleted file mode 100644
index e47df02f9..000000000
--- a/tests/integration/docs_generate_tests/bq_models/multi_clustered.sql
+++ /dev/null
@@ -1,9 +0,0 @@
-{{
-    config(
-        materialized='table',
-        partition_by={'field': 'updated_at', 'data_type': 'date'},
-        cluster_by=['first_name','email']
-    )
-}}
-
-select id,first_name,email,ip_address,DATE(updated_at) as updated_at from {{ ref('seed') }}
diff --git a/tests/integration/docs_generate_tests/bq_models/nested_table.sql b/tests/integration/docs_generate_tests/bq_models/nested_table.sql
deleted file mode 100644
index 22f9048d6..000000000
--- a/tests/integration/docs_generate_tests/bq_models/nested_table.sql
+++ /dev/null
@@ -1,15 +0,0 @@
-{{
-    config(
-        materialized='table'
-    )
-}}
-
-select
-    1 as field_1,
-    2 as field_2,
-    3 as field_3,
-
-    struct(
-        5 as field_5,
-        6 as field_6
-    ) as nested_field
diff --git a/tests/integration/docs_generate_tests/bq_models/nested_view.sql b/tests/integration/docs_generate_tests/bq_models/nested_view.sql
deleted file mode 100644
index 29e7b4a2d..000000000
--- a/tests/integration/docs_generate_tests/bq_models/nested_view.sql
+++ /dev/null
@@ -1,7 +0,0 @@
-{{
-    config(
-        materialized='view'
-    )
-}}
-
-select * from {{ ref('nested_table') }}
diff --git a/tests/integration/docs_generate_tests/bq_models/schema.yml b/tests/integration/docs_generate_tests/bq_models/schema.yml
deleted file mode 100644
index 87ebd8c71..000000000
--- a/tests/integration/docs_generate_tests/bq_models/schema.yml
+++ /dev/null
@@ -1,44 +0,0 @@
-version: 2
-
-models:
-  - name: nested_view
-    description: "The test model"
-    columns:
-      - name: field_1
-        description: The first field
-      - name: field_2
-        description: The second field
-      - name: field_3
-        description: The third field
-      - name: nested_field.field_4
-        description: The first nested field
-      - name: nested_field.field_5
-        description: The second nested field
-  - name: clustered
-    description: "A clustered and partitioned copy of the test model"
-    columns:
-      - name: id
-        description: The user id
-      - name: first_name
-        description: The user's name
-      - name: email
-        description: The user's email
-      - name: ip_address
-        description: The user's IP address
-      - name: updated_at
-        description: When the user was updated
-  - name: multi_clustered
-    description: "A clustered and partitioned copy of the test model, clustered on multiple columns"
-    columns:
-      - name: id
-        description: The user id
-      - name: first_name
-        description: The user's name
-      - name: email
-        description: The user's email
-      - name: ip_address
-        description: The user's IP address
-      - name: updated_at
-        description: When the user was updated
-
-
diff --git a/tests/integration/docs_generate_tests/bq_models_noschema/disabled.sql b/tests/integration/docs_generate_tests/bq_models_noschema/disabled.sql
deleted file mode 100644
index e4368a859..000000000
--- a/tests/integration/docs_generate_tests/bq_models_noschema/disabled.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-{{ config(disabled=true, schema='notrealnotreal') }}
-select 1 as id
diff --git a/tests/integration/docs_generate_tests/bq_models_noschema/model.sql b/tests/integration/docs_generate_tests/bq_models_noschema/model.sql
deleted file mode 100644
index 2fb872e84..000000000
--- a/tests/integration/docs_generate_tests/bq_models_noschema/model.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-{{ config(schema=var('extra_schema')) }}
-select 1 as id
diff --git a/tests/integration/docs_generate_tests/cross_db_models/model.sql b/tests/integration/docs_generate_tests/cross_db_models/model.sql
deleted file mode 100644
index 0759a4572..000000000
--- a/tests/integration/docs_generate_tests/cross_db_models/model.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-{{
-    config(
-        materialized='view',
-        database=var('alternate_db')
-    )
-}}
-
-select * from {{ ref('seed') }}
diff --git a/tests/integration/docs_generate_tests/cross_db_models/schema.yml b/tests/integration/docs_generate_tests/cross_db_models/schema.yml
deleted file mode 100644
index 6f3fc6619..000000000
--- a/tests/integration/docs_generate_tests/cross_db_models/schema.yml
+++ /dev/null
@@ -1,21 +0,0 @@
-version: 2
-
-models:
-  - name: model
-    description: "The test model"
-    columns:
-      - name: id
-        description: The user ID number
-        tests:
-          - unique
-          - not_null
-      - name: first_name
-        description: The user's first name
-      - name: email
-        description: The user's email
-      - name: ip_address
-        description: The user's IP address
-      - name: updated_at
-        description: The last time this user's email was updated
-    tests:
-      - test.nothing
diff --git a/tests/integration/docs_generate_tests/fail_macros/failure.sql b/tests/integration/docs_generate_tests/fail_macros/failure.sql
deleted file mode 100644
index f0519ed24..000000000
--- a/tests/integration/docs_generate_tests/fail_macros/failure.sql
+++ /dev/null
@@ -1,3 +0,0 @@
-{% macro get_catalog(information_schema, schemas) %}
-	{% do exceptions.raise_compiler_error('rejected: no catalogs for you') %}
-{% endmacro %}
diff --git a/tests/integration/docs_generate_tests/macros/dummy_test.sql b/tests/integration/docs_generate_tests/macros/dummy_test.sql
deleted file mode 100644
index adf4d7d93..000000000
--- a/tests/integration/docs_generate_tests/macros/dummy_test.sql
+++ /dev/null
@@ -1,9 +0,0 @@
-
-{% test nothing(model) %}
-
--- a silly test to make sure that table-level tests show up in the manifest
--- without a column_name field
-select 0
-
-{% endtest %}
-
diff --git a/tests/integration/docs_generate_tests/macros/macro.md b/tests/integration/docs_generate_tests/macros/macro.md
deleted file mode 100644
index 6a48ffc1b..000000000
--- a/tests/integration/docs_generate_tests/macros/macro.md
+++ /dev/null
@@ -1,8 +0,0 @@
-
-{% docs macro_info %}
-My custom test that I wrote that does nothing
-{% enddocs %}
-
-{% docs macro_arg_info %}
-The model for my custom test
-{% enddocs %}
diff --git a/tests/integration/docs_generate_tests/macros/schema.yml b/tests/integration/docs_generate_tests/macros/schema.yml
deleted file mode 100644
index 6b33f4730..000000000
--- a/tests/integration/docs_generate_tests/macros/schema.yml
+++ /dev/null
@@ -1,10 +0,0 @@
-version: 2
-macros:
-  - name: test_nothing
-    description: "{{ doc('macro_info') }}"
-    meta:
-      some_key: 100
-    arguments:
-      - name: model
-        type: Relation
-        description: "{{ doc('macro_arg_info') }}"
diff --git a/tests/integration/docs_generate_tests/models/model.sql b/tests/integration/docs_generate_tests/models/model.sql
deleted file mode 100644
index 0759a4572..000000000
--- a/tests/integration/docs_generate_tests/models/model.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-{{
-    config(
-        materialized='view',
-        database=var('alternate_db')
-    )
-}}
-
-select * from {{ ref('seed') }}
diff --git a/tests/integration/docs_generate_tests/models/readme.md b/tests/integration/docs_generate_tests/models/readme.md
deleted file mode 100644
index d59a7f447..000000000
--- a/tests/integration/docs_generate_tests/models/readme.md
+++ /dev/null
@@ -1 +0,0 @@
-This is a readme.md file with {{ invalid-ish jinja }} in it
diff --git a/tests/integration/docs_generate_tests/models/schema.yml b/tests/integration/docs_generate_tests/models/schema.yml
deleted file mode 100644
index 600be50ea..000000000
--- a/tests/integration/docs_generate_tests/models/schema.yml
+++ /dev/null
@@ -1,82 +0,0 @@
-version: 2
-
-models:
-  - name: model
-    description: "The test model"
-    docs:
-      show: false
-    columns:
-      - name: id
-        description: The user ID number
-        tests:
-          - unique
-          - not_null
-      - name: first_name
-        description: The user's first name
-      - name: email
-        description: The user's email
-      - name: ip_address
-        description: The user's IP address
-      - name: updated_at
-        description: The last time this user's email was updated
-    tests:
-      - test.nothing
-
-  - name: second_model
-    description: "The second test model"
-    docs:
-      show: false
-    columns:
-      - name: id
-        description: The user ID number
-      - name: first_name
-        description: The user's first name
-      - name: email
-        description: The user's email
-      - name: ip_address
-        description: The user's IP address
-      - name: updated_at
-        description: The last time this user's email was updated
-
-
-sources:
-  - name: my_source
-    description: "My source"
-    loader: a_loader
-    schema: "{{ var('test_schema') }}"
-    tables:
-      - name: my_table
-        description: "My table"
-        identifier: seed
-        quoting:
-          identifier: True
-        columns:
-          - name: id
-            description: "An ID field"
-
-
-exposures:
-  - name: simple_exposure
-    type: dashboard
-    depends_on:
-      - ref('model')
-      - source('my_source', 'my_table')
-    owner:
-      email: something@example.com
-  - name: notebook_exposure
-    type: notebook
-    depends_on:
-      - ref('model')
-      - ref('second_model')
-    owner:
-      email: something@example.com
-      name: Some name
-    description: >
-      A description of the complex exposure
-    maturity: medium
-    meta:
-      tool: 'my_tool'
-      languages:
-        - python
-    tags: ['my_department']
-    url: http://example.com/notebook/1
diff --git a/tests/integration/docs_generate_tests/models/second_model.sql b/tests/integration/docs_generate_tests/models/second_model.sql
deleted file mode 100644
index 37f83155c..000000000
--- a/tests/integration/docs_generate_tests/models/second_model.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-{{
-    config(
-        materialized='view',
-        schema='test',
-    )
-}}
-
-select * from {{ ref('seed') }}
diff --git a/tests/integration/docs_generate_tests/ref_models/docs.md b/tests/integration/docs_generate_tests/ref_models/docs.md
deleted file mode 100644
index c5ad96862..000000000
--- a/tests/integration/docs_generate_tests/ref_models/docs.md
+++ /dev/null
@@ -1,31 +0,0 @@
-{% docs ephemeral_summary %}
-A summmary table of the ephemeral copy of the seed data
-{% enddocs %}
-
-{% docs summary_first_name %}
-The first name being summarized
-{% enddocs %}
-
-{% docs summary_count %}
-The number of instances of the first name
-{% enddocs %}
-
-{% docs view_summary %}
-A view of the summary of the ephemeral copy of the seed data
-{% enddocs %}
-
-{% docs source_info %}
-My source
-{% enddocs %}
-
-{% docs table_info %}
-My table
-{% enddocs %}
-
-{% docs column_info %}
-An ID field
-{% enddocs %}
-
-{% docs notebook_info %}
-A description of the complex exposure
-{% enddocs %}
diff --git a/tests/integration/docs_generate_tests/ref_models/ephemeral_copy.sql b/tests/integration/docs_generate_tests/ref_models/ephemeral_copy.sql
deleted file mode 100644
index 3f7e698ce..000000000
--- a/tests/integration/docs_generate_tests/ref_models/ephemeral_copy.sql
+++ /dev/null
@@ -1,7 +0,0 @@
-{{
-  config(
-    materialized = "ephemeral"
-  )
-}}
-
-select * from {{ source("my_source", "my_table") }}
diff --git a/tests/integration/docs_generate_tests/ref_models/ephemeral_summary.sql b/tests/integration/docs_generate_tests/ref_models/ephemeral_summary.sql
deleted file mode 100644
index 4dba8a564..000000000
--- a/tests/integration/docs_generate_tests/ref_models/ephemeral_summary.sql
+++ /dev/null
@@ -1,9 +0,0 @@
-{{
-  config(
-    materialized = "table"
-  )
-}}
-
-select first_name, count(*) as ct from {{ref('ephemeral_copy')}}
-group by first_name
-order by first_name asc
diff --git a/tests/integration/docs_generate_tests/ref_models/schema.yml b/tests/integration/docs_generate_tests/ref_models/schema.yml
deleted file mode 100644
index 6d7dcea05..000000000
--- a/tests/integration/docs_generate_tests/ref_models/schema.yml
+++ /dev/null
@@ -1,48 +0,0 @@
-version: 2
-
-models:
-  - name: ephemeral_summary
-    description: "{{ doc('ephemeral_summary') }}"
-    columns: &summary_columns
-      - name: first_name
-        description: "{{ doc('summary_first_name') }}"
-      - name: ct
-        description: "{{ doc('summary_count') }}"
-  - name: view_summary
-    description: "{{ doc('view_summary') }}"
-    columns: *summary_columns
-
-sources:
-  - name: my_source
-    description: "{{ doc('source_info') }}"
-    loader: a_loader
-    schema: "{{ var('test_schema') }}"
-    quoting:
-      database: False
-      identifier: False
-    tables:
-      - name: my_table
-        description: "{{ doc('table_info') }}"
-        identifier: seed
-        quoting:
-          identifier: True
-        columns:
-          - name: id
-            description: "{{ doc('column_info') }}"
-
-exposures:
-  - name: notebook_exposure
-    type: notebook
-    depends_on:
-      - ref('view_summary')
-    owner:
-      email: something@example.com
-      name: Some name
-    description: "{{ doc('notebook_info') }}"
-    maturity: medium
-    url: http://example.com/notebook/1
-    meta:
-      tool: 'my_tool'
-      languages:
-        - python
-    tags: ['my_department']
diff --git a/tests/integration/docs_generate_tests/ref_models/view_summary.sql b/tests/integration/docs_generate_tests/ref_models/view_summary.sql
deleted file mode 100644
index 6ad6c3dd0..000000000
--- a/tests/integration/docs_generate_tests/ref_models/view_summary.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-{{
-  config(
-    materialized = "view"
-  )
-}}
-
-select first_name, ct from {{ref('ephemeral_summary')}}
-order by ct asc
diff --git a/tests/integration/docs_generate_tests/seed/schema.yml b/tests/integration/docs_generate_tests/seed/schema.yml
deleted file mode 100644
index ef5e7dc9e..000000000
--- a/tests/integration/docs_generate_tests/seed/schema.yml
+++ /dev/null
@@ -1,15 +0,0 @@
-version: 2
-seeds:
-  - name: seed
-    description: "The test seed"
-    columns:
-      - name: id
-        description: The user ID number
-      - name: first_name
-        description: The user's first name
-      - name: email
-        description: The user's email
-      - name: ip_address
-        description: The user's IP address
-      - name: updated_at
-        description: The last time this user's email was updated
diff --git a/tests/integration/docs_generate_tests/seed/seed.csv b/tests/integration/docs_generate_tests/seed/seed.csv
deleted file mode 100644
index ef154f552..000000000
--- a/tests/integration/docs_generate_tests/seed/seed.csv
+++ /dev/null
@@ -1,2 +0,0 @@
-id,first_name,email,ip_address,updated_at
-1,Larry,lking0@miitbeian.gov.cn,69.135.206.194,2008-09-12 19:08:31
diff --git a/tests/integration/docs_generate_tests/snapshot/snapshot_seed.sql b/tests/integration/docs_generate_tests/snapshot/snapshot_seed.sql
deleted file mode 100644
index 83bc54fe7..000000000
--- a/tests/integration/docs_generate_tests/snapshot/snapshot_seed.sql
+++ /dev/null
@@ -1,11 +0,0 @@
-{% snapshot snapshot_seed %}
-{{
-    config(
-      unique_key='id',
-      strategy='check',
-      check_cols='all',
-      target_schema=var('alternate_schema')
-    )
-}}
-select * from {{ ref('seed') }}
-{% endsnapshot %}
\ No newline at end of file
diff --git a/tests/integration/docs_generate_tests/test_docs_generate.py b/tests/integration/docs_generate_tests/test_docs_generate.py
deleted file mode 100644
index 0f62c025f..000000000
--- a/tests/integration/docs_generate_tests/test_docs_generate.py
+++ /dev/null
@@ -1,1988 +0,0 @@
-import hashlib
-import json
-import os
-import random
-import shutil
-import tempfile
-import time
-from datetime import datetime
-from unittest.mock import ANY, patch
-
-from pytest import mark
-from tests.integration.base import DBTIntegrationTest, use_profile, AnyFloat, \
-    AnyString, AnyStringWith, normalize, Normalized
-
-import dbt.tracking
-import dbt.version
-from dbt.exceptions import CompilationException
-
-
-def _read_file(path):
-    with open(path, 'r') as fp:
-        return fp.read().replace('\r', '').replace('\\r', '')
-
-
-class LineIndifferent:
-    def __init__(self, expected):
-        self.expected = expected.replace('\r', '')
-
-    def __eq__(self, other):
-        return self.expected == other.replace('\r', '')
-
-    def __repr__(self):
-        return 'LineIndifferent({!r})'.format(self.expected)
-
-    def __str__(self):
-        return self.__repr__()
-
-
-class OneOf:
-    def __init__(self, *options):
-        self.options = options
-
-    def __eq__(self, other):
-        return any(o == other for o in self.options)
-
-    def __repr__(self):
-        return 'OneOf(*{!r})'.format(self.options)
-
-    def __str__(self):
-        return self.__repr__()
-
-
-def _read_json(path):
-    # read json generated by dbt.
-    with open(path) as fp:
-        return json.load(fp)
-
-
-def walk_files(path):
-    for root, dirs, files in os.walk(path):
-        for basename in files:
-            yield os.path.join(root, basename)
-
-
-class TestDocsGenerateEscapes(DBTIntegrationTest):
-    prefix = "pgtest{}{:04}".format(int(time.time()), random.randint(0, 9999))
-
-    @property
-    def schema(self):
-        return 'docs_generate'
-
-    @staticmethod
-    def dir(path):
-        return normalize(path)
-
-    @property
-    def models(self):
-        return self.dir("trivial_models")
-
-    def run_and_generate(self):
-        self.assertEqual(len(self.run_dbt(['run'])), 1)
-        os.remove(normalize('target/manifest.json'))
-        os.remove(normalize('target/run_results.json'))
-        self.run_dbt(['docs', 'generate'])
-
-
-class TestDocsGenerate(DBTIntegrationTest):
-    setup_alternate_db = True
-
-    def adapter_case(self, value):
-        return value.lower()
-
-    def setUp(self):
-        super().setUp()
-        self.maxDiff = None
-        self.alternate_schema = self.unique_schema() + '_test'
-
-        self._created_schemas.add(self.alternate_schema)
-        os.environ['DBT_ENV_CUSTOM_ENV_env_key'] = 'env_value'
-
-    def tearDown(self):
-        super().tearDown()
-        del os.environ['DBT_ENV_CUSTOM_ENV_env_key']
-
-    @property
-    def schema(self):
-        return 'docs_generate'
-
-    @staticmethod
-    def dir(path):
-        return normalize(path)
-
-    @property
-    def models(self):
-        return self.dir("models")
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'quoting': {
-                'identifier': False
-            }
-        }
-
-    def run_and_generate(self, extra=None, seed_count=1, model_count=2, alternate_db=None, args=None):
-        if alternate_db is None:
-            alternate_db = self.alternative_database
-        project = {
-            "seed-paths": [self.dir("seed")],
-            'macro-paths': [self.dir('macros')],
-            'snapshot-paths': [self.dir('snapshot')],
-            'vars': {
-                'alternate_db': alternate_db,
-                'alternate_schema': self.alternate_schema,
-            },
-            'seeds': {
-                'quote_columns': True,
-            },
-        }
-        if extra:
-            project.update(extra)
-        self.use_default_project(project)
-
-        vars_arg = '--vars={{test_schema: {}}}'.format(self.unique_schema())
-
-        self.assertEqual(len(self.run_dbt(["seed", vars_arg])), seed_count)
-        self.assertEqual(len(self.run_dbt(['run', vars_arg])), model_count)
-        os.remove(normalize('target/manifest.json'))
-        os.remove(normalize('target/run_results.json'))
-        self.generate_start_time = datetime.utcnow()
-        base_args = ['docs', 'generate', vars_arg]
-        if args:
-            base_args.extend(args)
-        self.run_dbt(base_args)
-
-    def _no_stats(self):
-        return {
-            'has_stats': {
-                'id': 'has_stats',
-                'label': 'Has Stats?',
-                'value': False,
-                'description': 'Indicates whether there are statistics for this table',
-                'include': False,
-            },
-        }
-
-    def _bigquery_stats(self, is_table, partition=None, cluster=None):
-        stats = {}
-
-        if is_table:
-            stats.update({
-                'num_bytes': {
-                    'id': 'num_bytes',
-                    'label': AnyString(),
-                    'value': AnyFloat(),
-                    'description': AnyString(),
-                    'include': True,
-                },
-                'num_rows': {
-                    'id': 'num_rows',
-                    'label': AnyString(),
-                    'value': AnyFloat(),
-                    'description': AnyString(),
-                    'include': True,
-                }
-            })
-
-        if partition is not None:
-            stats.update({
-                'partitioning_type': {
-                    'id': 'partitioning_type',
-                    'label': AnyString(),
-                    'value': partition,
-                    'description': AnyString(),
-                    'include': True
-                }
-            })
-
-        if cluster is not None:
-            stats.update({
-                'clustering_fields': {
-                    'id': 'clustering_fields',
-                    'label': AnyString(),
-                    'value': cluster,
-                    'description': AnyString(),
-                    'include': True
-                }
-            })
-
-        has_stats = {
-            'id': 'has_stats',
-            'label': 'Has Stats?',
-            'value': bool(stats),
-            'description': 'Indicates whether there are statistics for this table',
-            'include': False,
-        }
-        stats['has_stats'] = has_stats
-
-        return stats
-
-    def _expected_catalog(self, id_type, text_type, time_type, view_type,
-                          table_type, model_stats, seed_stats=None, case=None,
-                          case_columns=False, model_database=None):
-        if case is None:
-            def case(x): return x
-        col_case = case if case_columns else lambda x: x
-
-        if seed_stats is None:
-            seed_stats = model_stats
-
-        if model_database is None:
-            model_database = self.default_database
-        my_schema_name = self.unique_schema()
-        role = self.get_role()
-        expected_cols = {
-            col_case('id'): {
-                'name': col_case('id'),
-                'index': 1,
-                'type': id_type,
-                'comment': None,
-            },
-            col_case('first_name'): {
-                'name': col_case('first_name'),
-                'index': 2,
-                'type': text_type,
-                'comment': None,
-            },
-            col_case('email'): {
-                'name': col_case('email'),
-                'index': 3,
-                'type': text_type,
-                'comment': None,
-            },
-            col_case('ip_address'): {
-                'name': col_case('ip_address'),
-                'index': 4,
-                'type': text_type,
-                'comment': None,
-            },
-            col_case('updated_at'): {
-                'name': col_case('updated_at'),
-                'index': 5,
-                'type': time_type,
-                'comment': None,
-            },
-        }
-        return {
-            'nodes': {
-                'model.test.model': {
-                    'unique_id': 'model.test.model',
-                    'metadata': {
-                        'schema': my_schema_name,
-                        'database': model_database,
-                        'name': case('model'),
-                        'type': view_type,
-                        'comment': None,
-                        'owner': role,
-                    },
-                    'stats': model_stats,
-                    'columns': expected_cols,
-                },
-                'model.test.second_model': {
-                    'unique_id': 'model.test.second_model',
-                    'metadata': {
-                        'schema': self.alternate_schema,
-                        'database': self.default_database,
-                        'name': case('second_model'),
-                        'type': view_type,
-                        'comment': None,
-                        'owner': role,
-                    },
-                    'stats': model_stats,
-                    'columns': expected_cols,
-                },
-                'seed.test.seed': {
-                    'unique_id': 'seed.test.seed',
-                    'metadata': {
-                        'schema': my_schema_name,
-                        'database': self.default_database,
-                        'name': case('seed'),
-                        'type': table_type,
-                        'comment': None,
-                        'owner': role,
-                    },
-                    'stats': seed_stats,
-                    'columns': expected_cols,
-                },
-            },
-            'sources': {
-                'source.test.my_source.my_table': {
-                    'unique_id': 'source.test.my_source.my_table',
-                    'metadata': {
-                        'schema': my_schema_name,
-                        'database': self.default_database,
-                        'name': case('seed'),
-                        'type': table_type,
-                        'comment': None,
-                        'owner': role,
-                    },
-                    'stats': seed_stats,
-                    'columns': expected_cols,
-                },
-            },
-        }
-
-    def get_role(self):
-        return None
-
-    def expected_bigquery_catalog(self):
-        return self._expected_catalog(
-            id_type='INT64',
-            text_type='STRING',
-            time_type='DATETIME',
-            view_type='view',
-            table_type='table',
-            model_stats=self._bigquery_stats(False),
-            seed_stats=self._bigquery_stats(True),
-            model_database=self.alternative_database,
-        )
-
-    @staticmethod
-    def _clustered_bigquery_columns(update_type):
-        return {
-            'id': {
-                'comment': None,
-                'index': 1,
-                'name': 'id',
-                'type': 'INT64',
-            },
-            'first_name': {
-                'comment': None,
-                'index': 2,
-                'name': 'first_name',
-                'type': 'STRING',
-            },
-            'email': {
-                'comment': None,
-                'index': 3,
-                'name': 'email',
-                'type': 'STRING',
-            },
-            'ip_address': {
-                'comment': None,
-                'index': 4,
-                'name': 'ip_address',
-                'type': 'STRING',
-            },
-            'updated_at': {
-                'comment': None,
-                'index': 5,
-                'name': 'updated_at',
-                'type': update_type,
-            },
-        }
-
-    def expected_bigquery_complex_catalog(self):
-        my_schema_name = self.unique_schema()
-        role = self.get_role()
-        table_stats = self._bigquery_stats(True)
-        clustering_stats = self._bigquery_stats(True, partition='updated_at',
-                                                cluster='first_name')
-        multi_clustering_stats = self._bigquery_stats(True, partition='updated_at',
-                                                      cluster='first_name, email')
-        nesting_columns = {
-            'field_1': {
-                'name': 'field_1',
-                'index': 1,
-                'type': 'INT64',
-                'comment': None
-            },
-            'field_2': {
-                'name': 'field_2',
-                'index': 2,
-                'type': 'INT64',
-                'comment': None
-            },
-            'field_3': {
-                'name': 'field_3',
-                'index': 3,
-                'type': 'INT64',
-                'comment': None
-            },
-            'nested_field': {
-                'name': 'nested_field',
-                'index': 4,
-                'type': 'STRUCT',
-                'comment': None
-            },
-            'nested_field.field_5': {
-                'name': 'nested_field.field_5',
-                'index': 5,
-                'type': 'INT64',
-                'comment': None
-            },
-            'nested_field.field_6': {
-                'name': 'nested_field.field_6',
-                'index': 6,
-                'type': 'INT64',
-                'comment': None
-            }
-        }
-
-        return {
-            'nodes': {
-                'model.test.clustered': {
-                    'unique_id': 'model.test.clustered',
-                    'metadata': {
-                        'comment': None,
-                        'name': 'clustered',
-                        'owner': None,
-                        'schema': my_schema_name,
-                        'database': self.default_database,
-                        'type': 'table'
-                    },
-                    'stats': clustering_stats,
-                    'columns': self._clustered_bigquery_columns('DATE'),
-                },
-                'model.test.multi_clustered': {
-                    'unique_id': 'model.test.multi_clustered',
-                    'metadata': {
-                        'comment': None,
-                        'name': 'multi_clustered',
-                        'owner': None,
-                        'schema': my_schema_name,
-                        'database': self.default_database,
-                        'type': 'table'
-                    },
-                    'stats': multi_clustering_stats,
-                    'columns': self._clustered_bigquery_columns('DATE'),
-                },
-                'seed.test.seed': {
-                    'unique_id': 'seed.test.seed',
-                    'metadata': {
-                        'comment': None,
-                        'name': 'seed',
-                        'owner': None,
-                        'schema': my_schema_name,
-                        'database': self.default_database,
-                        'type': 'table',
-                    },
-                    'stats': table_stats,
-                    'columns': self._clustered_bigquery_columns('DATETIME'),
-                },
-                'model.test.nested_view': {
-                    'unique_id': 'model.test.nested_view',
-                    'metadata': {
-                        'schema': my_schema_name,
-                        'database': self.default_database,
-                        'name': 'nested_view',
-                        'type': 'view',
-                        'owner': role,
-                        'comment': None
-                    },
-                    'stats': self._bigquery_stats(False),
-                    'columns': nesting_columns,
-                },
-                'model.test.nested_table': {
-                    'unique_id': 'model.test.nested_table',
-                    'metadata': {
-                        'schema': my_schema_name,
-                        'database': self.default_database,
-                        'name': 'nested_table',
-                        'type': 'table',
-                        'owner': role,
-                        'comment': None
-                    },
-                    'stats': table_stats,
-                    'columns': nesting_columns,
-                },
-            },
-            'sources': {},
-        }
-
-    def verify_catalog(self, expected):
-        self.assertTrue(os.path.exists('./target/catalog.json'))
-
-        catalog = _read_json('./target/catalog.json')
-
-        assert set(catalog) == {'errors', 'metadata', 'nodes', 'sources'}
-
-        self.verify_metadata(
-            catalog['metadata'], 'https://schemas.getdbt.com/dbt/catalog/v1.json')
-        assert not catalog['errors']
-
-        for key in 'nodes', 'sources':
-            assert catalog[key] == expected[key]
-
-    def verify_manifest_macros(self, manifest, expected=None):
-        self.assertIn('macros', manifest)
-        if expected is None:
-            self._verify_generic_macro_structure(manifest)
-            return
-        for unique_id, expected_macro in expected.items():
-            self.assertIn(unique_id, manifest['macros'])
-            actual_macro = manifest['macros'][unique_id]
-            self.assertEqual(expected_macro, actual_macro)
-
-    def rendered_model_config(self, **updates):
-        result = {
-            'database': None,
-            'schema': None,
-            'alias': None,
-            'enabled': True,
-            'materialized': 'view',
-            'pre-hook': [],
-            'post-hook': [],
-            'column_types': {},
-            'quoting': {},
-            'tags': [],
-            'persist_docs': {},
-            'full_refresh': None,
-            'on_schema_change': 'ignore',
-            'meta': {},
-            'unique_key': None,
-        }
-        result.update(updates)
-        return result
-
-    def unrendered_model_config(self, **updates):
-        return updates
-
-    def rendered_seed_config(self, **updates):
-        result = {
-            'enabled': True,
-            'materialized': 'seed',
-            'persist_docs': {},
-            'pre-hook': [],
-            'post-hook': [],
-            'column_types': {},
-            'quoting': {},
-            'tags': [],
-            'quote_columns': True,
-            'full_refresh': None,
-            'on_schema_change': 'ignore',
-            'database': None,
-            'schema': None,
-            'alias': None,
-            'meta': {},
-            'unique_key': None,
-        }
-        result.update(updates)
-        return result
-
-    def unrendered_seed_config(self, **updates):
-        result = {'quote_columns': True}
-        result.update(updates)
-        return result
-
-    def rendered_snapshot_config(self, **updates):
-        result = {
-            'database': None,
-            'schema': None,
-            'alias': None,
-            'enabled': True,
-            'materialized': 'snapshot',
-            'pre-hook': [],
-            'post-hook': [],
-            'column_types': {},
-            'quoting': {},
-            'tags': [],
-            'persist_docs': {},
-            'full_refresh': None,
-            'on_schema_change': 'ignore',
-            'strategy': 'check',
-            'check_cols': 'all',
-            'unique_key': 'id',
-            'target_schema': None,
-            'meta': {},
-        }
-        result.update(updates)
-        return result
-
-    def unrendered_snapshot_config(self, **updates):
-        result = {
-            'check_cols': 'all',
-            'strategy': 'check',
-            'target_schema': None,
-            'unique_key': 'id'
-        }
-        result.update(updates)
-        return result
-
-    def rendered_tst_config(self, **updates):
-        result = {
-            'enabled': True,
-            'materialized': 'test',
-            'tags': [],
-            'severity': 'ERROR',
-            'store_failures': None,
-            'warn_if': '!= 0',
-            'error_if': '!= 0',
-            'fail_calc': 'count(*)',
-            'where': None,
-            'limit': None,
-            'database': None,
-            'schema': 'dbt_test__audit',
-            'alias': None,
-            'meta': {},
-        }
-        result.update(updates)
-        return result
-
-    def unrendered_tst_config(self, **updates):
-        result = {}
-        result.update(updates)
-        return result
-
-    def _verify_generic_macro_structure(self, manifest):
-        # just test a known global macro to avoid having to update this every
-        # time they change.
-        self.assertIn('macro.dbt.get_quoted_csv', manifest['macros'])
-        macro = manifest['macros']['macro.dbt.get_quoted_csv']
-        self.assertEqual(
-            set(macro),
-            {
-                'path', 'original_file_path', 'package_name',
-                'root_path', 'name', 'unique_id', 'tags', 'resource_type',
-                'depends_on', 'meta', 'description', 'patch_path', 'arguments',
-                'macro_sql', 'docs', 'created_at',
-            }
-        )
-        # Don't compare the sql, just make sure it exists
-        self.assertTrue(len(macro['macro_sql']) > 10)
-        without_sql = {
-            k: v for k, v in macro.items()
-            if k not in {'macro_sql'}
-        }
-        # Windows means we can't hard-code these.
-        helpers_path = Normalized('macros/materializations/models/incremental/column_helpers.sql')
-        root_path = Normalized(os.path.join(
-            self.dbt_core_install_root, 'include', 'global_project'
-        ))
-        self.assertEqual(
-            {
-                'path': helpers_path,
-                'original_file_path': helpers_path,
-                'package_name': 'dbt',
-                'root_path': root_path,
-                'name': 'get_quoted_csv',
-                'unique_id': 'macro.dbt.get_quoted_csv',
-                'created_at': ANY,
-                'tags': [],
-                'resource_type': 'macro',
-                'depends_on': {'macros': []},
-                'description': '',
-                'docs': {'show': True},
-                'patch_path': None,
-                'meta': {},
-                'arguments': [],
-            },
-            without_sql,
-        )
-
-    def expected_seeded_manifest(self, model_database=None, quote_model=False):
-        models_path = self.dir('models')
-        model_sql_path = os.path.join(models_path, 'model.sql')
-        second_model_sql_path = os.path.join(models_path, 'second_model.sql')
-        model_schema_yml_path = os.path.join(models_path, 'schema.yml')
-        seed_schema_yml_path = os.path.join(self.dir('seed'), 'schema.yml')
-        seed_path = self.dir(os.path.join('seed', 'seed.csv'))
-        snapshot_path = self.dir(os.path.join('snapshot', 'snapshot_seed.sql'))
-
-        my_schema_name = self.unique_schema()
-        test_audit_schema = my_schema_name + '_dbt_test__audit'
-
-        if model_database is None:
-            model_database = self.alternative_database
-
-        model_config = self.rendered_model_config(database=model_database)
-        second_config = self.rendered_model_config(
-            schema=self.alternate_schema[-4:])
-
-        unrendered_model_config = self.unrendered_model_config(
-            database=model_database, materialized='view')
-        unrendered_second_config = self.unrendered_model_config(
-            schema=self.alternate_schema[-4:], materialized='view')
-
-        seed_config = self.rendered_seed_config()
-        unrendered_seed_config = self.unrendered_seed_config()
-
-        test_config = self.rendered_tst_config()
-        unrendered_test_config = self.unrendered_tst_config()
-
-        snapshot_config = self.rendered_snapshot_config(
-            target_schema=self.alternate_schema)
-        unrendered_snapshot_config = self.unrendered_snapshot_config(
-            target_schema=self.alternate_schema
-        )
-
-        quote_database = quote_schema = True
-        relation_name_node_format = self._relation_name_format(
-            quote_database, quote_schema, quote_model
-        )
-        relation_name_source_format = self._relation_name_format(
-            quote_database, quote_schema, quote_identifier=True
-        )
-
-        return {
-            'dbt_schema_version': 'https://schemas.getdbt.com/dbt/manifest/v5.json',
-            'dbt_version': dbt.version.__version__,
-            'nodes': {
-                'model.test.model': {
-                    'compiled_path': Normalized('target/compiled/test/models/model.sql'),
-                    'build_path': None,
-                    'created_at': ANY,
-                    'name': 'model',
-                    'root_path': self.test_root_realpath,
-                    'relation_name': relation_name_node_format.format(
-                        model_database, my_schema_name, 'model'
-                    ),
-                    'resource_type': 'model',
-                    'path': 'model.sql',
-                    'original_file_path': model_sql_path,
-                    'package_name': 'test',
-                    'raw_sql': LineIndifferent(_read_file(model_sql_path).rstrip('\r\n')),
-                    'refs': [['seed']],
-                    'sources': [],
-                    'depends_on': {'nodes': ['seed.test.seed'], 'macros': []},
-                    'unique_id': 'model.test.model',
-                    'fqn': ['test', 'model'],
-                    'tags': [],
-                    'meta': {},
-                    'config': model_config,
-                    'schema': my_schema_name,
-                    'database': model_database,
-                    'deferred': False,
-                    'alias': 'model',
-                    'description': 'The test model',
-                    'columns': {
-                        'id': {
-                            'name': 'id',
-                            'description': 'The user ID number',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'first_name': {
-                            'name': 'first_name',
-                            'description': "The user's first name",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'email': {
-                            'name': 'email',
-                            'description': "The user's email",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'ip_address': {
-                            'name': 'ip_address',
-                            'description': "The user's IP address",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'updated_at': {
-                            'name': 'updated_at',
-                            'description': "The last time this user's email was updated",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                    },
-                    'patch_path': 'test://' + model_schema_yml_path,
-                    'docs': {'show': False},
-                    'compiled': True,
-                    'compiled_sql': ANY,
-                    'extra_ctes_injected': True,
-                    'extra_ctes': [],
-                    'checksum': self._checksum_file(model_sql_path),
-                    'unrendered_config': unrendered_model_config,
-                },
-                'model.test.second_model': {
-                    'compiled_path': Normalized('target/compiled/test/models/second_model.sql'),
-                    'build_path': None,
-                    'created_at': ANY,
-                    'name': 'second_model',
-                    'root_path': self.test_root_realpath,
-                    'relation_name': relation_name_node_format.format(
-                        self.default_database, self.alternate_schema,
-                        'second_model'
-                    ),
-                    'resource_type': 'model',
-                    'path': 'second_model.sql',
-                    'original_file_path': second_model_sql_path,
-                    'package_name': 'test',
-                    'raw_sql': LineIndifferent(_read_file(second_model_sql_path).rstrip('\r\n')),
-                    'refs': [['seed']],
-                    'sources': [],
-                    'depends_on': {'nodes': ['seed.test.seed'], 'macros': []},
-                    'unique_id': 'model.test.second_model',
-                    'fqn': ['test', 'second_model'],
-                    'tags': [],
-                    'meta': {},
-                    'config': second_config,
-                    'schema': self.alternate_schema,
-                    'database': self.default_database,
-                    'deferred': False,
-                    'alias': 'second_model',
-                    'description': 'The second test model',
-                    'columns': {
-                        'id': {
-                            'name': 'id',
-                            'description': 'The user ID number',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'first_name': {
-                            'name': 'first_name',
-                            'description': "The user's first name",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'email': {
-                            'name': 'email',
-                            'description': "The user's email",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'ip_address': {
-                            'name': 'ip_address',
-                            'description': "The user's IP address",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'updated_at': {
-                            'name': 'updated_at',
-                            'description': "The last time this user's email was updated",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                    },
-                    'patch_path': 'test://' + model_schema_yml_path,
-                    'docs': {'show': False},
-                    'compiled': True,
-                    'compiled_sql': ANY,
-                    'extra_ctes_injected': True,
-                    'extra_ctes': [],
-                    'checksum': self._checksum_file(second_model_sql_path),
-                    'unrendered_config': unrendered_second_config
-                },
-                'seed.test.seed': {
-                    'compiled_path': None,
-                    'build_path': None,
-                    'created_at': ANY,
-                    'compiled': True,
-                    'compiled_sql': '',
-                    'config': seed_config,
-                    'patch_path': 'test://' + seed_schema_yml_path,
-                    'path': 'seed.csv',
-                    'name': 'seed',
-                    'root_path': self.test_root_realpath,
-                    'relation_name': relation_name_node_format.format(
-                        self.default_database, my_schema_name, 'seed'
-                    ),
-                    'resource_type': 'seed',
-                    'raw_sql': '',
-                    'package_name': 'test',
-                    'original_file_path': seed_path,
-                    'refs': [],
-                    'sources': [],
-                    'depends_on': {'nodes': [], 'macros': []},
-                    'unique_id': 'seed.test.seed',
-                    'fqn': ['test', 'seed'],
-                    'tags': [],
-                    'meta': {},
-                    'schema': my_schema_name,
-                    'database': self.default_database,
-                    'alias': 'seed',
-                    'deferred': False,
-                    'description': 'The test seed',
-                    'columns': {
-                        'id': {
-                            'name': 'id',
-                            'description': 'The user ID number',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'first_name': {
-                            'name': 'first_name',
-                            'description': "The user's first name",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'email': {
-                            'name': 'email',
-                            'description': "The user's email",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'ip_address': {
-                            'name': 'ip_address',
-                            'description': "The user's IP address",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'updated_at': {
-                            'name': 'updated_at',
-                            'description': "The last time this user's email was updated",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                    },
-                    'docs': {'show': True},
-                    'compiled': True,
-                    'compiled_sql': '',
-                    'extra_ctes_injected': True,
-                    'extra_ctes': [],
-                    'checksum': self._checksum_file(seed_path),
-                    'unrendered_config': unrendered_seed_config,
-                },
-                'test.test.not_null_model_id.d01cc630e6': {
-                    'alias': 'not_null_model_id',
-                    'compiled_path': Normalized('target/compiled/test/models/schema.yml/not_null_model_id.sql'),
-                    'build_path': None,
-                    'created_at': ANY,
-                    'column_name': 'id',
-                    'columns': {},
-                    'config': test_config,
-                    'sources': [],
-                    'depends_on': {
-                        'macros': ['macro.dbt.test_not_null', 'macro.dbt.get_where_subquery'],
-                        'nodes': ['model.test.model'],
-                    },
-                    'deferred': False,
-                    'description': '',
-                    'file_key_name': 'models.model',
-                    'fqn': ['test', 'not_null_model_id'],
-                    'name': 'not_null_model_id',
-                    'original_file_path': model_schema_yml_path,
-                    'package_name': 'test',
-                    'patch_path': None,
-                    'path': Normalized('not_null_model_id.sql'),
-                    'raw_sql': "{{ test_not_null(**_dbt_generic_test_kwargs) }}",
-                    'refs': [['model']],
-                    'relation_name': None,
-                    'resource_type': 'test',
-                    'root_path': self.test_root_realpath,
-                    'schema': test_audit_schema,
-                    'database': self.default_database,
-                    'tags': [],
-                    'meta': {},
-                    'unique_id': 'test.test.not_null_model_id.d01cc630e6',
-                    'docs': {'show': True},
-                    'compiled': True,
-                    'compiled_sql': AnyStringWith('where id is null'),
-                    'extra_ctes_injected': True,
-                    'extra_ctes': [],
-                    'test_metadata': {
-                        'namespace': None,
-                        'name': 'not_null',
-                        'kwargs': {
-                            'column_name': 'id',
-                            'model': "{{ get_where_subquery(ref('model')) }}",
-                        },
-                    },
-                    'checksum': {'name': 'none', 'checksum': ''},
-                    'unrendered_config': unrendered_test_config,
-                },
-                'snapshot.test.snapshot_seed': {
-                    'alias': 'snapshot_seed',
-                    'compiled_path': None,
-                    'build_path': None,
-                    'created_at': ANY,
-                    'checksum': self._checksum_file(snapshot_path),
-                    'columns': {},
-                    'compiled': True,
-                    'compiled_sql': ANY,
-                    'config': snapshot_config,
-                    'database':  self.default_database,
-                    'deferred': False,
-                    'depends_on': {
-                        'macros': [],
-                        'nodes': ['seed.test.seed'],
-                    },
-                    'description': '',
-                    'docs': {'show': True},
-                    'extra_ctes': [],
-                    'extra_ctes_injected': True,
-                    'fqn': ['test', 'snapshot_seed', 'snapshot_seed'],
-                    'meta': {},
-                    'name': 'snapshot_seed',
-                    'original_file_path': snapshot_path,
-                    'package_name': 'test',
-                    'patch_path': None,
-                    'path': normalize('snapshot_seed.sql'),
-                    'raw_sql': LineIndifferent(
-                        _read_file(snapshot_path)
-                            .replace('{% snapshot snapshot_seed %}', '')
-                            .replace('{% endsnapshot %}', '')),
-                    'refs': [['seed']],
-                    'relation_name': relation_name_node_format.format(
-                        self.default_database, self.alternate_schema,
-                        'snapshot_seed'
-                    ),
-                    'resource_type': 'snapshot',
-                    'root_path': self.test_root_realpath,
-                    'schema': self.alternate_schema,
-                    'sources': [],
-                    'tags': [],
-                    'unique_id': 'snapshot.test.snapshot_seed',
-                    'unrendered_config': unrendered_snapshot_config,
-                },
-                'test.test.test_nothing_model_.5d38568946': {
-                    'alias': 'test_nothing_model_',
-                    'compiled_path': Normalized('target/compiled/test/models/schema.yml/test_nothing_model_.sql'),
-                    'build_path': None,
-                    'created_at': ANY,
-                    'column_name': None,
-                    'columns': {},
-                    'config': test_config,
-                    'sources': [],
-                    'depends_on': {
-                        'macros': ['macro.test.test_nothing', 'macro.dbt.get_where_subquery'],
-                        'nodes': ['model.test.model'],
-                    },
-                    'deferred': False,
-                    'description': '',
-                    'file_key_name': 'models.model',
-                    'fqn': ['test', 'test_nothing_model_'],
-                    'name': 'test_nothing_model_',
-                    'original_file_path': model_schema_yml_path,
-                    'package_name': 'test',
-                    'patch_path': None,
-                    'path': normalize('test_nothing_model_.sql'),
-                    'raw_sql': "{{ test.test_nothing(**_dbt_generic_test_kwargs) }}",
-                    'refs': [['model']],
-                    'relation_name': None,
-                    'resource_type': 'test',
-                    'root_path': self.test_root_realpath,
-                    'schema': test_audit_schema,
-                    'database': self.default_database,
-                    'tags': [],
-                    'meta': {},
-                    'unique_id': 'test.test.test_nothing_model_.5d38568946',
-                    'docs': {'show': True},
-                    'compiled': True,
-                    'compiled_sql': AnyStringWith('select 0'),
-                    'extra_ctes_injected': True,
-                    'extra_ctes': [],
-                    'test_metadata': {
-                        'namespace': 'test',
-                        'name': 'nothing',
-                        'kwargs': {
-                            'model': "{{ get_where_subquery(ref('model')) }}",
-                        },
-                    },
-                    'checksum': {'name': 'none', 'checksum': ''},
-                    'unrendered_config': unrendered_test_config,
-                },
-                'test.test.unique_model_id.67b76558ff': {
-                    'alias': 'unique_model_id',
-                    'compiled_path': Normalized('target/compiled/test/models/schema.yml/unique_model_id.sql'),
-                    'build_path': None,
-                    'created_at': ANY,
-                    'column_name': 'id',
-                    'columns': {},
-                    'config': test_config,
-                    'sources': [],
-                    'depends_on': {
-                        'macros': ['macro.dbt.test_unique', 'macro.dbt.get_where_subquery'],
-                        'nodes': ['model.test.model'],
-                    },
-                    'deferred': False,
-                    'description': '',
-                    'file_key_name': 'models.model',
-                    'fqn': ['test', 'unique_model_id'],
-                    'name': 'unique_model_id',
-                    'original_file_path': model_schema_yml_path,
-                    'package_name': 'test',
-                    'patch_path': None,
-                    'path': normalize('unique_model_id.sql'),
-                    'raw_sql': "{{ test_unique(**_dbt_generic_test_kwargs) }}",
-                    'refs': [['model']],
-                    'relation_name': None,
-                    'resource_type': 'test',
-                    'root_path': self.test_root_realpath,
-                    'schema': test_audit_schema,
-                    'database': self.default_database,
-                    'tags': [],
-                    'meta': {},
-                    'unique_id': 'test.test.unique_model_id.67b76558ff',
-                    'docs': {'show': True},
-                    'compiled': True,
-                    'compiled_sql': AnyStringWith('count(*)'),
-                    'extra_ctes_injected': True,
-                    'extra_ctes': [],
-                    'test_metadata': {
-                        'namespace': None,
-                        'name': 'unique',
-                        'kwargs': {
-                            'column_name': 'id',
-                            'model': "{{ get_where_subquery(ref('model')) }}",
-                        },
-                    },
-                    'checksum': {'name': 'none', 'checksum': ''},
-                    'unrendered_config': unrendered_test_config,
-                },
-            },
-            'sources': {
-                'source.test.my_source.my_table': {
-                    'created_at': ANY,
-                    'columns': {
-                        'id': {
-                            'description': 'An ID field',
-                            'name': 'id',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        }
-                    },
-                    'config': {
-                        'enabled': True,
-                    },
-                    'quoting': {
-                        'database': None,
-                        'schema': None,
-                        'identifier': True,
-                        'column': None,
-                    },
-                    'database': self.default_database,
-                    'description': 'My table',
-                    'external': None,
-                    'freshness': {'error_after': {'count': None, 'period': None}, 'warn_after': {'count': None, 'period': None}, 'filter': None},
-                    'identifier': 'seed',
-                    'loaded_at_field': None,
-                    'loader': 'a_loader',
-                    'meta': {},
-                    'name': 'my_table',
-                    'original_file_path': self.dir('models/schema.yml'),
-                    'package_name': 'test',
-                    'path': self.dir('models/schema.yml'),
-                    'patch_path': None,
-                    'relation_name': relation_name_source_format.format(
-                        self.default_database, my_schema_name, 'seed'
-                    ),
-                    'resource_type': 'source',
-                    'root_path': self.test_root_realpath,
-                    'schema': my_schema_name,
-                    'source_description': 'My source',
-                    'source_name': 'my_source',
-                    'source_meta': {},
-                    'tags': [],
-                    'unique_id': 'source.test.my_source.my_table',
-                    'fqn': ['test', 'my_source', 'my_table'],
-                    'unrendered_config': {},
-                },
-            },
-            'exposures': {
-                'exposure.test.notebook_exposure': {
-                    'created_at': ANY,
-                    'depends_on': {
-                        'macros': [],
-                        'nodes': ['model.test.model', 'model.test.second_model']
-                    },
-                    'description': 'A description of the complex exposure\n',
-                    'fqn': ['test', 'notebook_exposure'],
-                    'maturity': 'medium',
-                    'meta': {'tool': 'my_tool', 'languages': ['python']},
-                    'tags': ['my_department'],
-                    'name': 'notebook_exposure',
-                    'original_file_path': self.dir('models/schema.yml'),
-                    'owner': {
-                        'email': 'something@example.com',
-                        'name': 'Some name'
-                    },
-                    'package_name': 'test',
-                    'path': 'schema.yml',
-                    'refs': [['model'], ['second_model']],
-                    'resource_type': 'exposure',
-                    'root_path': self.test_root_realpath,
-                    'sources': [],
-                    'type': 'notebook',
-                    'unique_id': 'exposure.test.notebook_exposure',
-                    'url': 'http://example.com/notebook/1'
-                },
-                'exposure.test.simple_exposure': {
-                    'created_at': ANY,
-                    'depends_on': {
-                        'macros': [],
-                        'nodes': [
-                            'source.test.my_source.my_table',
-                            'model.test.model'
-                        ],
-                    },
-                    'description': '',
-                    'fqn': ['test', 'simple_exposure'],
-                    'name': 'simple_exposure',
-                    'original_file_path': self.dir('models/schema.yml'),
-                    'owner': {
-                        'email': 'something@example.com',
-                        'name': None,
-                    },
-                    'package_name': 'test',
-                    'path': 'schema.yml',
-                    'refs': [['model']],
-                    'resource_type': 'exposure',
-                    'root_path': self.test_root_realpath,
-                    'sources': [['my_source', 'my_table']],
-                    'type': 'dashboard',
-                    'unique_id': 'exposure.test.simple_exposure',
-                    'url': None,
-                    'maturity': None,
-                    'meta': {},
-                    'tags': []
-                }
-            },
-            'metrics': {},
-            'selectors': {},
-            'parent_map': {
-                'model.test.model': ['seed.test.seed'],
-                'model.test.second_model': ['seed.test.seed'],
-                'exposure.test.notebook_exposure': ['model.test.model', 'model.test.second_model'],
-                'exposure.test.simple_exposure': ['model.test.model', 'source.test.my_source.my_table'],
-                'seed.test.seed': [],
-                'snapshot.test.snapshot_seed': ['seed.test.seed'],
-                'source.test.my_source.my_table': [],
-                'test.test.not_null_model_id.d01cc630e6': ['model.test.model'],
-                'test.test.test_nothing_model_.5d38568946': ['model.test.model'],
-                'test.test.unique_model_id.67b76558ff': ['model.test.model'],
-            },
-            'child_map': {
-                'model.test.model': [
-                    'exposure.test.notebook_exposure',
-                    'exposure.test.simple_exposure',
-                    'test.test.not_null_model_id.d01cc630e6',
-                    'test.test.test_nothing_model_.5d38568946',
-                    'test.test.unique_model_id.67b76558ff',
-                ],
-                'model.test.second_model': ['exposure.test.notebook_exposure'],
-                'exposure.test.notebook_exposure': [],
-                'exposure.test.simple_exposure': [],
-                'seed.test.seed': ['model.test.model',
-                                   'model.test.second_model',
-                                   'snapshot.test.snapshot_seed'],
-                'snapshot.test.snapshot_seed': [],
-                'source.test.my_source.my_table': ['exposure.test.simple_exposure'],
-                'test.test.not_null_model_id.d01cc630e6': [],
-                'test.test.test_nothing_model_.5d38568946': [],
-                'test.test.unique_model_id.67b76558ff': [],
-            },
-            'docs': {
-                'dbt.__overview__': ANY,
-                'test.macro_info': ANY,
-                'test.macro_arg_info': ANY,
-            },
-            'disabled': {},
-        }
-
-    def expected_bigquery_complex_manifest(self):
-        nested_view_sql_path = self.dir('bq_models/nested_view.sql')
-        nested_table_sql_path = self.dir('bq_models/nested_table.sql')
-        clustered_sql_path = self.dir('bq_models/clustered.sql')
-        multi_clustered_sql_path = self.dir('bq_models/multi_clustered.sql')
-        seed_path = self.dir('seed/seed.csv')
-        snapshot_path = self.dir('snapshot/snapshot_seed.sql')
-        my_schema_name = self.unique_schema()
-
-        return {
-            'dbt_schema_version': 'https://schemas.getdbt.com/dbt/manifest/v5.json',
-            'dbt_version': dbt.version.__version__,
-            'nodes': {
-                'model.test.clustered': {
-                    'alias': 'clustered',
-                    'config': self.rendered_model_config(
-                        cluster_by=['first_name'],
-                        materialized='table',
-                        partition_by={'field': 'updated_at',
-                                      'data_type': 'date'},
-                    ),
-                    'sources': [],
-                    'depends_on': {'macros': [], 'nodes': ['seed.test.seed']},
-                    'fqn': ['test', 'clustered'],
-                    'compiled_path': Normalized('target/compiled/test/bq_models/clustered.sql'),
-                    'build_path': None,
-                    'created_at': ANY,
-                    'name': 'clustered',
-                    'original_file_path': clustered_sql_path,
-                    'package_name': 'test',
-                    'path': 'clustered.sql',
-                    'raw_sql': LineIndifferent(_read_file(clustered_sql_path).rstrip('\r\n')),
-                    'refs': [['seed']],
-                    'relation_name': '`{0}`.`{1}`.clustered'.format(
-                        self.default_database, my_schema_name
-                    ),
-                    'resource_type': 'model',
-                    'root_path': self.test_root_realpath,
-                    'schema': my_schema_name,
-                    'database': self.default_database,
-                    'tags': [],
-                    'meta': {},
-                    'unique_id': 'model.test.clustered',
-                    'columns': {
-                        'email': {
-                            'description': "The user's email",
-                            'name': 'email',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'first_name': {
-                            'description': "The user's name",
-                            'name': 'first_name',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'id': {
-                            'description': 'The user id',
-                            'name': 'id',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'ip_address': {
-                            'description': "The user's IP address",
-                            'name': 'ip_address',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'updated_at': {
-                            'description': 'When the user was updated',
-                            'name': 'updated_at',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                    },
-                    'deferred': False,
-                    'description': 'A clustered and partitioned copy of the test model',
-                    'patch_path': 'test://' + self.dir('bq_models/schema.yml'),
-                    'docs': {'show': True},
-                    'compiled': True,
-                    'compiled_sql': ANY,
-                    'extra_ctes_injected': True,
-                    'extra_ctes': [],
-                    'checksum': self._checksum_file(clustered_sql_path),
-                    'unrendered_config': self.unrendered_model_config(
-                        cluster_by=['first_name'],
-                        materialized='table',
-                        partition_by={'field': 'updated_at',
-                                      'data_type': 'date'},
-                    ),
-                },
-                'model.test.multi_clustered': {
-                    'alias': 'multi_clustered',
-                    'compiled_path': Normalized('target/compiled/test/bq_models/multi_clustered.sql'),
-                    'build_path': None,
-                    'created_at': ANY,
-                    'config': self.rendered_model_config(
-                        cluster_by=['first_name', 'email'],
-                        materialized='table',
-                        partition_by={'field': 'updated_at',
-                                      'data_type': 'date'}
-                    ),
-                    'sources': [],
-                    'depends_on': {'macros': [], 'nodes': ['seed.test.seed']},
-                    'fqn': ['test', 'multi_clustered'],
-                    'name': 'multi_clustered',
-                    'original_file_path': multi_clustered_sql_path,
-                    'package_name': 'test',
-                    'path': 'multi_clustered.sql',
-                    'raw_sql': LineIndifferent(_read_file(multi_clustered_sql_path).rstrip('\r\n')),
-                    'refs': [['seed']],
-                    'relation_name': '`{0}`.`{1}`.multi_clustered'.format(
-                        self.default_database, my_schema_name
-                    ),
-                    'resource_type': 'model',
-                    'root_path': self.test_root_realpath,
-                    'schema': my_schema_name,
-                    'database': self.default_database,
-                    'tags': [],
-                    'meta': {},
-                    'unique_id': 'model.test.multi_clustered',
-                    'columns': {
-                        'email': {
-                            'description': "The user's email",
-                            'name': 'email',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'first_name': {
-                            'description': "The user's name",
-                            'name': 'first_name',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'id': {
-                            'description': 'The user id',
-                            'name': 'id',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'ip_address': {
-                            'description': "The user's IP address",
-                            'name': 'ip_address',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'updated_at': {
-                            'description': 'When the user was updated',
-                            'name': 'updated_at',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                    },
-                    'deferred': False,
-                    'description': 'A clustered and partitioned copy of the test model, clustered on multiple columns',
-                    'patch_path': 'test://' + self.dir('bq_models/schema.yml'),
-                    'docs': {'show': True},
-                    'compiled': True,
-                    'compiled_sql': ANY,
-                    'extra_ctes_injected': True,
-                    'extra_ctes': [],
-                    'checksum': self._checksum_file(multi_clustered_sql_path),
-                    'unrendered_config': self.unrendered_model_config(
-                        cluster_by=['first_name', 'email'],
-                        materialized='table',
-                        partition_by={'field': 'updated_at',
-                                      'data_type': 'date'}
-                    ),
-                },
-                'model.test.nested_view': {
-                    'alias': 'nested_view',
-                    'compiled_path': Normalized('target/compiled/test/bq_models/nested_view.sql'),
-                    'build_path': None,
-                    'created_at': ANY,
-                    'config': self.rendered_model_config(),
-                    'sources': [],
-                    'depends_on': {
-                        'macros': [],
-                        'nodes': ['model.test.nested_table']
-                    },
-                    'fqn': ['test', 'nested_view'],
-                    'name': 'nested_view',
-                    'original_file_path': nested_view_sql_path,
-                    'package_name': 'test',
-                    'path': 'nested_view.sql',
-                    'raw_sql': LineIndifferent(_read_file(nested_view_sql_path).rstrip('\r\n')),
-                    'refs': [['nested_table']],
-                    'relation_name': '`{0}`.`{1}`.nested_view'.format(
-                        self.default_database, my_schema_name
-                    ),
-                    'resource_type': 'model',
-                    'root_path': self.test_root_realpath,
-                    'schema': my_schema_name,
-                    'database': self.default_database,
-                    'tags': [],
-                    'meta': {},
-                    'unique_id': 'model.test.nested_view',
-                    'columns': {
-                        'field_1': {
-                            'name': 'field_1',
-                            'description': 'The first field',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'field_2': {
-                            'name': 'field_2',
-                            'description': 'The second field',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'field_3': {
-                            'name': 'field_3',
-                            'description': 'The third field',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'nested_field.field_4': {
-                            'name': 'nested_field.field_4',
-                            'description': 'The first nested field',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'nested_field.field_5': {
-                            'name': 'nested_field.field_5',
-                            'description': 'The second nested field',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                    },
-                    'deferred': False,
-                    'description': 'The test model',
-                    'patch_path': 'test://' + self.dir('bq_models/schema.yml'),
-                    'docs': {'show': True},
-                    'compiled': True,
-                    'compiled_sql': ANY,
-                    'extra_ctes_injected': True,
-                    'extra_ctes': [],
-                    'checksum': self._checksum_file(nested_view_sql_path),
-                    'unrendered_config': self.unrendered_model_config(materialized='view'),
-                },
-                'model.test.nested_table': {
-                    'alias': 'nested_table',
-                    'compiled_path': Normalized('target/compiled/test/bq_models/nested_table.sql'),
-                    'build_path': None,
-                    'created_at': ANY,
-                    'config': self.rendered_model_config(materialized='table'),
-                    'sources': [],
-                    'depends_on': {
-                        'macros': [],
-                        'nodes': []
-                    },
-                    'fqn': ['test', 'nested_table'],
-                    'name': 'nested_table',
-                    'original_file_path': nested_table_sql_path,
-                    'package_name': 'test',
-                    'patch_path': None,
-                    'path': 'nested_table.sql',
-                    'raw_sql': LineIndifferent(_read_file(nested_table_sql_path).rstrip('\r\n')),
-                    'refs': [],
-                    'relation_name': '`{0}`.`{1}`.nested_table'.format(
-                        self.default_database, my_schema_name
-                    ),
-                    'resource_type': 'model',
-                    'root_path': self.test_root_realpath,
-                    'schema': my_schema_name,
-                    'database': self.default_database,
-                    'tags': [],
-                    'meta': {},
-                    'unique_id': 'model.test.nested_table',
-                    'columns': {},
-                    'deferred': False,
-                    'description': '',
-                    'docs': {'show': True},
-                    'compiled': True,
-                    'compiled_sql': ANY,
-                    'extra_ctes_injected': True,
-                    'extra_ctes': [],
-                    'checksum': self._checksum_file(nested_table_sql_path),
-                    'unrendered_config': self.unrendered_model_config(materialized='table'),
-                },
-                'seed.test.seed': {
-                    'compiled_path': None,
-                    'build_path': None,
-                    'created_at': ANY,
-                    'patch_path': 'test://' + self.dir('seed/schema.yml'),
-                    'path': 'seed.csv',
-                    'name': 'seed',
-                    'root_path': self.test_root_realpath,
-                    'relation_name': '`{0}`.`{1}`.seed'.format(
-                        self.default_database, my_schema_name
-                    ),
-                    'resource_type': 'seed',
-                    'raw_sql': '',
-                    'package_name': 'test',
-                    'original_file_path': seed_path,
-                    'refs': [],
-                    'sources': [],
-                    'depends_on': {
-                        'nodes': [],
-                        'macros': [],
-                    },
-                    'unique_id': 'seed.test.seed',
-                    'fqn': ['test', 'seed'],
-                    'tags': [],
-                    'meta': {},
-                    'config': self.rendered_seed_config(),
-                    'schema': my_schema_name,
-                    'database': self.default_database,
-                    'alias': 'seed',
-                    'columns': {
-                        'id': {
-                            'name': 'id',
-                            'description': 'The user ID number',
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'first_name': {
-                            'name': 'first_name',
-                            'description': "The user's first name",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'email': {
-                            'name': 'email',
-                            'description': "The user's email",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'ip_address': {
-                            'name': 'ip_address',
-                            'description': "The user's IP address",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                        'updated_at': {
-                            'name': 'updated_at',
-                            'description': "The last time this user's email was updated",
-                            'data_type': None,
-                            'meta': {},
-                            'quote': None,
-                            'tags': [],
-                        },
-                    },
-                    'deferred': False,
-                    'description': 'The test seed',
-                    'docs': {'show': True},
-                    'compiled': True,
-                    'compiled_sql': '',
-                    'extra_ctes_injected': True,
-                    'extra_ctes': [],
-                    'checksum': self._checksum_file(seed_path),
-                    'unrendered_config': self.unrendered_seed_config(),
-                },
-                'snapshot.test.snapshot_seed': {
-                    'alias': 'snapshot_seed',
-                    'compiled_path': None,
-                    'build_path': None,
-                    'created_at': ANY,
-                    'checksum': self._checksum_file(snapshot_path),
-                    'columns': {},
-                    'compiled': True,
-                    'compiled_sql': ANY,
-                    'config': self.rendered_snapshot_config(
-                        target_schema=self.alternate_schema
-                    ),
-                    'database': self.default_database,
-                    'deferred': False,
-                    'depends_on': {'macros': [],
-                                   'nodes': ['seed.test.seed']},
-                    'description': '',
-                    'docs': {'show': True},
-                    'extra_ctes': [],
-                    'extra_ctes_injected': True,
-                    'fqn': ['test', 'snapshot_seed', 'snapshot_seed'],
-                    'meta': {},
-                    'name': 'snapshot_seed',
-                    'original_file_path': snapshot_path,
-                    'package_name': 'test',
-                    'patch_path': None,
-                    'path': 'snapshot_seed.sql',
-                    'raw_sql': ANY,
-                    'refs': [['seed']],
-                    'relation_name': '`{0}`.`{1}`.snapshot_seed'.format(
-                        self.default_database, self.alternate_schema
-                    ),
-                    'resource_type': 'snapshot',
-                    'root_path': self.test_root_realpath,
-                    'schema': self.alternate_schema,
-                    'sources': [],
-                    'tags': [],
-                    'unique_id': 'snapshot.test.snapshot_seed',
-                    'unrendered_config': self.unrendered_snapshot_config(
-                        target_schema=self.alternate_schema
-                    )}
-            },
-            'sources': {},
-            'exposures': {},
-            'metrics': {},
-            'selectors': {},
-            'child_map': {
-                'model.test.clustered': [],
-                'model.test.multi_clustered': [],
-                'model.test.nested_table': ['model.test.nested_view'],
-                'model.test.nested_view': [],
-                'seed.test.seed': ['model.test.clustered',
-                                   'model.test.multi_clustered',
-                                   'snapshot.test.snapshot_seed'],
-                'snapshot.test.snapshot_seed': []
-            },
-            'parent_map': {
-                'model.test.clustered': ['seed.test.seed'],
-                'model.test.multi_clustered': ['seed.test.seed'],
-                'seed.test.seed': [],
-                'snapshot.test.snapshot_seed': ['seed.test.seed'],
-                'model.test.nested_table': [],
-                'model.test.nested_view': ['model.test.nested_table'],
-            },
-            'docs': {
-                'dbt.__overview__': ANY,
-                'test.macro_info': ANY,
-                'test.macro_arg_info': ANY,
-            },
-            'disabled': {},
-        }
-
-    def _checksum_file(self, path):
-        """windows has silly git behavior that adds newlines, and python does
-        silly things if we just open(..., 'r').encode('utf-8').
-        """
-        with open(self.dir(path), 'rb') as fp:
-            hashed = hashlib.sha256(fp.read()).hexdigest()
-        return {
-            'name': 'sha256',
-            'checksum': hashed,
-        }
-
-    def _path_to(self, searched_path: str, relative_path: str):
-        return {
-            'searched_path': normalize(searched_path),
-            'relative_path': normalize(relative_path),
-            'project_root': normalize(self.test_root_dir),
-        }
-
-    def _absolute_path_to(self, searched_path: str, relative_path: str):
-        return os.path.join(
-            normalize(self.test_root_dir),
-            normalize(searched_path),
-            normalize(relative_path)
-        )
-
-    def _relation_name_format(self, quote_database: bool, quote_schema: bool,
-                              quote_identifier: bool):
-        return ".".join((
-            self._quote("{0}") if quote_database else '{0}',
-            self._quote("{1}") if quote_schema else '{1}',
-            self._quote("{2}") if quote_identifier else '{2}',
-        ))
-
-    def verify_metadata(self, metadata, dbt_schema_version):
-        assert 'generated_at' in metadata
-        self.assertBetween(metadata['generated_at'],
-                           start=self.generate_start_time)
-        assert 'dbt_version' in metadata
-        assert metadata['dbt_version'] == dbt.version.__version__
-        assert 'dbt_schema_version' in metadata
-        assert metadata['dbt_schema_version'] == dbt_schema_version
-        assert metadata['invocation_id'] == dbt.tracking.active_user.invocation_id
-        key = 'env_key'
-        if os.name == 'nt':
-            key = key.upper()
-        assert metadata['env'] == {
-            key: 'env_value'
-        }
-
-    def verify_manifest(self, expected_manifest):
-        self.assertTrue(os.path.exists('./target/manifest.json'))
-
-        manifest = _read_json('./target/manifest.json')
-
-        manifest_keys = frozenset({
-            'nodes', 'sources', 'macros', 'parent_map', 'child_map',
-            'docs', 'metadata', 'docs', 'disabled', 'exposures', 'selectors',
-            'metrics',
-        })
-
-        self.assertEqual(frozenset(manifest), manifest_keys)
-
-        for key in manifest_keys:
-            if key == 'macros':
-                self.verify_manifest_macros(
-                    manifest, expected_manifest.get('macros'))
-            elif key == 'metadata':
-                metadata = manifest['metadata']
-                self.verify_metadata(
-                    metadata, 'https://schemas.getdbt.com/dbt/manifest/v5.json')
-                assert 'project_id' in metadata and metadata[
-                    'project_id'] == '098f6bcd4621d373cade4e832627b4f6'
-                assert 'send_anonymous_usage_stats' in metadata and metadata[
-                    'send_anonymous_usage_stats'] is False
-                assert 'adapter_type' in metadata and metadata['adapter_type'] == self.adapter_type
-            else:
-                self.assertIn(key, expected_manifest)  # sanity check
-                self.assertEqual(manifest[key], expected_manifest[key])
-                
-    def _quote(self, value):
-        quote_char = '`'
-        return '{0}{1}{0}'.format(quote_char, value)
-
-    def expected_run_results(self):
-        """
-        The expected results of this run.
-        """
-
-        return [
-            {
-                'status': 'success',
-                'message': None,
-                'execution_time': AnyFloat(),
-                'unique_id': 'model.test.model',
-                'adapter_response': ANY,
-                'thread_id': ANY,
-                'timing': [ANY, ANY],
-                'failures': ANY,
-            },
-            {
-                'status': 'success',
-                'message': None,
-                'execution_time': AnyFloat(),
-                'unique_id': 'model.test.second_model',
-                'adapter_response': ANY,
-                'thread_id': ANY,
-                'timing': [ANY, ANY],
-                'failures': ANY,
-            },
-            {
-                'status': 'success',
-                'message': None,
-                'execution_time': AnyFloat(),
-                'unique_id': 'seed.test.seed',
-                'adapter_response': ANY,
-                'thread_id': ANY,
-                'timing': [ANY, ANY],
-                'failures': ANY,
-            },
-            {
-                'status': 'success',
-                'message': None,
-                'execution_time': AnyFloat(),
-                'unique_id': 'snapshot.test.snapshot_seed',
-                'adapter_response': ANY,
-                'thread_id': ANY,
-                'timing': [ANY, ANY],
-                'failures': ANY,
-            },
-            {
-                'status': 'success',
-                'message': None,
-                'execution_time': AnyFloat(),
-                'unique_id': 'test.test.not_null_model_id.d01cc630e6',
-                'adapter_response': ANY,
-                'thread_id': ANY,
-                'timing': [ANY, ANY],
-                'failures': ANY,
-            },
-            {
-                'status': 'success',
-                'message': None,
-                'execution_time': AnyFloat(),
-                'unique_id': 'test.test.test_nothing_model_.5d38568946',
-                'adapter_response': ANY,
-                'thread_id': ANY,
-                'timing': [ANY, ANY],
-                'failures': ANY,
-            },
-            {
-                'status': 'success',
-                'message': None,
-                'execution_time': AnyFloat(),
-                'unique_id': 'test.test.unique_model_id.67b76558ff',
-                'adapter_response': ANY,
-                'thread_id': ANY,
-                'timing': [ANY, ANY],
-                'failures': ANY,
-            },
-        ]
-
-    def verify_run_results(self, expected_run_results):
-        run_results = _read_json('./target/run_results.json')
-        assert 'metadata' in run_results
-        self.verify_metadata(
-            run_results['metadata'], 'https://schemas.getdbt.com/dbt/run-results/v4.json')
-        self.assertIn('elapsed_time', run_results)
-        self.assertGreater(run_results['elapsed_time'], 0)
-        self.assertTrue(
-            isinstance(run_results['elapsed_time'], float),
-            "run_results['elapsed_time'] is of type {}, expected float".format(
-                str(type(run_results['elapsed_time'])))
-        )
-
-        assert 'args' in run_results
-        # sort the results so we can make reasonable assertions
-        run_results['results'].sort(key=lambda r: r['unique_id'])
-        assert run_results['results'] == expected_run_results
-        set(run_results) == {'elapsed_time', 'results', 'metadata'}
-
-    @use_profile('bigquery')
-    def test__bigquery__run_and_generate(self):
-        self.run_and_generate()
-
-        self.verify_catalog(self.expected_bigquery_catalog())
-        self.verify_manifest(self.expected_seeded_manifest())
-        self.verify_run_results(self.expected_run_results())
-
-    @use_profile('bigquery')
-    def test__bigquery__complex_models(self):
-        self.run_and_generate(
-            extra={'model-paths': [self.dir('bq_models')]},
-            model_count=4
-        )
-
-        self.verify_catalog(self.expected_bigquery_complex_catalog())
-        self.verify_manifest(self.expected_bigquery_complex_manifest())
-
-
-class TestDocsGenerateMissingSchema(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return 'docs_generate'
-
-    @staticmethod
-    def dir(path):
-        return normalize(path)
-
-    @property
-    def models(self):
-        return self.dir("bq_models_noschema")
-
-    def setUp(self):
-        super().setUp()
-        self.extra_schema = self.unique_schema() + '_bq_test'
-
-    def tearDown(self):
-        with self.adapter.connection_named('__test'):
-            self._drop_schema_named(self.default_database, self.extra_schema)
-        super().tearDown()
-
-    @use_profile('bigquery')
-    def test_bigquery_docs_generate_noschema(self):
-        self.run_dbt([
-            'docs', 'generate',
-            '--vars', "{{extra_schema: {}}}".format(self.extra_schema)
-        ])
diff --git a/tests/integration/docs_generate_tests/trivial_models/model.sql b/tests/integration/docs_generate_tests/trivial_models/model.sql
deleted file mode 100644
index 43258a714..000000000
--- a/tests/integration/docs_generate_tests/trivial_models/model.sql
+++ /dev/null
@@ -1 +0,0 @@
-select 1 as id
diff --git a/tox.ini b/tox.ini
index 348be15af..80191db8c 100644
--- a/tox.ini
+++ b/tox.ini
@@ -17,7 +17,7 @@ skip_install = true
 passenv = DBT_* BIGQUERY_TEST_* PYTEST_ADDOPTS
 commands =
   bigquery: {envpython} -m pytest {posargs} -m profile_bigquery tests/integration
-  bigquery: {envpython} -m pytest {posargs} tests/functional --profile service_account
+  bigquery: {envpython} -m pytest {posargs} -vv tests/functional --profile service_account
 deps =
   -rdev-requirements.txt
   -e.

From 799790e149b980466d966b52a42c33eec4285ef2 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 1 Jun 2022 11:16:19 -0500
Subject: [PATCH 447/860] Add regression test case (#188)

* init push or pytest conversion of override_database for bigquery

* tests are currently passing, need to implement some test changes overs (commented out tests)

* implemented a vew versions of the check_realations_equal test on test_bigquery_database_override test working on getting the rest to work for first test

* trying to grab local env to test against need to find way to grab information from configs

* attempting to move over to using check_relations_equal_with_relations

* ordering of models

* post pair update

* debugged test, had passing locally

* adding change to conftest to see if it passes tests in ci/cd

* removing conftest change

* trying ci/cd after some changes

* trying to take into account CT-604, remove old test, remove BIGQUERY_TEST_DATABASE env

* moving override_database out of adapter folder up one

* adding regression test case draft

* changing filename to match yml, changes tests -> macros to define path
---
 .../test_get_columns_incomplete_database.py   | 54 +++++++++++++++++++
 1 file changed, 54 insertions(+)
 create mode 100644 tests/functional/test_get_columns_incomplete_database.py

diff --git a/tests/functional/test_get_columns_incomplete_database.py b/tests/functional/test_get_columns_incomplete_database.py
new file mode 100644
index 000000000..35e37779d
--- /dev/null
+++ b/tests/functional/test_get_columns_incomplete_database.py
@@ -0,0 +1,54 @@
+import pytest
+from dbt.tests.util import run_dbt
+
+# This is to test a edge case found in https://github.com/dbt-labs/dbt-bigquery/pull/165/files
+
+tests__get_cols_in_sql = """
+{% test get_cols_in(model) %}
+
+  {# The step which causes the issue #}
+  {%- set relation = api.Relation.create(identifier=model.table) if execute -%}
+
+  {% set columns = adapter.get_columns_in_relation(relation) %}
+
+  select
+    {% for col in columns %}
+      {{ col.name }} {{ "," if not loop.last }}
+    {% endfor %}
+
+  from {{ model }}
+  limit 0
+
+{% endtest %}
+"""
+
+models__my_model = """select 1 as id, 'text' as another_col
+"""
+
+properties__model_yml = """
+version: 2
+models:
+  - name: my_model
+    tests:
+      - get_cols_in
+"""
+
+class TestIncompleteRelationSetup:
+    @pytest.fixture(scope="class")
+    def properties(self):
+        return {"properties__model_yml.yml": properties__model_yml}
+
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {"get_col_in.sql": tests__get_cols_in_sql}
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return { "my_model.sql": models__my_model }
+
+
+class TestIncompleteRelation(TestIncompleteRelationSetup):
+
+  def test_incomplete_relation(self, project):
+     run_dbt(["build"])
+

From 1b85fc50287ac9afd81fd92e715dffa5f0757379 Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Wed, 1 Jun 2022 13:55:02 -0400
Subject: [PATCH 448/860] Fixing pip update for CI (#196)

---
 .github/workflows/integration.yml  |  8 ++++----
 .github/workflows/main.yml         | 30 +++++++++++++++---------------
 .github/workflows/version-bump.yml |  4 ++--
 3 files changed, 21 insertions(+), 21 deletions(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 2097fded1..439ed1820 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -153,14 +153,14 @@ jobs:
 
       - name: Install python dependencies
         run: |
-          pip install --user --upgrade pip
-          pip install tox
-          pip --version
+          python -m pip install --user --upgrade pip
+          python -m pip install tox
+          python -m pip --version
           tox --version
 
       - name: Install dbt-core latest
         run: |
-          pip install "git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core"
+          python -m pip install "git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core"
 
       - name: Run tox (bigquery)
         if: matrix.adapter == 'bigquery'
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 2e961dfa7..9e6ef8688 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -52,9 +52,9 @@ jobs:
 
       - name: Install python dependencies
         run: |
-          pip install --user --upgrade pip
-          pip install -r dev-requirements.txt
-          pip --version
+          python -m pip install --user --upgrade pip
+          python -m pip install -r dev-requirements.txt
+          python -m pip --version
           pre-commit --version
           mypy --version
           dbt --version
@@ -88,13 +88,13 @@ jobs:
 
       - name: Install python dependencies
         run: |
-          pip install --user --upgrade pip
-          pip install tox
-          pip --version
+          python -m pip install --user --upgrade pip
+          python -m pip install tox
+          python -m pip --version
           tox --version
       - name: Install dbt-core latest
         run: |
-          pip install "git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core"
+          python -m pip install "git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core"
       - name: Run tox
         run: tox
 
@@ -130,9 +130,9 @@ jobs:
 
       - name: Install python dependencies
         run: |
-          pip install --user --upgrade pip
-          pip install --upgrade setuptools wheel twine check-wheel-contents
-          pip --version
+          python -m pip install --user --upgrade pip
+          python -m pip install --upgrade setuptools wheel twine check-wheel-contents
+          python -m pip --version
 
       - name: Build distributions
         run: ./scripts/build-dist.sh
@@ -183,9 +183,9 @@ jobs:
 
       - name: Install python dependencies
         run: |
-          pip install --user --upgrade pip
-          pip install --upgrade wheel
-          pip --version
+          python -m pip install --user --upgrade pip
+          python -m pip install --upgrade wheel
+          python -m pip --version
       - uses: actions/download-artifact@v2
         with:
           name: dist
@@ -196,13 +196,13 @@ jobs:
 
       - name: Install wheel distributions
         run: |
-          find ./dist/*.whl -maxdepth 1 -type f | xargs pip install --force-reinstall --find-links=dist/
+          find ./dist/*.whl -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/
       - name: Check wheel distributions
         run: |
           dbt --version
       - name: Install source distributions
         run: |
-          find ./dist/*.gz -maxdepth 1 -type f | xargs pip install --force-reinstall --find-links=dist/
+          find ./dist/*.gz -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/
       - name: Check source distributions
         run: |
           dbt --version
diff --git a/.github/workflows/version-bump.yml b/.github/workflows/version-bump.yml
index dbd36786a..d9df91c52 100644
--- a/.github/workflows/version-bump.yml
+++ b/.github/workflows/version-bump.yml
@@ -57,7 +57,7 @@ jobs:
         run: |
           python3 -m venv env
           source env/bin/activate
-          pip install --upgrade pip
+          python -m pip install --upgrade pip
 
       - name: Create PR branch
         if: ${{ steps.variables.outputs.IS_DRY_RUN  == 'true' }}
@@ -69,7 +69,7 @@ jobs:
       - name: Bumping version
         run: |
           source env/bin/activate
-          pip install -r dev-requirements.txt
+          python -m pip install -r dev-requirements.txt
           env/bin/bumpversion --allow-dirty --new-version ${{steps.variables.outputs.VERSION_NUMBER}} major
           git status
 

From 87a68e55ea690fe805b004012e9701752b1ddfe5 Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Fri, 17 Jun 2022 07:30:17 -0600
Subject: [PATCH 449/860] Lift + shift for cross-db macros (#192)

* Initialize lift + shift, dateadd + datediff

* Switch to alternative dev branches

* Lift and shift cross-database macros from dbt-utils

* Switch namespace from `dbt_utils` to `dbt`

* Remove conflicting definition of current_timestamp()

* Trim trailing whitespace

* Copy functional tests for cross-database macros from dbt-utils

* Remove references to other profiles

* Kick out the `current_timestamp` + `type_*` macros/tests

* Remove branch identifiers

Co-authored-by: Jeremy Cohen 
---
 dbt/include/bigquery/macros/utils/bool_or.sql |   5 +
 .../bigquery/macros/utils/date_trunc.sql      |   7 ++
 dbt/include/bigquery/macros/utils/dateadd.sql |   8 ++
 .../bigquery/macros/utils/datediff.sql        |  15 +++
 .../macros/utils/escape_single_quotes.sql     |   4 +
 dbt/include/bigquery/macros/utils/except.sql  |   5 +
 dbt/include/bigquery/macros/utils/hash.sql    |   3 +
 .../bigquery/macros/utils/intersect.sql       |   5 +
 dbt/include/bigquery/macros/utils/listagg.sql |  14 +++
 .../bigquery/macros/utils/position.sql        |   9 ++
 dbt/include/bigquery/macros/utils/right.sql   |  12 ++
 .../bigquery/macros/utils/safe_cast.sql       |   3 +
 .../bigquery/macros/utils/split_part.sql      |  20 ++++
 tests/functional/adapter/test_utils.py        | 103 ++++++++++++++++++
 14 files changed, 213 insertions(+)
 create mode 100644 dbt/include/bigquery/macros/utils/bool_or.sql
 create mode 100644 dbt/include/bigquery/macros/utils/date_trunc.sql
 create mode 100644 dbt/include/bigquery/macros/utils/dateadd.sql
 create mode 100644 dbt/include/bigquery/macros/utils/datediff.sql
 create mode 100644 dbt/include/bigquery/macros/utils/escape_single_quotes.sql
 create mode 100644 dbt/include/bigquery/macros/utils/except.sql
 create mode 100644 dbt/include/bigquery/macros/utils/hash.sql
 create mode 100644 dbt/include/bigquery/macros/utils/intersect.sql
 create mode 100644 dbt/include/bigquery/macros/utils/listagg.sql
 create mode 100644 dbt/include/bigquery/macros/utils/position.sql
 create mode 100644 dbt/include/bigquery/macros/utils/right.sql
 create mode 100644 dbt/include/bigquery/macros/utils/safe_cast.sql
 create mode 100644 dbt/include/bigquery/macros/utils/split_part.sql
 create mode 100644 tests/functional/adapter/test_utils.py

diff --git a/dbt/include/bigquery/macros/utils/bool_or.sql b/dbt/include/bigquery/macros/utils/bool_or.sql
new file mode 100644
index 000000000..5b2e371f9
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/bool_or.sql
@@ -0,0 +1,5 @@
+{% macro bigquery__bool_or(expression) -%}
+
+    logical_or({{ expression }})
+
+{%- endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/date_trunc.sql b/dbt/include/bigquery/macros/utils/date_trunc.sql
new file mode 100644
index 000000000..f534c2014
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/date_trunc.sql
@@ -0,0 +1,7 @@
+{% macro bigquery__date_trunc(datepart, date) -%}
+    timestamp_trunc(
+        cast({{date}} as timestamp),
+        {{datepart}}
+    )
+
+{%- endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/dateadd.sql b/dbt/include/bigquery/macros/utils/dateadd.sql
new file mode 100644
index 000000000..8c6131512
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/dateadd.sql
@@ -0,0 +1,8 @@
+{% macro bigquery__dateadd(datepart, interval, from_date_or_timestamp) %}
+
+        datetime_add(
+            cast( {{ from_date_or_timestamp }} as datetime),
+        interval {{ interval }} {{ datepart }}
+        )
+
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/datediff.sql b/dbt/include/bigquery/macros/utils/datediff.sql
new file mode 100644
index 000000000..12bdcb3b0
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/datediff.sql
@@ -0,0 +1,15 @@
+{% macro bigquery__datediff(first_date, second_date, datepart) -%}
+
+  {% if dbt_version[0] == 1 and dbt_version[2] >= 2 %}
+    {{ return(dbt.datediff(first_date, second_date, datepart)) }}
+  {% else %}
+
+    datetime_diff(
+        cast({{second_date}} as datetime),
+        cast({{first_date}} as datetime),
+        {{datepart}}
+    )
+
+  {% endif %}
+
+{%- endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/escape_single_quotes.sql b/dbt/include/bigquery/macros/utils/escape_single_quotes.sql
new file mode 100644
index 000000000..0f39b4e85
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/escape_single_quotes.sql
@@ -0,0 +1,4 @@
+{# /*BigQuery uses a single backslash: they're -> they\'re. The second backslash is to escape it from Jinja */ #}
+{% macro bigquery__escape_single_quotes(expression) -%}
+{{ expression | replace("'", "\\'") }}
+{%- endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/except.sql b/dbt/include/bigquery/macros/utils/except.sql
new file mode 100644
index 000000000..561004722
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/except.sql
@@ -0,0 +1,5 @@
+{% macro bigquery__except() %}
+
+    except distinct
+
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/hash.sql b/dbt/include/bigquery/macros/utils/hash.sql
new file mode 100644
index 000000000..7150bf332
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/hash.sql
@@ -0,0 +1,3 @@
+{% macro bigquery__hash(field) -%}
+    to_hex({{dbt.default__hash(field)}})
+{%- endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/intersect.sql b/dbt/include/bigquery/macros/utils/intersect.sql
new file mode 100644
index 000000000..a2a348718
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/intersect.sql
@@ -0,0 +1,5 @@
+{% macro bigquery__intersect() %}
+
+    intersect distinct
+
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/listagg.sql b/dbt/include/bigquery/macros/utils/listagg.sql
new file mode 100644
index 000000000..928d36d21
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/listagg.sql
@@ -0,0 +1,14 @@
+{% macro bigquery__listagg(measure, delimiter_text, order_by_clause, limit_num) -%}
+
+    string_agg(
+        {{ measure }},
+        {{ delimiter_text }}
+        {% if order_by_clause -%}
+        {{ order_by_clause }}
+        {%- endif %}
+        {% if limit_num -%}
+        limit {{ limit_num }}
+        {%- endif %}
+        )
+
+{%- endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/position.sql b/dbt/include/bigquery/macros/utils/position.sql
new file mode 100644
index 000000000..0afc5676a
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/position.sql
@@ -0,0 +1,9 @@
+{% macro bigquery__position(substring_text, string_text) %}
+
+    strpos(
+        {{ string_text }},
+        {{ substring_text }}
+
+    )
+
+{%- endmacro -%}
diff --git a/dbt/include/bigquery/macros/utils/right.sql b/dbt/include/bigquery/macros/utils/right.sql
new file mode 100644
index 000000000..f0748a0a0
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/right.sql
@@ -0,0 +1,12 @@
+{% macro bigquery__right(string_text, length_expression) %}
+
+    case when {{ length_expression }} = 0
+        then ''
+    else
+        substr(
+            {{ string_text }},
+            -1 * ({{ length_expression }})
+        )
+    end
+
+{%- endmacro -%}
diff --git a/dbt/include/bigquery/macros/utils/safe_cast.sql b/dbt/include/bigquery/macros/utils/safe_cast.sql
new file mode 100644
index 000000000..ac62bb050
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/safe_cast.sql
@@ -0,0 +1,3 @@
+{% macro bigquery__safe_cast(field, type) %}
+    safe_cast({{field}} as {{type}})
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/split_part.sql b/dbt/include/bigquery/macros/utils/split_part.sql
new file mode 100644
index 000000000..2add716f8
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/split_part.sql
@@ -0,0 +1,20 @@
+{% macro bigquery__split_part(string_text, delimiter_text, part_number) %}
+
+  {% if part_number >= 0 %}
+    split(
+        {{ string_text }},
+        {{ delimiter_text }}
+        )[safe_offset({{ part_number - 1 }})]
+  {% else %}
+    split(
+        {{ string_text }},
+        {{ delimiter_text }}
+        )[safe_offset(
+          length({{ string_text }})
+          - length(
+              replace({{ string_text }},  {{ delimiter_text }}, '')
+          ) + 1
+        )]
+  {% endif %}
+
+{% endmacro %}
diff --git a/tests/functional/adapter/test_utils.py b/tests/functional/adapter/test_utils.py
new file mode 100644
index 000000000..2a37ac6b7
--- /dev/null
+++ b/tests/functional/adapter/test_utils.py
@@ -0,0 +1,103 @@
+import pytest
+from dbt.tests.adapter.utils.base_utils import BaseUtils
+from dbt.tests.adapter.utils.test_any_value import BaseAnyValue
+from dbt.tests.adapter.utils.test_bool_or import BaseBoolOr
+from dbt.tests.adapter.utils.test_cast_bool_to_text import BaseCastBoolToText
+from dbt.tests.adapter.utils.test_concat import BaseConcat
+from dbt.tests.adapter.utils.test_dateadd import BaseDateAdd
+from dbt.tests.adapter.utils.test_datediff import BaseDateDiff
+from dbt.tests.adapter.utils.test_date_trunc import BaseDateTrunc
+from dbt.tests.adapter.utils.test_escape_single_quotes import BaseEscapeSingleQuotesQuote
+from dbt.tests.adapter.utils.test_escape_single_quotes import BaseEscapeSingleQuotesBackslash
+from dbt.tests.adapter.utils.test_except import BaseExcept
+from dbt.tests.adapter.utils.test_hash import BaseHash
+from dbt.tests.adapter.utils.test_intersect import BaseIntersect
+from dbt.tests.adapter.utils.test_last_day import BaseLastDay
+from dbt.tests.adapter.utils.test_length import BaseLength
+from dbt.tests.adapter.utils.test_listagg import BaseListagg
+from dbt.tests.adapter.utils.test_position import BasePosition
+from dbt.tests.adapter.utils.test_replace import BaseReplace
+from dbt.tests.adapter.utils.test_right import BaseRight
+from dbt.tests.adapter.utils.test_safe_cast import BaseSafeCast
+from dbt.tests.adapter.utils.test_split_part import BaseSplitPart
+from dbt.tests.adapter.utils.test_string_literal import BaseStringLiteral
+
+
+class TestAnyValue(BaseAnyValue):
+    pass
+
+
+class TestBoolOr(BaseBoolOr):
+    pass
+
+
+class TestCastBoolToText(BaseCastBoolToText):
+    pass
+
+
+class TestConcat(BaseConcat):
+    pass
+
+
+class TestDateAdd(BaseDateAdd):
+    pass
+
+
+class TestDateDiff(BaseDateDiff):
+    pass
+
+
+class TestDateTrunc(BaseDateTrunc):
+    pass
+
+
+class TestEscapeSingleQuotes(BaseEscapeSingleQuotesBackslash):
+    pass
+
+
+class TestExcept(BaseExcept):
+    pass
+
+
+class TestHash(BaseHash):
+    pass
+
+
+class TestIntersect(BaseIntersect):
+    pass
+
+
+class TestLastDay(BaseLastDay):
+    pass
+
+
+class TestLength(BaseLength):
+    pass
+
+
+class TestListagg(BaseListagg):
+    pass
+
+
+class TestPosition(BasePosition):
+    pass
+
+
+class TestReplace(BaseReplace):
+    pass
+
+
+class TestRight(BaseRight):
+    pass
+
+
+class TestSafeCast(BaseSafeCast):
+    pass
+
+
+class TestSplitPart(BaseSplitPart):
+    pass
+
+
+class TestStringLiteral(BaseStringLiteral):
+    pass

From 5e5437610482e83d4790085ef4deb35b9726ede9 Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Fri, 24 Jun 2022 12:59:53 -0400
Subject: [PATCH 450/860] Bumping version to 1.2.0b1 (#204)

* Bumping version to 1.2.0b1

* Update CHANGELOG.md

* Remove whitespaces

* Fixing more whitespace

Co-authored-by: Github Build Bot 
Co-authored-by: leahwicz <60146280+leahwicz@users.noreply.github.com>
---
 .bumpversion.cfg                     |  2 +-
 CHANGELOG.md                         | 16 +++++++++++++++-
 dbt/adapters/bigquery/__version__.py |  2 +-
 setup.py                             |  2 +-
 4 files changed, 18 insertions(+), 4 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 78087456a..62e92542c 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.2.0a1
+current_version = 1.2.0b1
 parse = (?P\d+)
 	\.(?P\d+)
 	\.(?P\d+)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index be153ce60..28b1caf2f 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,19 @@
-## dbt-bigquery 1.2.0 (Release TBD)
+## dbt-bigquery 1.2.0b1 (June 24, 2022)
+
+### Fixes
+- Incorrect parameter is passed to the bq_insert_overwrite macro call ([#172](https://github.com/dbt-labs/dbt-bigquery/pull/172))
+- Fix validate connection failing ([#168](https://github.com/dbt-labs/dbt-bigquery/pull/168))
+
+### Under the hood
 - Adding Python 3.10 testing and enabling mypy ([#177](https://github.com/dbt-labs/dbt-bigquery/pull/177))
+- Lift and shift cross-database macros from dbt-utils ([#192](https://github.com/dbt-labs/dbt-bigquery/pull/192)
+- Implement TestDocsGenerateBigQuery test ([#190](https://github.com/dbt-labs/dbt-bigquery/pull/190))
+- Init push for pytest conversion of override_database for bigquery ([#165](https://github.com/dbt-labs/dbt-bigquery/pull/165), [#188](https://github.com/dbt-labs/dbt-bigquery/pull/188))
+- Add oauth test target as default ([#166](https://github.com/dbt-labs/dbt-bigquery/pull/166))
+
+### Contributors
+- [@robomill](https://github.com/robomill) ([#172](https://github.com/dbt-labs/dbt-bigquery/pull/172))
+- [@dbeatty10](https://github.com/dbeatty10) ([#192](https://github.com/dbt-labs/dbt-bigquery/pull/192)
 
 ## dbt-bigquery 1.1.0 (April 28, 2022)
 
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index a6b977228..6e8eee929 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.2.0a1"
+version = "1.2.0b1"
diff --git a/setup.py b/setup.py
index 0adf95f3b..50d0d16eb 100644
--- a/setup.py
+++ b/setup.py
@@ -50,7 +50,7 @@ def _get_dbt_core_version():
 
 
 package_name = "dbt-bigquery"
-package_version = "1.2.0a1"
+package_version = "1.2.0b1"
 dbt_core_version = _get_dbt_core_version()
 description = """The BigQuery adapter plugin for dbt"""
 

From defb70e5311ae79620b9caf7567e45b26ffd5f56 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Tue, 5 Jul 2022 16:05:16 +0200
Subject: [PATCH 451/860] Data type macros (#214)

* Run tests for data type macros. Adjust numeric_type

* Repoint to dbt-core main

* Add changelog entry
---
 CHANGELOG.md                                  |  5 +++
 dbt/adapters/bigquery/column.py               |  8 ++++-
 .../adapter/utils/test_data_types.py          | 34 +++++++++++++++++++
 .../adapter/{ => utils}/test_utils.py         |  0
 4 files changed, 46 insertions(+), 1 deletion(-)
 create mode 100644 tests/functional/adapter/utils/test_data_types.py
 rename tests/functional/adapter/{ => utils}/test_utils.py (100%)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 28b1caf2f..2dccb571b 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,3 +1,8 @@
+## dbt-bigquery 1.2.0rc1 (Release TBD)
+
+### Under the hood
+- Modify `BigQueryColumn.numeric_type` to always exclude precision + scale, since the functionality of ["parametrized data types on BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types#parameterized_data_types) is highly constrained ([#214](https://github.com/dbt-labs/dbt-bigquery/pull/214))
+
 ## dbt-bigquery 1.2.0b1 (June 24, 2022)
 
 ### Fixes
diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py
index 2422fcc0b..1a6d042b6 100644
--- a/dbt/adapters/bigquery/column.py
+++ b/dbt/adapters/bigquery/column.py
@@ -1,5 +1,5 @@
 from dataclasses import dataclass
-from typing import Optional, List, TypeVar, Iterable, Type
+from typing import Optional, List, TypeVar, Iterable, Type, Any
 
 from dbt.adapters.base.column import Column
 
@@ -92,6 +92,12 @@ def data_type(self) -> str:
         else:
             return field_type
 
+    @classmethod
+    def numeric_type(cls, dtype: str, precision: Any, scale: Any) -> str:
+        # BigQuery makes life much harder if precision + scale are specified
+        # even if they're fed in here, just return the data type by itself
+        return dtype
+
     def is_string(self) -> bool:
         return self.dtype.lower() == "string"
 
diff --git a/tests/functional/adapter/utils/test_data_types.py b/tests/functional/adapter/utils/test_data_types.py
new file mode 100644
index 000000000..af084a0ef
--- /dev/null
+++ b/tests/functional/adapter/utils/test_data_types.py
@@ -0,0 +1,34 @@
+import pytest
+from dbt.tests.adapter.utils.data_types.test_type_bigint import BaseTypeBigInt
+from dbt.tests.adapter.utils.data_types.test_type_float import BaseTypeFloat
+from dbt.tests.adapter.utils.data_types.test_type_int import BaseTypeInt
+from dbt.tests.adapter.utils.data_types.test_type_numeric import BaseTypeNumeric
+from dbt.tests.adapter.utils.data_types.test_type_string import BaseTypeString
+from dbt.tests.adapter.utils.data_types.test_type_timestamp import BaseTypeTimestamp
+
+
+class TestTypeBigInt(BaseTypeBigInt):
+    pass
+
+    
+class TestTypeFloat(BaseTypeFloat):
+    pass
+
+    
+class TestTypeInt(BaseTypeInt):
+    pass
+
+    
+class TestTypeNumeric(BaseTypeNumeric):
+    def numeric_fixture_type(self):
+        return "numeric"
+
+    
+class TestTypeString(BaseTypeString):
+    pass
+
+    
+class TestTypeTimestamp(BaseTypeTimestamp):
+    pass
+
+    
\ No newline at end of file
diff --git a/tests/functional/adapter/test_utils.py b/tests/functional/adapter/utils/test_utils.py
similarity index 100%
rename from tests/functional/adapter/test_utils.py
rename to tests/functional/adapter/utils/test_utils.py

From 6828a619b4ce2a88bc01770ec5e227e916c3e1d3 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 7 Jul 2022 09:58:56 -0400
Subject: [PATCH 452/860] Bump black from 22.3.0 to 22.6.0 (#208)

Bumps [black](https://github.com/psf/black) from 22.3.0 to 22.6.0.
- [Release notes](https://github.com/psf/black/releases)
- [Changelog](https://github.com/psf/black/blob/main/CHANGES.md)
- [Commits](https://github.com/psf/black/compare/22.3.0...22.6.0)

---
updated-dependencies:
- dependency-name: black
  dependency-type: direct:development
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] 

Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: leahwicz <60146280+leahwicz@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index 09cf7f7c2..2775ca784 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -3,7 +3,7 @@
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
 
-black==22.3.0
+black==22.6.0
 bumpversion
 flake8
 flaky

From 0bb49fcf75d688802607de19c78ed303cde8df32 Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Fri, 8 Jul 2022 09:48:27 -0600
Subject: [PATCH 453/860] Env vars for grantees for testing (#216)

* Env vars for grantees for testing

* Updated user / group names
---
 .github/workflows/integration.yml | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 439ed1820..9b5d9f88c 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -168,6 +168,9 @@ jobs:
           BIGQUERY_TEST_SERVICE_ACCOUNT_JSON: ${{ secrets.BIGQUERY_TEST_SERVICE_ACCOUNT_JSON }}
           BIGQUERY_TEST_ALT_DATABASE: ${{ secrets.BIGQUERY_TEST_ALT_DATABASE }}
           BIGQUERY_TEST_NO_ACCESS_DATABASE: ${{ secrets.BIGQUERY_TEST_NO_ACCESS_DATABASE }}
+          DBT_TEST_USER_1: user:buildbot@dbtlabs.com
+          DBT_TEST_USER_2: user:buildbot@fishtownanalytics.com
+          DBT_TEST_USER_3: group:dev-core@dbtlabs.com
         run: tox
 
       - uses: actions/upload-artifact@v2

From 8b2feb58a06622672d04720b56966c4a5f91691c Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Mon, 11 Jul 2022 10:38:55 -0600
Subject: [PATCH 454/860] Updated user / group names (#217)

* Updated user / group names

* Updated third value
---
 .github/workflows/integration.yml | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 9b5d9f88c..4f920d100 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -168,9 +168,9 @@ jobs:
           BIGQUERY_TEST_SERVICE_ACCOUNT_JSON: ${{ secrets.BIGQUERY_TEST_SERVICE_ACCOUNT_JSON }}
           BIGQUERY_TEST_ALT_DATABASE: ${{ secrets.BIGQUERY_TEST_ALT_DATABASE }}
           BIGQUERY_TEST_NO_ACCESS_DATABASE: ${{ secrets.BIGQUERY_TEST_NO_ACCESS_DATABASE }}
-          DBT_TEST_USER_1: user:buildbot@dbtlabs.com
-          DBT_TEST_USER_2: user:buildbot@fishtownanalytics.com
-          DBT_TEST_USER_3: group:dev-core@dbtlabs.com
+          DBT_TEST_USER_1: group:buildbot@dbtlabs.com
+          DBT_TEST_USER_2: group:dev-core@dbtlabs.com
+          DBT_TEST_USER_3: serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserviceaccount.com
         run: tox
 
       - uses: actions/upload-artifact@v2

From 942d460fc60beb87325871903b26afee7e5f4d85 Mon Sep 17 00:00:00 2001
From: Emily Rockman 
Date: Mon, 11 Jul 2022 14:49:39 -0500
Subject: [PATCH 455/860] Add Grants to BigQuery Materializations (#212)

* Add get_show_grant_sql

* first pass at granting with sql

* more macro overrides

* update macros, temp update to dev-req for CI

* tweak to sql and added some TODOs

* move things around

* exclude session_user

* fix mypy error, fix exclude to be more than users

* simplify revoke logic

* small cleanup, point to ct-660 branch

* grant entire list in one statement

* wip

* wip, broken tests

* Update dbt/include/bigquery/macros/materializations/incremental.sql

Co-authored-by: Anders 

* working on getting tests working, very WIP

* All tests passing locally

* Updated user / group names

* Ongoing test cleanup

* Account for refactor in dbt-labs/dbt-core@c763601

* Updated third value

* Alt approach for grant location (#218)

* Alt approach to grabbing + factoring info_schema region

* add exception for blank location

* Update dbt/adapters/bigquery/relation.py

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>

* point to existing branch

* fix pre commit errors

Co-authored-by: Emily Rockman 
Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>

* reset to point dbt-core to main instead of branch

* fix examples in test.env.example

* add changelog

Co-authored-by: Anders 
Co-authored-by: Jeremy Cohen 
Co-authored-by: Doug Beatty 
Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
---
 CHANGELOG.md                                  |  5 ++-
 dbt/adapters/bigquery/impl.py                 |  8 ++++
 dbt/adapters/bigquery/relation.py             | 43 ++++++++++++++++++
 .../bigquery/macros/adapters/apply_grants.sql | 20 +++++++++
 .../bigquery/macros/materializations/copy.sql |  3 +-
 .../macros/materializations/incremental.sql   |  6 +++
 .../macros/materializations/table.sql         |  6 +++
 .../bigquery/macros/materializations/view.sql |  4 ++
 test.env.example                              |  4 ++
 tests/conftest.py                             |  1 -
 tests/functional/adapter/test_grants.py       | 44 +++++++++++++++++++
 11 files changed, 141 insertions(+), 3 deletions(-)
 create mode 100644 dbt/include/bigquery/macros/adapters/apply_grants.sql
 create mode 100644 tests/functional/adapter/test_grants.py

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 2dccb571b..e5cab993e 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,7 @@
-## dbt-bigquery 1.2.0rc1 (Release TBD)
+## dbt-bigquery 1.2.0rc1 (June 11, 2022)
+
+### Features
+- Add grants to materializations ([#198](https://github.com/dbt-labs/dbt-bigquery/issues/198), [#212](https://github.com/dbt-labs/dbt-bigquery/pull/212))
 
 ### Under the hood
 - Modify `BigQueryColumn.numeric_type` to always exclude precision + scale, since the functionality of ["parametrized data types on BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types#parameterized_data_types) is highly constrained ([#214](https://github.com/dbt-labs/dbt-bigquery/pull/214))
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 344ffb22e..7c66b014d 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -757,6 +757,14 @@ def grant_access_to(self, entity, entity_type, role, grant_target_dict):
         dataset.access_entries = access_entries
         client.update_dataset(dataset, ["access_entries"])
 
+    @available.parse_none
+    def get_dataset_location(self, relation):
+        conn = self.connections.get_thread_connection()
+        client = conn.handle
+        dataset_ref = self.connections.dataset_ref(relation.project, relation.dataset)
+        dataset = client.get_dataset(dataset_ref)
+        return dataset.location
+
     def get_rows_different_sql(  # type: ignore[override]
         self,
         relation_a: BigQueryRelation,
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index 8156e360d..7224de8cf 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -1,7 +1,10 @@
 from dataclasses import dataclass
 from typing import Optional
 
+from itertools import chain, islice
+
 from dbt.adapters.base.relation import BaseRelation, ComponentName, InformationSchema
+from dbt.exceptions import raise_compiler_error
 from dbt.utils import filter_null_values
 from typing import TypeVar
 
@@ -12,6 +15,7 @@
 @dataclass(frozen=True, eq=False, repr=False)
 class BigQueryRelation(BaseRelation):
     quote_character: str = "`"
+    location: Optional[str] = None
 
     def matches(
         self,
@@ -52,6 +56,7 @@ def information_schema(self, identifier: Optional[str] = None) -> "BigQueryInfor
 @dataclass(frozen=True, eq=False, repr=False)
 class BigQueryInformationSchema(InformationSchema):
     quote_character: str = "`"
+    location: Optional[str] = None
 
     @classmethod
     def get_include_policy(cls, relation, information_schema_view):
@@ -63,11 +68,49 @@ def get_include_policy(cls, relation, information_schema_view):
         if information_schema_view == "__TABLES__":
             identifier = False
 
+        # In the future, let's refactor so that location/region can also be a
+        # ComponentName, so that we can have logic like:
+        #
+        # region = False
+        # if information_schema_view == "OBJECT_PRIVILEGES":
+        #     region = True
+
         return relation.include_policy.replace(
             schema=schema,
             identifier=identifier,
         )
 
+    def get_region_identifier(self) -> str:
+        region_id = f"region-{self.location}"
+        return self.quoted(region_id)
+
+    @classmethod
+    def from_relation(cls, relation, information_schema_view):
+        info_schema = super().from_relation(relation, information_schema_view)
+        if information_schema_view == "OBJECT_PRIVILEGES":
+            # OBJECT_PRIVILEGES require a location.  If the location is blank there is nothing
+            # the user can do about it.
+            if not relation.location:
+                msg = (
+                    f'No location/region found when trying to retrieve "{information_schema_view}"'
+                )
+                raise raise_compiler_error(msg)
+            info_schema = info_schema.incorporate(location=relation.location)
+        return info_schema
+
+    # override this method to interpolate the region identifier,
+    # if a location is required for this information schema view
+    def _render_iterator(self):
+        iterator = super()._render_iterator()
+        if self.location:
+            return chain(
+                islice(iterator, 1),  # project,
+                [(None, self.get_region_identifier())],  # region id,
+                islice(iterator, 1, None),  # remaining components
+            )
+        else:
+            return iterator
+
     def replace(self, **kwargs):
         if "information_schema_view" in kwargs:
             view = kwargs["information_schema_view"]
diff --git a/dbt/include/bigquery/macros/adapters/apply_grants.sql b/dbt/include/bigquery/macros/adapters/apply_grants.sql
new file mode 100644
index 000000000..e344862ae
--- /dev/null
+++ b/dbt/include/bigquery/macros/adapters/apply_grants.sql
@@ -0,0 +1,20 @@
+{% macro bigquery__get_show_grant_sql(relation) %}
+    {% set location = adapter.get_dataset_location(relation) %}
+    {% set relation = relation.incorporate(location=location) %}
+
+    select privilege_type, grantee
+    from {{ relation.information_schema("OBJECT_PRIVILEGES") }}
+    where object_schema = "{{ relation.dataset }}"
+      and object_name = "{{ relation.identifier }}"
+      -- filter out current user
+      and split(grantee, ':')[offset(1)] != session_user()
+{% endmacro %}
+
+
+{%- macro bigquery__get_grant_sql(relation, privilege, grantee) -%}
+    grant `{{ privilege }}` on {{ relation.type }} {{ relation }} to {{ '\"' + grantee|join('\", \"') + '\"' }}
+{%- endmacro -%}
+
+{%- macro bigquery__get_revoke_sql(relation, privilege, grantee) -%}
+    revoke `{{ privilege }}` on {{ relation.type }} {{ relation }} from {{ '\"' + grantee|join('\", \"') + '\"' }}
+{%- endmacro -%}
diff --git a/dbt/include/bigquery/macros/materializations/copy.sql b/dbt/include/bigquery/macros/materializations/copy.sql
index 6a86fbe44..8285dc845 100644
--- a/dbt/include/bigquery/macros/materializations/copy.sql
+++ b/dbt/include/bigquery/macros/materializations/copy.sql
@@ -16,7 +16,7 @@
     {{ source_array.append(source(*src_table)) }}
   {% endfor %}
 
-  {# Call adapter's copy_table function #}
+  {# Call adapter copy_table function #}
   {%- set result_str = adapter.copy_table(
       source_array,
       destination,
@@ -26,6 +26,7 @@
 
   {# Clean up #}
   {{ run_hooks(post_hooks) }}
+  {%- do apply_grants(target_relation, grant_config) -%}
   {{ adapter.commit() }}
 
   {{ return({'relations': [destination]}) }}
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index b6d387890..8cf2ab65c 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -152,6 +152,9 @@
 
   {% set on_schema_change = incremental_validate_on_schema_change(config.get('on_schema_change'), default='ignore') %}
 
+   -- grab current tables grants config for comparision later on
+  {% set grant_config = config.get('grants') %}
+
   {{ run_hooks(pre_hooks) }}
 
   {% if existing_relation is none %}
@@ -197,6 +200,9 @@
 
   {% set target_relation = this.incorporate(type='table') %}
 
+  {% set should_revoke = should_revoke(existing_relation, full_refresh_mode) %}
+  {% do apply_grants(target_relation, grant_config, should_revoke) %}
+
   {% do persist_docs(target_relation, model) %}
 
   {{ return({'relations': [target_relation]}) }}
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index a7452265b..9e63637c1 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -5,6 +5,9 @@
   {%- set exists_not_as_table = (old_relation is not none and not old_relation.is_table) -%}
   {%- set target_relation = api.Relation.create(database=database, schema=schema, identifier=identifier, type='table') -%}
 
+  -- grab current tables grants config for comparision later on
+  {%- set grant_config = config.get('grants') -%}
+
   {{ run_hooks(pre_hooks) }}
 
   {#
@@ -30,6 +33,9 @@
 
   {{ run_hooks(post_hooks) }}
 
+  {% set should_revoke = should_revoke(old_relation, full_refresh_mode=True) %}
+  {% do apply_grants(target_relation, grant_config, should_revoke) %}
+
   {% do persist_docs(target_relation, model) %}
 
   {{ return({'relations': [target_relation]}) }}
diff --git a/dbt/include/bigquery/macros/materializations/view.sql b/dbt/include/bigquery/macros/materializations/view.sql
index 97e3d2761..e68a51421 100644
--- a/dbt/include/bigquery/macros/materializations/view.sql
+++ b/dbt/include/bigquery/macros/materializations/view.sql
@@ -9,9 +9,13 @@
 
 
 {% materialization view, adapter='bigquery' -%}
+    -- grab current tables grants config for comparision later on
+    {% set grant_config = config.get('grants') %}
+
     {% set to_return = create_or_replace_view() %}
 
     {% set target_relation = this.incorporate(type='view') %}
+
     {% do persist_docs(target_relation, model) %}
 
     {% if config.get('grant_access_to') %}
diff --git a/test.env.example b/test.env.example
index 2065e4393..f2e59e6d0 100644
--- a/test.env.example
+++ b/test.env.example
@@ -1,3 +1,7 @@
 BIGQUERY_TEST_ALT_DATABASE=
 BIGQUERY_TEST_NO_ACCESS_DATABASE=
 BIGQUERY_TEST_SERVICE_ACCOUNT_JSON='{}'
+
+DBT_TEST_USER_1="group:buildbot@dbtlabs.com"
+DBT_TEST_USER_2="group:dev-core@dbtlabs.com"
+DBT_TEST_USER_3="serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserviceaccount.com"
diff --git a/tests/conftest.py b/tests/conftest.py
index 4bbdb00e0..69a29b39c 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -29,7 +29,6 @@ def oauth_target():
         'type': 'bigquery',
         'method': 'oauth',
         'threads': 1,
-        # project isn't needed if you configure a default, via 'gcloud config set project'
     }
 
 
diff --git a/tests/functional/adapter/test_grants.py b/tests/functional/adapter/test_grants.py
new file mode 100644
index 000000000..b35e4787e
--- /dev/null
+++ b/tests/functional/adapter/test_grants.py
@@ -0,0 +1,44 @@
+import pytest
+
+from dbt.tests.adapter.grants.base_grants import BaseGrants
+from dbt.tests.adapter.grants.test_model_grants import BaseModelGrants
+from dbt.tests.adapter.grants.test_incremental_grants import BaseIncrementalGrants
+from dbt.tests.adapter.grants.test_invalid_grants import BaseInvalidGrants
+from dbt.tests.adapter.grants.test_seed_grants import BaseSeedGrants
+from dbt.tests.adapter.grants.test_snapshot_grants import BaseSnapshotGrants
+
+
+class BaseGrantsBigQuery(BaseGrants):
+    def privilege_grantee_name_overrides(self):
+        return {
+            "select": "roles/bigquery.dataViewer",
+            "insert": "roles/bigquery.dataEditor",
+            "fake_privilege": "roles/invalid",
+            "invalid_user": "user:fake@dbtlabs.com",
+        }
+
+class TestModelGrantsBigQuery(BaseGrantsBigQuery, BaseModelGrants):
+    pass
+
+
+class TestIncrementalGrantsBigQuery(BaseGrantsBigQuery, BaseIncrementalGrants):
+    pass
+
+
+class TestSeedGrantsBigQuery(BaseGrantsBigQuery, BaseSeedGrants):
+    # seeds in dbt-bigquery are always "full refreshed," in such a way that
+    # the grants do not carry over
+    def seeds_support_partial_refresh(self):
+        return False
+
+
+class TestSnapshotGrantsBigQuery(BaseGrantsBigQuery, BaseSnapshotGrants):
+    pass
+
+
+class TestInvalidGrantsBigQuery(BaseGrantsBigQuery, BaseInvalidGrants):
+    def grantee_does_not_exist_error(self):
+        return "User fake@dbtlabs.com does not exist."
+
+    def privilege_does_not_exist_error(self):
+        return "Role roles/invalid is not supported for this resource."

From 0212fd621ede4c24929a008de718a7e45bc32cec Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Tue, 12 Jul 2022 14:40:41 -0400
Subject: [PATCH 456/860] Bumping version to 1.3.0a1 (#221)

* Bumping version to 1.3.0a1

* Update CHANGELOG.md

* Fix whitespace

Co-authored-by: Github Build Bot 
Co-authored-by: leahwicz <60146280+leahwicz@users.noreply.github.com>
---
 .bumpversion.cfg                     | 2 +-
 CHANGELOG.md                         | 5 ++++-
 dbt/adapters/bigquery/__version__.py | 2 +-
 setup.py                             | 2 +-
 4 files changed, 7 insertions(+), 4 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 62e92542c..1fd657599 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.2.0b1
+current_version = 1.3.0a1
 parse = (?P\d+)
 	\.(?P\d+)
 	\.(?P\d+)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index e5cab993e..8253c552e 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,7 @@
-## dbt-bigquery 1.2.0rc1 (June 11, 2022)
+## dbt-bigquery 1.3.0b1 (Release TBD)
+
+
+## dbt-bigquery 1.2.0rc1 (June 12, 2022)
 
 ### Features
 - Add grants to materializations ([#198](https://github.com/dbt-labs/dbt-bigquery/issues/198), [#212](https://github.com/dbt-labs/dbt-bigquery/pull/212))
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 6e8eee929..a9fe3c3ee 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.2.0b1"
+version = "1.3.0a1"
diff --git a/setup.py b/setup.py
index 50d0d16eb..38ca5a599 100644
--- a/setup.py
+++ b/setup.py
@@ -50,7 +50,7 @@ def _get_dbt_core_version():
 
 
 package_name = "dbt-bigquery"
-package_version = "1.2.0b1"
+package_version = "1.3.0a1"
 dbt_core_version = _get_dbt_core_version()
 description = """The BigQuery adapter plugin for dbt"""
 

From 803b954394efe4e09a7b4b830d9964be47e1525b Mon Sep 17 00:00:00 2001
From: Jeremy Cohen 
Date: Thu, 14 Jul 2022 12:08:35 +0200
Subject: [PATCH 457/860] Try using SQL for `create_schema` (#183)

* Try using SQL create_schema + drop_schema

* Disable unit tests. Reorder tox

* Revert drop_schema to old way

* Reenable unit test for drop_schema

* Code cleanup

* Add changelog entry

* Add import back
---
 CHANGELOG.md                             |  2 ++
 dbt/adapters/bigquery/impl.py            | 29 ++++++++++++++++++++----
 dbt/include/bigquery/macros/adapters.sql |  4 ----
 tests/unit/test_bigquery_adapter.py      |  5 ----
 4 files changed, 26 insertions(+), 14 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 8253c552e..c78783f17 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,7 @@
 ## dbt-bigquery 1.3.0b1 (Release TBD)
 
+### Features
+- Implement `create_schema` via SQL, instead of Python method, allowing users to override if desired. `drop_schema` remains a Python method for the time being. ([#182](https://github.com/dbt-labs/dbt-bigquery/issues/182), [#183](https://github.com/dbt-labs/dbt-bigquery/pull/183))
 
 ## dbt-bigquery 1.2.0rc1 (June 12, 2022)
 
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 7c66b014d..9a198658c 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -8,11 +8,15 @@
 
 from dbt import ui  # type: ignore
 from dbt.adapters.base import BaseAdapter, available, RelationType, SchemaSearchMap, AdapterConfig
+from dbt.adapters.cache import _make_key
+
 from dbt.adapters.bigquery.relation import BigQueryRelation
 from dbt.adapters.bigquery import BigQueryColumn
 from dbt.adapters.bigquery import BigQueryConnectionManager
 from dbt.contracts.graph.manifest import Manifest
 from dbt.events import AdapterLogger
+from dbt.events.functions import fire_event
+from dbt.events.types import SchemaCreation, SchemaDrop
 from dbt.utils import filter_null_values
 
 import google.auth
@@ -33,6 +37,8 @@
 WRITE_APPEND = google.cloud.bigquery.job.WriteDisposition.WRITE_APPEND
 WRITE_TRUNCATE = google.cloud.bigquery.job.WriteDisposition.WRITE_TRUNCATE
 
+CREATE_SCHEMA_MACRO_NAME = "create_schema"
+
 
 def sql_escape(string):
     if not isinstance(string, str):
@@ -273,16 +279,29 @@ def get_relation(self, database: str, schema: str, identifier: str) -> BigQueryR
             table = None
         return self._bq_table_to_relation(table)
 
+    # BigQuery added SQL support for 'create schema' + 'drop schema' in March 2021
+    # Unfortunately, 'drop schema' runs into permissions issues during tests
+    # Most of the value here comes from user overrides of 'create_schema'
+
+    # TODO: the code below is copy-pasted from SQLAdapter.create_schema. Is there a better way?
     def create_schema(self, relation: BigQueryRelation) -> None:
-        database = relation.database
-        schema = relation.schema
-        logger.debug('Creating schema "{}.{}".', database, schema)
-        self.connections.create_dataset(database, schema)
+        # use SQL 'create schema'
+        relation = relation.without_identifier()
+        fire_event(SchemaCreation(relation=_make_key(relation)))
+        kwargs = {
+            "relation": relation,
+        }
+        self.execute_macro(CREATE_SCHEMA_MACRO_NAME, kwargs=kwargs)
+        self.commit_if_has_connection()
+        # we can't update the cache here, as if the schema already existed we
+        # don't want to (incorrectly) say that it's empty
 
     def drop_schema(self, relation: BigQueryRelation) -> None:
+        # still use a client method, rather than SQL 'drop schema ... cascade'
         database = relation.database
         schema = relation.schema
-        logger.debug('Dropping schema "{}.{}".', database, schema)
+        logger.debug('Dropping schema "{}.{}".', database, schema)  # in lieu of SQL
+        fire_event(SchemaDrop(relation=_make_key(relation)))
         self.connections.drop_dataset(database, schema)
         self.cache.drop_schema(database, schema)
 
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index f5a732d4d..f75947001 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -76,10 +76,6 @@
 
 {% endmacro %}
 
-{% macro bigquery__create_schema(relation) -%}
-  {{ adapter.create_schema(relation) }}
-{% endmacro %}
-
 {% macro bigquery__drop_schema(relation) -%}
   {{ adapter.drop_schema(relation) }}
 {% endmacro %}
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index eee92f1ab..188c19b7c 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -349,11 +349,6 @@ def test_get_relation(self):
         self.adapter.get_relation('db', 'schema', 'my_model')
         self.mock_connection_manager.get_bq_table.assert_called_once_with('db', 'schema', 'my_model')
 
-    def test_create_schema(self):
-        relation = BigQueryRelation.create(database='db', schema='schema')
-        self.adapter.create_schema(relation)
-        self.mock_connection_manager.create_dataset.assert_called_once_with('db', 'schema')
-
     @patch.object(BigQueryAdapter, 'check_schema_exists')
     def test_drop_schema(self, mock_check_schema):
         mock_check_schema.return_value = True

From f634a57282c47861db6b68b0acba624d80dce405 Mon Sep 17 00:00:00 2001
From: Gerda Shank 
Date: Mon, 25 Jul 2022 11:17:36 -0400
Subject: [PATCH 458/860] Changes for incremental strategy refactor
 compatibility (#223)

---
 CHANGELOG.md                                       |  3 +++
 .../macros/materializations/incremental.sql        | 14 ++++++++------
 2 files changed, 11 insertions(+), 6 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index c78783f17..7adea32ee 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -3,6 +3,9 @@
 ### Features
 - Implement `create_schema` via SQL, instead of Python method, allowing users to override if desired. `drop_schema` remains a Python method for the time being. ([#182](https://github.com/dbt-labs/dbt-bigquery/issues/182), [#183](https://github.com/dbt-labs/dbt-bigquery/pull/183))
 
+### Under the hood
+- Implement minimal changes to support dbt Core incremental materialization refactor. ([#232](http://github.com/dbt-labs/dbt-bigquery/issues/232), [#223](https://github.com/dbt-labs/dbt-biquery/pull/223))
+
 ## dbt-bigquery 1.2.0rc1 (June 12, 2022)
 
 ### Features
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index 8cf2ab65c..1ff7e1c80 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -14,7 +14,7 @@
 
 {% macro dbt_bigquery_validate_get_incremental_strategy(config) %}
   {#-- Find and validate the incremental strategy #}
-  {%- set strategy = config.get("incremental_strategy", default="merge") -%}
+  {%- set strategy = config.get("incremental_strategy") or 'merge' -%}
 
   {% set invalid_strategy_msg -%}
     Invalid incremental strategy provided: {{ strategy }}
@@ -46,6 +46,12 @@
         )
       {%- endset -%}
 
+      {#-- Because we're putting the model SQL _directly_ into the MERGE statement,
+         we need to prepend the MERGE statement with the user-configured sql_header,
+         which may be needed to resolve that model SQL (e.g. referencing a variable or UDF in the header)
+         in the "dynamic" case, we save the model SQL result as a temp table first, wherein the
+         sql_header is included by the create_table_as macro.
+      #}
       {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate], include_sql_header=true) }}
 
   {% else %} {# dynamic #}
@@ -80,12 +86,8 @@
           from {{ tmp_relation }}
       );
 
-      {#
-        TODO: include_sql_header is a hack; consider a better approach that includes
-              the sql_header at the materialization-level instead
-      #}
       -- 3. run the merge statement
-      {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate], include_sql_header=false) }};
+      {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate]) }};
 
       -- 4. clean up the temp table
       drop table if exists {{ tmp_relation }}

From 2c2a0e65664c90f9599ce82be17c388cc6f0a62f Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Tue, 26 Jul 2022 16:01:31 -0400
Subject: [PATCH 459/860] Adding scheduled runs to release branches (#222)

* Adding scheduled runs to release branches

* Fixing newline issue

* Removing PR trigger used for testing
---
 .github/workflows/integration.yml          | 18 ------
 .github/workflows/release-branch-tests.yml | 64 ++++++++++++++++++++++
 2 files changed, 64 insertions(+), 18 deletions(-)
 create mode 100644 .github/workflows/release-branch-tests.yml

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 4f920d100..f4be418bb 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -33,9 +33,6 @@ on:
   pull_request_target:
   # manual trigger
   workflow_dispatch:
-  # run this once per night to ensure no regressions from latest dbt-core changes
-  schedule:
-    - cron: '0 5 * * *' # 5 UTC
 
 # explicitly turn off permissions for `GITHUB_TOKEN`
 permissions: read-all
@@ -212,18 +209,3 @@ jobs:
             "You do not have permissions to run integration tests, @dbt-labs/core "\
             "needs to label this PR with `ok to test` in order to run integration tests!"
           check_for_duplicate_msg: true
-
-  post-failure:
-    runs-on: ubuntu-latest
-    needs: test
-    if: ${{ failure() }}
-
-    steps:
-      - name: Posting scheduled run failures
-        uses: ravsamhq/notify-slack-action@v1
-        if: ${{ github.event_name == 'schedule' }}
-        with:
-          notification_title: 'BigQuery nightly integration test failed'
-          status: ${{ job.status }}
-    env:
-      SLACK_WEBHOOK_URL: ${{ secrets.SLACK_DEV_CORE_ALERTS }}
diff --git a/.github/workflows/release-branch-tests.yml b/.github/workflows/release-branch-tests.yml
new file mode 100644
index 000000000..4e29dd0f9
--- /dev/null
+++ b/.github/workflows/release-branch-tests.yml
@@ -0,0 +1,64 @@
+# **what?**
+# The purpose of this workflow is to trigger CI to run for each
+# release branch and main branch on a regular cadence. If the CI workflow
+# fails for a branch, it will post to dev-core-alerts to raise awareness.
+# The 'aurelien-baudet/workflow-dispatch' Action triggers the existing
+# CI worklow file on the given branch to run so that even if we change the
+# CI workflow file in the future, the one that is tailored for the given
+# release branch will be used.
+
+# **why?**
+# Ensures release branches and main are always shippable and not broken.
+# Also, can catch any dependencies shifting beneath us that might
+# introduce breaking changes (could also impact Cloud).
+
+# **when?**
+# Mainly on a schedule of 9:00, 13:00, 18:00 UTC everyday.
+# Manual trigger can also test on demand
+
+name: Release branch scheduled testing
+
+on:
+  schedule:
+    - cron: '0 9,13,18 * * *' # 9:00, 13:00, 18:00 UTC
+
+  workflow_dispatch: # for manual triggering
+
+# no special access is needed
+permissions: read-all
+
+jobs:
+  kick-off-ci:
+    name: Kick-off CI
+    runs-on: ubuntu-latest
+
+    strategy:
+      # must run CI 1 branch at a time b/c the workflow-dispatch Action polls for
+      # latest run for results and it gets confused when we kick off multiple runs
+      # at once. There is a race condition so we will just run in sequential order.
+      max-parallel: 1
+      fail-fast: false
+      matrix:
+        branch: [1.0.latest, 1.1.latest, 1.2.latest, main]
+        workflow_name: [main.yml, integration.yml]
+
+    steps:
+    - name: Call CI workflow for ${{ matrix.branch }} branch
+      id: trigger-step
+      continue-on-error: true
+      uses: aurelien-baudet/workflow-dispatch@v2.1.1
+      with:
+        workflow: ${{ matrix.workflow_name }}
+        ref: ${{ matrix.branch }}
+        token: ${{ secrets.FISHTOWN_BOT_PAT }}
+
+    - name: Post failure to Slack
+      uses: ravsamhq/notify-slack-action@v1
+      if: ${{ always() && !contains(steps.trigger-step.outputs.workflow-conclusion,'success') }}
+      with:
+        status: ${{ job.status }}
+        notification_title: 'dbt-bigquery scheduled run of ${{ matrix.workflow_name }} on "${{ matrix.branch }}" branch not successful'
+        message_format: ':x: ${{ matrix.workflow_name }} CI on branch "${{ matrix.branch }}" ${{ steps.trigger-step.outputs.workflow-conclusion }}'
+        footer: 'Linked failed CI run ${{ steps.trigger-step.outputs.workflow-url }}'
+      env:
+        SLACK_WEBHOOK_URL: ${{ secrets.SLACK_DEV_CORE_ALERTS }}

From 946ec69f2181dbc71a60fad01536c3c5558abb29 Mon Sep 17 00:00:00 2001
From: Chenyu Li 
Date: Wed, 27 Jul 2022 10:02:25 -0700
Subject: [PATCH 460/860] add env var for python model test (#240)

---
 .github/workflows/integration.yml | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index f4be418bb..12265849a 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -168,6 +168,9 @@ jobs:
           DBT_TEST_USER_1: group:buildbot@dbtlabs.com
           DBT_TEST_USER_2: group:dev-core@dbtlabs.com
           DBT_TEST_USER_3: serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserviceaccount.com
+          DATAPROC_REGION: us-central1
+          DATAPROC_CLUSTER_NAME: dbt-test-1
+          GCS_BUCKET: dbt-ci
         run: tox
 
       - uses: actions/upload-artifact@v2

From 55bbd3d5a0760a66b01424c0a0884e6161b71b0e Mon Sep 17 00:00:00 2001
From: Chenyu Li 
Date: Thu, 28 Jul 2022 14:12:32 -0700
Subject: [PATCH 461/860] Feature/python model v1 (#209)

Co-authored-by: Jeremy Cohen 
Co-authored-by: Emily Rockman 
Co-authored-by: Stu Kilgore 
---
 CHANGELOG.md                                  |   1 +
 dbt/adapters/bigquery/connections.py          |  21 +++-
 dbt/adapters/bigquery/impl.py                 | 118 +++++++++++++++++-
 dbt/include/bigquery/macros/adapters.sql      |  45 ++++---
 .../macros/materializations/incremental.sql   |  57 ++++++---
 .../macros/materializations/table.sql         |  35 +++++-
 requirements.txt                              |   2 +
 test.env.example                              |   6 +-
 tests/conftest.py                             |   4 +
 tests/functional/adapter/test_python_model.py |  49 ++++++++
 .../defer_state_test/test_defer_state.py      |   4 +-
 tox.ini                                       |   3 +-
 12 files changed, 296 insertions(+), 49 deletions(-)
 create mode 100644 requirements.txt
 create mode 100644 tests/functional/adapter/test_python_model.py

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 7adea32ee..af1c1f13f 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -2,6 +2,7 @@
 
 ### Features
 - Implement `create_schema` via SQL, instead of Python method, allowing users to override if desired. `drop_schema` remains a Python method for the time being. ([#182](https://github.com/dbt-labs/dbt-bigquery/issues/182), [#183](https://github.com/dbt-labs/dbt-bigquery/pull/183))
+- Added incrementail materializations for python models via DataProc. ([#226](https://github.com/dbt-labs/dbt-bigquery/pull/226))
 
 ### Under the hood
 - Implement minimal changes to support dbt Core incremental materialization refactor. ([#232](http://github.com/dbt-labs/dbt-bigquery/issues/232), [#223](https://github.com/dbt-labs/dbt-biquery/pull/223))
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 454e84d7d..da7b0f687 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -116,6 +116,10 @@ class BigQueryCredentials(Credentials):
     client_secret: Optional[str] = None
     token_uri: Optional[str] = None
 
+    dataproc_region: Optional[str] = None
+    dataproc_cluster_name: Optional[str] = None
+    gcs_bucket: Optional[str] = None
+
     scopes: Optional[Tuple[str, ...]] = (
         "https://www.googleapis.com/auth/bigquery",
         "https://www.googleapis.com/auth/cloud-platform",
@@ -154,6 +158,7 @@ def _connection_keys(self):
             "job_retries",
             "job_creation_timeout_seconds",
             "job_execution_timeout_seconds",
+            "gcs_bucket",
         )
 
     @classmethod
@@ -265,7 +270,7 @@ def format_rows_number(self, rows_number):
         return f"{rows_number:3.1f}{unit}".strip()
 
     @classmethod
-    def get_bigquery_credentials(cls, profile_credentials):
+    def get_google_credentials(cls, profile_credentials) -> GoogleCredentials:
         method = profile_credentials.method
         creds = GoogleServiceAccountCredentials.Credentials
 
@@ -295,8 +300,8 @@ def get_bigquery_credentials(cls, profile_credentials):
         raise FailedToConnectException(error)
 
     @classmethod
-    def get_impersonated_bigquery_credentials(cls, profile_credentials):
-        source_credentials = cls.get_bigquery_credentials(profile_credentials)
+    def get_impersonated_credentials(cls, profile_credentials):
+        source_credentials = cls.get_google_credentials(profile_credentials)
         return impersonated_credentials.Credentials(
             source_credentials=source_credentials,
             target_principal=profile_credentials.impersonate_service_account,
@@ -305,11 +310,15 @@ def get_impersonated_bigquery_credentials(cls, profile_credentials):
         )
 
     @classmethod
-    def get_bigquery_client(cls, profile_credentials):
+    def get_credentials(cls, profile_credentials):
         if profile_credentials.impersonate_service_account:
-            creds = cls.get_impersonated_bigquery_credentials(profile_credentials)
+            return cls.get_impersonated_credentials(profile_credentials)
         else:
-            creds = cls.get_bigquery_credentials(profile_credentials)
+            return cls.get_google_credentials(profile_credentials)
+
+    @classmethod
+    def get_bigquery_client(cls, profile_credentials):
+        creds = cls.get_credentials(profile_credentials)
         execution_project = profile_credentials.execution_project
         location = getattr(profile_credentials, "location", None)
 
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 9a198658c..dde6f865d 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -8,11 +8,14 @@
 
 from dbt import ui  # type: ignore
 from dbt.adapters.base import BaseAdapter, available, RelationType, SchemaSearchMap, AdapterConfig
+from dbt.adapters.base.impl import log_code_execution
+
 from dbt.adapters.cache import _make_key
 
 from dbt.adapters.bigquery.relation import BigQueryRelation
 from dbt.adapters.bigquery import BigQueryColumn
 from dbt.adapters.bigquery import BigQueryConnectionManager
+from dbt.adapters.bigquery.connections import BigQueryAdapterResponse
 from dbt.contracts.graph.manifest import Manifest
 from dbt.events import AdapterLogger
 from dbt.events.functions import fire_event
@@ -286,7 +289,8 @@ def get_relation(self, database: str, schema: str, identifier: str) -> BigQueryR
     # TODO: the code below is copy-pasted from SQLAdapter.create_schema. Is there a better way?
     def create_schema(self, relation: BigQueryRelation) -> None:
         # use SQL 'create schema'
-        relation = relation.without_identifier()
+        relation = relation.without_identifier()  # type: ignore
+
         fire_event(SchemaCreation(relation=_make_key(relation)))
         kwargs = {
             "relation": relation,
@@ -373,11 +377,11 @@ def _materialize_as_view(self, model: Dict[str, Any]) -> str:
         model_database = model.get("database")
         model_schema = model.get("schema")
         model_alias = model.get("alias")
-        model_sql = model.get("compiled_sql")
+        model_code = model.get("compiled_code")
 
-        logger.debug("Model SQL ({}):\n{}".format(model_alias, model_sql))
+        logger.debug("Model SQL ({}):\n{}".format(model_alias, model_code))
         self.connections.create_view(
-            database=model_database, schema=model_schema, table_name=model_alias, sql=model_sql
+            database=model_database, schema=model_schema, table_name=model_alias, sql=model_code
         )
         return "CREATE VIEW"
 
@@ -830,3 +834,109 @@ def run_sql_for_tests(self, sql, fetch, conn=None):
             return res[0]
         else:
             return list(res)
+
+    @available.parse_none
+    @log_code_execution
+    def submit_python_job(self, parsed_model: dict, compiled_code: str):
+        # TODO improve the typing here.  N.B. Jinja returns a `jinja2.runtime.Undefined` instead
+        # of `None` which evaluates to True!
+
+        # TODO limit this function to run only when doing the materialization of python nodes
+        # TODO should we also to timeout here?
+
+        # validate all additional stuff for python is set
+        schema = getattr(parsed_model, "schema", self.config.credentials.schema)
+        identifier = parsed_model["alias"]
+        python_required_configs = [
+            "dataproc_region",
+            "dataproc_cluster_name",
+            "gcs_bucket",
+        ]
+        for required_config in python_required_configs:
+            if not getattr(self.connections.profile.credentials, required_config):
+                raise ValueError(
+                    f"Need to supply {required_config} in profile to submit python job"
+                )
+        if not hasattr(self, "dataproc_helper"):
+            self.dataproc_helper = DataProcHelper(self.connections.profile.credentials)
+        model_file_name = f"{schema}/{identifier}.py"
+        # upload python file to GCS
+        self.dataproc_helper.upload_to_gcs(model_file_name, compiled_code)
+        # submit dataproc job
+        self.dataproc_helper.submit_dataproc_job(model_file_name)
+
+        # TODO proper result for this
+        message = "OK"
+        code = None
+        num_rows = None
+        bytes_processed = None
+        return BigQueryAdapterResponse(  # type: ignore[call-arg]
+            _message=message,
+            rows_affected=num_rows,
+            code=code,
+            bytes_processed=bytes_processed,
+        )
+
+
+class DataProcHelper:
+    def __init__(self, credential):
+        """_summary_
+
+        Args:
+            credential (_type_): _description_
+        """
+        try:
+            # Library only needed for python models
+            from google.cloud import dataproc_v1
+            from google.cloud import storage
+        except ImportError:
+            raise RuntimeError(
+                "You need to install [dataproc] extras to run python model in dbt-bigquery"
+            )
+        self.credential = credential
+        self.GoogleCredentials = BigQueryConnectionManager.get_credentials(credential)
+        self.storage_client = storage.Client(
+            project=self.credential.database, credentials=self.GoogleCredentials
+        )
+        self.job_client = dataproc_v1.JobControllerClient(
+            client_options={
+                "api_endpoint": "{}-dataproc.googleapis.com:443".format(
+                    self.credential.dataproc_region
+                )
+            },
+            credentials=self.GoogleCredentials,
+        )
+
+    def upload_to_gcs(self, filename: str, compiled_code: str):
+        bucket = self.storage_client.get_bucket(self.credential.gcs_bucket)
+        blob = bucket.blob(filename)
+        blob.upload_from_string(compiled_code)
+
+    def submit_dataproc_job(self, filename: str):
+        # Create the job config.
+        job = {
+            "placement": {"cluster_name": self.credential.dataproc_cluster_name},
+            "pyspark_job": {
+                "main_python_file_uri": "gs://{}/{}".format(self.credential.gcs_bucket, filename)
+            },
+        }
+        operation = self.job_client.submit_job_as_operation(
+            request={
+                "project_id": self.credential.database,
+                "region": self.credential.dataproc_region,
+                "job": job,
+            }
+        )
+        response = operation.result()
+        return response
+
+        # TODO: there might be useful results here that we can parse and return
+        # Dataproc job output is saved to the Cloud Storage bucket
+        # allocated to the job. Use regex to obtain the bucket and blob info.
+        # matches = re.match("gs://(.*?)/(.*)", response.driver_output_resource_uri)
+        # output = (
+        #     self.storage_client
+        #     .get_bucket(matches.group(1))
+        #     .blob(f"{matches.group(2)}.000000000")
+        #     .download_as_string()
+        # )
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index f75947001..cbfba2627 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -41,22 +41,35 @@
   {%- do return(bigquery_options(opts)) -%}
 {%- endmacro -%}
 
-{% macro bigquery__create_table_as(temporary, relation, sql) -%}
-  {%- set raw_partition_by = config.get('partition_by', none) -%}
-  {%- set raw_cluster_by = config.get('cluster_by', none) -%}
-  {%- set sql_header = config.get('sql_header', none) -%}
-
-  {%- set partition_config = adapter.parse_partition_by(raw_partition_by) -%}
-
-  {{ sql_header if sql_header is not none }}
-
-  create or replace table {{ relation }}
-  {{ partition_by(partition_config) }}
-  {{ cluster_by(raw_cluster_by) }}
-  {{ bigquery_table_options(config, model, temporary) }}
-  as (
-    {{ sql }}
-  );
+{% macro bigquery__create_table_as(temporary, relation, compiled_code, language='sql') -%}
+  {%- if language == 'sql' -%}
+    {%- set raw_partition_by = config.get('partition_by', none) -%}
+    {%- set raw_cluster_by = config.get('cluster_by', none) -%}
+    {%- set sql_header = config.get('sql_header', none) -%}
+
+    {%- set partition_config = adapter.parse_partition_by(raw_partition_by) -%}
+
+    {{ sql_header if sql_header is not none }}
+
+    create or replace table {{ relation }}
+    {{ partition_by(partition_config) }}
+    {{ cluster_by(raw_cluster_by) }}
+    {{ bigquery_table_options(config, model, temporary) }}
+    as (
+      {{ compiled_code }}
+    );
+  {%- elif language == 'python' -%}
+    {#--
+    N.B. Python models _can_ write to temp views HOWEVER they use a different session
+    and have already expired by the time they need to be used (I.E. in merges for incremental models)
+
+    TODO: Deep dive into spark sessions to see if we can reuse a single session for an entire
+    dbt invocation.
+     --#}
+    {{ py_write_table(compiled_code=compiled_code, target_relation=relation.quote(database=False, schema=False, identifier=False)) }}
+  {%- else -%}
+    {% do exceptions.raise_compiler_error("bigquery__create_table_as macro didn't get supported language, it got %s" % language) %}
+  {%- endif -%}
 
 {%- endmacro -%}
 
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index 1ff7e1c80..c699e816e 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -1,13 +1,14 @@
-{% macro declare_dbt_max_partition(relation, partition_by, sql) %}
+{% macro declare_dbt_max_partition(relation, partition_by, complied_code, language='sql') %}
 
-  {% if '_dbt_max_partition' in sql %}
+  {#-- TODO: revisit partitioning with python models --#}
+  {%- if '_dbt_max_partition' in complied_code and language == 'sql' -%}
 
     declare _dbt_max_partition {{ partition_by.data_type }} default (
       select max({{ partition_by.field }}) from {{ this }}
       where {{ partition_by.field }} is not null
     );
 
-  {% endif %}
+  {%- endif -%}
 
 {% endmacro %}
 
@@ -74,7 +75,7 @@
         {{ declare_dbt_max_partition(this, partition_by, sql) }}
 
         -- 1. create a temp table
-        {{ create_table_as(True, tmp_relation, sql) }}
+        {{ create_table_as(True, tmp_relation, compiled_code) }}
       {% else %}
         -- 1. temp table already exists, we used it to check for schema changes
       {% endif %}
@@ -139,6 +140,7 @@
 
   {%- set unique_key = config.get('unique_key') -%}
   {%- set full_refresh_mode = (should_full_refresh()) -%}
+  {%- set language = model['language'] %}
 
   {%- set target_relation = this %}
   {%- set existing_relation = load_relation(this) %}
@@ -160,12 +162,16 @@
   {{ run_hooks(pre_hooks) }}
 
   {% if existing_relation is none %}
-      {% set build_sql = create_table_as(False, target_relation, sql) %}
+      {%- call statement('main', language=language) -%}
+        {{ create_table_as(False, target_relation, compiled_code, language) }}
+      {%- endcall -%}
 
   {% elif existing_relation.is_view %}
       {#-- There's no way to atomically replace a view with a table on BQ --#}
       {{ adapter.drop_relation(existing_relation) }}
-      {% set build_sql = create_table_as(False, target_relation, sql) %}
+      {%- call statement('main', language=language) -%}
+        {{ create_table_as(False, target_relation, compiled_code, language) }}
+      {%- endcall -%}
 
   {% elif full_refresh_mode %}
       {#-- If the partition/cluster config has changed, then we must drop and recreate --#}
@@ -173,30 +179,49 @@
           {% do log("Hard refreshing " ~ existing_relation ~ " because it is not replaceable") %}
           {{ adapter.drop_relation(existing_relation) }}
       {% endif %}
-      {% set build_sql = create_table_as(False, target_relation, sql) %}
+      {%- call statement('main', language=language) -%}
+        {{ create_table_as(False, target_relation, compiled_code, language) }}
+      {%- endcall -%}
 
   {% else %}
+    {%- if language == 'python' and strategy == 'insert_overwrite' -%}
+      {#-- This lets us move forward assuming no python will be directly templated into a query --#}
+      {%- set python_unsupported_msg -%}
+        The 'insert_overwrite' strategy is not yet supported for python models.
+      {%- endset %}
+      {% do exceptions.raise_compiler_error(python_unsupported_msg) %}
+    {%- endif -%}
+
     {% set tmp_relation_exists = false %}
-    {% if on_schema_change != 'ignore' %} {# Check first, since otherwise we may not build a temp table #}
-      {% do run_query(
-        declare_dbt_max_partition(this, partition_by, sql) + create_table_as(True, tmp_relation, sql)
-      ) %}
+    {% if on_schema_change != 'ignore' or language == 'python' %}
+      {#-- Check first, since otherwise we may not build a temp table --#}
+      {#-- Python always needs to create a temp table --#}
+      {%- call statement('create_tmp_relation', language=language) -%}
+        {{ declare_dbt_max_partition(this, partition_by, compiled_code, language) +
+           create_table_as(True, tmp_relation, compiled_code, language)
+        }}
+      {%- endcall -%}
       {% set tmp_relation_exists = true %}
       {#-- Process schema changes. Returns dict of changes if successful. Use source columns for upserting/merging --#}
       {% set dest_columns = process_schema_changes(on_schema_change, tmp_relation, existing_relation) %}
     {% endif %}
+
     {% if not dest_columns %}
       {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %}
     {% endif %}
     {% set build_sql = bq_generate_incremental_build_sql(
-        strategy, tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
+        strategy, tmp_relation, target_relation, compiled_code, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
     ) %}
 
-  {% endif %}
+    {%- call statement('main') -%}
+      {{ build_sql }}
+    {% endcall %}
+
+    {%- if language == 'python' and tmp_relation -%}
+      {{ adapter.drop_relation(tmp_relation) }}
+    {%- endif -%}
 
-  {%- call statement('main') -%}
-    {{ build_sql }}
-  {% endcall %}
+  {% endif %}
 
   {{ run_hooks(post_hooks) }}
 
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index 9e63637c1..886ba3fa9 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -1,5 +1,6 @@
 {% materialization table, adapter='bigquery' -%}
 
+  {%- set language = model['language'] -%}
   {%- set identifier = model['alias'] -%}
   {%- set old_relation = adapter.get_relation(database=database, schema=schema, identifier=identifier) -%}
   {%- set exists_not_as_table = (old_relation is not none and not old_relation.is_table) -%}
@@ -27,9 +28,11 @@
     {% do log("Hard refreshing " ~ old_relation ~ " because it is not replaceable") %}
     {% do adapter.drop_relation(old_relation) %}
   {% endif %}
-  {% call statement('main') -%}
-    {{ create_table_as(False, target_relation, sql) }}
-  {% endcall -%}
+
+  -- build model
+  {%- call statement('main', language=language) -%}
+    {{ create_table_as(False, target_relation, compiled_code, language) }}
+  {%- endcall -%}
 
   {{ run_hooks(post_hooks) }}
 
@@ -41,3 +44,29 @@
   {{ return({'relations': [target_relation]}) }}
 
 {% endmaterialization %}
+
+-- TODO dataproc requires a temp bucket to perform BQ write
+-- this is hard coded to internal testing ATM. need to adjust to render
+-- or find another way around
+{% macro py_write_table(compiled_code, target_relation) %}
+from pyspark.sql import SparkSession
+
+spark = SparkSession.builder.appName('smallTest').getOrCreate()
+
+spark.conf.set("viewsEnabled","true")
+spark.conf.set("temporaryGcsBucket","{{target.gcs_bucket}}")
+
+{{ compiled_code }}
+dbt = dbtObj(spark.read.format("bigquery").load)
+df = model(dbt, spark)
+
+# COMMAND ----------
+# this is materialization code dbt generated, please do not modify
+
+
+df.write \
+  .mode("overwrite") \
+  .format("bigquery") \
+  .option("writeMethod", "direct") \
+  .save("{{target_relation}}")
+{% endmacro %}
diff --git a/requirements.txt b/requirements.txt
new file mode 100644
index 000000000..7cf32603d
--- /dev/null
+++ b/requirements.txt
@@ -0,0 +1,2 @@
+google-cloud-storage[dataproc]>=2.4.0
+google-cloud-dataproc[dataproc]>=4.0.3
diff --git a/test.env.example b/test.env.example
index f2e59e6d0..d49649307 100644
--- a/test.env.example
+++ b/test.env.example
@@ -1,7 +1,11 @@
 BIGQUERY_TEST_ALT_DATABASE=
 BIGQUERY_TEST_NO_ACCESS_DATABASE=
 BIGQUERY_TEST_SERVICE_ACCOUNT_JSON='{}'
-
 DBT_TEST_USER_1="group:buildbot@dbtlabs.com"
 DBT_TEST_USER_2="group:dev-core@dbtlabs.com"
 DBT_TEST_USER_3="serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserviceaccount.com"
+
+# only needed for python model
+DATAPROC_REGION=us-
+DATAPROC_CLUSTER_NAME=
+GCS_BUCKET=
diff --git a/tests/conftest.py b/tests/conftest.py
index 69a29b39c..7b0c69fc3 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -42,4 +42,8 @@ def service_account_target():
         'threads': 1,
         'project': project_id,
         'keyfile_json': credentials,
+        # following 3 for python model
+        'dataproc_region': os.getenv("DATAPROC_REGION"),
+        'dataproc_cluster_name': os.getenv("DATAPROC_CLUSTER_NAME"),
+        'gcs_bucket': os.getenv("GCS_BUCKET")
     }
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
new file mode 100644
index 000000000..3602bb9b9
--- /dev/null
+++ b/tests/functional/adapter/test_python_model.py
@@ -0,0 +1,49 @@
+import os
+import pytest
+from dbt.tests.util import run_dbt, write_file
+import dbt.tests.adapter.python_model.test_python_model as dbt_tests
+
+class TestPythonIncrementalMatsDataproc(dbt_tests.BasePythonIncrementalTests):
+    pass
+
+class TestPythonModelDataproc(dbt_tests.BasePythonModelTests):
+    pass
+
+models__simple_python_model = """
+import pandas
+
+def model(dbt, spark):
+    dbt.config(
+        materialized='table',
+    )
+    data = [[1,2]] * 10
+    return spark.createDataFrame(data, schema=['test', 'test2'])
+"""
+models__simple_python_model_v2 = """
+import pandas
+
+def model(dbt, spark):
+    dbt.config(
+        materialized='table',
+    )
+    data = [[1,2]] * 10
+    return spark.createDataFrame(data, schema=['test1', 'test3'])
+"""
+
+class TestChangingSchemaDataproc:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "simple_python_model.py": models__simple_python_model
+            }
+    def test_changing_schema(self,project, logs_dir):
+        run_dbt(["run"])
+        write_file(models__simple_python_model_v2, project.project_root + '/models', "simple_python_model.py")
+        run_dbt(["run"])
+        log_file = os.path.join(logs_dir, "dbt.log")
+        with open(log_file, "r") as f:
+            log = f.read()
+            # validate #5510 log_code_execution works
+            assert "On model.test.simple_python_model:" in log
+            assert "return spark.createDataFrame(data, schema=['test1', 'test3'])" in log
+            assert "Execution status: OK in" in log
diff --git a/tests/integration/defer_state_test/test_defer_state.py b/tests/integration/defer_state_test/test_defer_state.py
index 16e0df5f8..d8296babf 100644
--- a/tests/integration/defer_state_test/test_defer_state.py
+++ b/tests/integration/defer_state_test/test_defer_state.py
@@ -71,8 +71,8 @@ def run_and_defer(self):
 
         # with state it should work though
         results = self.run_dbt(['run', '-m', 'view_model', '--state', 'state', '--defer', '--target', 'otherschema'])
-        assert self.other_schema not in results[0].node.compiled_sql
-        assert self.unique_schema() in results[0].node.compiled_sql
+        assert self.other_schema not in results[0].node.compiled_code
+        assert self.unique_schema() in results[0].node.compiled_code
 
         with open('target/manifest.json') as fp:
             data = json.load(fp)
diff --git a/tox.ini b/tox.ini
index 80191db8c..ae96fa823 100644
--- a/tox.ini
+++ b/tox.ini
@@ -14,10 +14,11 @@ deps =
 [testenv:{integration,py37,py38,py39,py310,py}-{bigquery}]
 description = adapter plugin integration testing
 skip_install = true
-passenv = DBT_* BIGQUERY_TEST_* PYTEST_ADDOPTS
+passenv = DBT_* BIGQUERY_TEST_* PYTEST_ADDOPTS DATAPROC_* GCS_BUCKET
 commands =
   bigquery: {envpython} -m pytest {posargs} -m profile_bigquery tests/integration
   bigquery: {envpython} -m pytest {posargs} -vv tests/functional --profile service_account
 deps =
   -rdev-requirements.txt
   -e.
+  -rrequirements.txt

From ba3acf76bb4e8d4d55ddb62efe83798fe951954d Mon Sep 17 00:00:00 2001
From: Ian Knox <81931810+iknox-fa@users.noreply.github.com>
Date: Fri, 29 Jul 2022 11:20:10 -0500
Subject: [PATCH 462/860] requirements.txt incorrect (#245)

---
 requirements.txt | 2 --
 1 file changed, 2 deletions(-)
 delete mode 100644 requirements.txt

diff --git a/requirements.txt b/requirements.txt
deleted file mode 100644
index 7cf32603d..000000000
--- a/requirements.txt
+++ /dev/null
@@ -1,2 +0,0 @@
-google-cloud-storage[dataproc]>=2.4.0
-google-cloud-dataproc[dataproc]>=4.0.3

From 19207a6614464693cbe72c57a475ba10b437450e Mon Sep 17 00:00:00 2001
From: Chenyu Li 
Date: Fri, 29 Jul 2022 11:38:35 -0700
Subject: [PATCH 463/860] Add extra requirements in setup.py and skip python
 model tests (#246)

---
 dev-requirements.txt                          | 4 ++++
 setup.py                                      | 6 ++++++
 tests/functional/adapter/test_python_model.py | 1 +
 tox.ini                                       | 1 -
 4 files changed, 11 insertions(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index 2775ca784..5ff54ac51 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -22,3 +22,7 @@ tox>=3.13
 types-requests
 twine
 wheel
+
+# For dataproc running
+google-cloud-storage[dataproc]>=2.4.0
+google-cloud-dataproc[dataproc]>=4.0.3
diff --git a/setup.py b/setup.py
index 38ca5a599..ee47e85b5 100644
--- a/setup.py
+++ b/setup.py
@@ -73,6 +73,12 @@ def _get_dbt_core_version():
         "google-api-core>=1.16.0,<3",
         "googleapis-common-protos>=1.6.0,<2",
     ],
+    extras_require={
+        "dataproc": [
+            "google-cloud-storage[dataproc]>=2.4.0",
+            "google-cloud-dataproc[dataproc]>=4.0.3",
+        ],
+    },
     zip_safe=False,
     classifiers=[
         "Development Status :: 5 - Production/Stable",
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
index 3602bb9b9..03ad871e2 100644
--- a/tests/functional/adapter/test_python_model.py
+++ b/tests/functional/adapter/test_python_model.py
@@ -3,6 +3,7 @@
 from dbt.tests.util import run_dbt, write_file
 import dbt.tests.adapter.python_model.test_python_model as dbt_tests
 
+@pytest.skip("cluster unstable", allow_module_level=True)
 class TestPythonIncrementalMatsDataproc(dbt_tests.BasePythonIncrementalTests):
     pass
 
diff --git a/tox.ini b/tox.ini
index ae96fa823..38797c0cf 100644
--- a/tox.ini
+++ b/tox.ini
@@ -21,4 +21,3 @@ commands =
 deps =
   -rdev-requirements.txt
   -e.
-  -rrequirements.txt

From f14e4c269ca5deb18fa3823f5e40586c95a542d9 Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Fri, 29 Jul 2022 15:49:30 -0400
Subject: [PATCH 464/860] Bumping version to 1.3.0b1 (#243)

* Bumping version to 1.3.0b1

* update changelog and fix format

* Update CHANGELOG.md

Co-authored-by: Github Build Bot 
Co-authored-by: Chenyu Li 
Co-authored-by: leahwicz <60146280+leahwicz@users.noreply.github.com>
---
 .bumpversion.cfg                     | 2 +-
 CHANGELOG.md                         | 6 ++++--
 dbt/adapters/bigquery/__version__.py | 2 +-
 setup.py                             | 2 +-
 4 files changed, 7 insertions(+), 5 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 1fd657599..f0da053eb 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.3.0a1
+current_version = 1.3.0b1
 parse = (?P\d+)
 	\.(?P\d+)
 	\.(?P\d+)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index af1c1f13f..f6a823c3d 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,8 +1,10 @@
-## dbt-bigquery 1.3.0b1 (Release TBD)
+## dbt-bigquery 1.3.0b2 (Release TBD)
+
+## dbt-bigquery 1.3.0b1 (July 29, 2020)
 
 ### Features
 - Implement `create_schema` via SQL, instead of Python method, allowing users to override if desired. `drop_schema` remains a Python method for the time being. ([#182](https://github.com/dbt-labs/dbt-bigquery/issues/182), [#183](https://github.com/dbt-labs/dbt-bigquery/pull/183))
-- Added incrementail materializations for python models via DataProc. ([#226](https://github.com/dbt-labs/dbt-bigquery/pull/226))
+- Added table and incrementail materializations for python models via DataProc. ([#209](https://github.com/dbt-labs/dbt-bigquery/pull/209), [#226](https://github.com/dbt-labs/dbt-bigquery/pull/226))
 
 ### Under the hood
 - Implement minimal changes to support dbt Core incremental materialization refactor. ([#232](http://github.com/dbt-labs/dbt-bigquery/issues/232), [#223](https://github.com/dbt-labs/dbt-biquery/pull/223))
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index a9fe3c3ee..4b49b750d 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.3.0a1"
+version = "1.3.0b1"
diff --git a/setup.py b/setup.py
index ee47e85b5..175fa2f1a 100644
--- a/setup.py
+++ b/setup.py
@@ -50,7 +50,7 @@ def _get_dbt_core_version():
 
 
 package_name = "dbt-bigquery"
-package_version = "1.3.0a1"
+package_version = "1.3.0b1"
 dbt_core_version = _get_dbt_core_version()
 description = """The BigQuery adapter plugin for dbt"""
 

From e2273d44da4326d2296311324e883b096ab2aae7 Mon Sep 17 00:00:00 2001
From: Christophe Oudar 
Date: Thu, 4 Aug 2022 18:09:43 +0200
Subject: [PATCH 465/860] Add job_id/project_id to adapter response to enable
 easy job linking (#225)

---
 CHANGELOG.md                                  |  4 ++
 dbt/adapters/bigquery/connections.py          | 56 +++++++++++++------
 .../test_bigquery_adapter_functions.py        |  6 ++
 3 files changed, 50 insertions(+), 16 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index f6a823c3d..e765c8127 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -8,7 +8,11 @@
 
 ### Under the hood
 - Implement minimal changes to support dbt Core incremental materialization refactor. ([#232](http://github.com/dbt-labs/dbt-bigquery/issues/232), [#223](https://github.com/dbt-labs/dbt-biquery/pull/223))
+- Add job_id/project_id to adapter response to enable easy job linking ([#225](https://github.com/dbt-labs/dbt-bigquery/pull/225))
 
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#225](https://github.com/dbt-labs/dbt-bigquery/pull/225))
+-
 ## dbt-bigquery 1.2.0rc1 (June 12, 2022)
 
 ### Features
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index da7b0f687..3972f613b 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -86,6 +86,9 @@ class BigQueryConnectionMethod(StrEnum):
 @dataclass
 class BigQueryAdapterResponse(AdapterResponse):
     bytes_processed: Optional[int] = None
+    location: Optional[str] = None
+    project_id: Optional[str] = None
+    job_id: Optional[str] = None
 
 
 @dataclass
@@ -188,6 +191,12 @@ class BigQueryConnectionManager(BaseConnectionManager):
     @classmethod
     def handle_error(cls, error, message):
         error_msg = "\n".join([item["message"] for item in error.errors])
+        if hasattr(error, "query_job"):
+            logger.error(
+                cls._bq_job_link(
+                    error.query_job.location, error.query_job.project_id, error.query_job.job_id
+                )
+            )
         raise DatabaseException(error_msg)
 
     def clear_transaction(self):
@@ -446,55 +455,70 @@ def execute(
         code = None
         num_rows = None
         bytes_processed = None
+        location = None
+        job_id = None
+        project_id = None
+        num_rows_formatted = None
+        processed_bytes = None
 
         if query_job.statement_type == "CREATE_VIEW":
             code = "CREATE VIEW"
 
         elif query_job.statement_type == "CREATE_TABLE_AS_SELECT":
+            code = "CREATE TABLE"
             conn = self.get_thread_connection()
             client = conn.handle
             query_table = client.get_table(query_job.destination)
-            code = "CREATE TABLE"
             num_rows = query_table.num_rows
-            num_rows_formated = self.format_rows_number(num_rows)
-            bytes_processed = query_job.total_bytes_processed
-            processed_bytes = self.format_bytes(bytes_processed)
-            message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
 
         elif query_job.statement_type == "SCRIPT":
             code = "SCRIPT"
-            bytes_processed = query_job.total_bytes_processed
-            message = f"{code} ({self.format_bytes(bytes_processed)} processed)"
 
         elif query_job.statement_type in ["INSERT", "DELETE", "MERGE", "UPDATE"]:
             code = query_job.statement_type
             num_rows = query_job.num_dml_affected_rows
-            num_rows_formated = self.format_rows_number(num_rows)
-            bytes_processed = query_job.total_bytes_processed
-            processed_bytes = self.format_bytes(bytes_processed)
-            message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
 
         elif query_job.statement_type == "SELECT":
+            code = "SELECT"
             conn = self.get_thread_connection()
             client = conn.handle
             # use anonymous table for num_rows
             query_table = client.get_table(query_job.destination)
-            code = "SELECT"
             num_rows = query_table.num_rows
-            num_rows_formated = self.format_rows_number(num_rows)
-            bytes_processed = query_job.total_bytes_processed
-            processed_bytes = self.format_bytes(bytes_processed)
-            message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
+
+        # set common attributes
+        bytes_processed = query_job.total_bytes_processed
+        processed_bytes = self.format_bytes(bytes_processed)
+        location = query_job.location
+        job_id = query_job.job_id
+        project_id = query_job.project
+        if num_rows is not None:
+            num_rows_formatted = self.format_rows_number(num_rows)
+            message = f"{code} ({num_rows_formatted} rows, {processed_bytes} processed)"
+        elif bytes_processed is not None:
+            message = f"{code} ({processed_bytes} processed)"
+        else:
+            message = f"{code}"
+
+        if location is not None and job_id is not None and project_id is not None:
+            logger.debug(self._bq_job_link(job_id, project_id, location))
 
         response = BigQueryAdapterResponse(  # type: ignore[call-arg]
             _message=message,
             rows_affected=num_rows,
             code=code,
             bytes_processed=bytes_processed,
+            location=location,
+            project_id=project_id,
+            job_id=job_id,
         )
 
         return response, table
 
+    @staticmethod
+    def _bq_job_link(location, project_id, job_id) -> str:
+        return f"https://console.cloud.google.com/bigquery?project={project_id}&j=bq:{location}:{job_id}&page=queryresults"
+
     def get_partitions_metadata(self, table):
         def standard_to_legacy(table):
             return table.project + ":" + table.dataset + "." + table.identifier
diff --git a/tests/integration/bigquery_test/test_bigquery_adapter_functions.py b/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
index 06e230f30..c0793e9c7 100644
--- a/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
+++ b/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
@@ -21,6 +21,12 @@ def test__bigquery_adapter_functions(self):
         results = self.run_dbt()
         self.assertEqual(len(results), 3)
 
+        for result in results:
+            # all queries in adapter models are jobs that are expected to have a job_id
+            assert result.adapter_response["location"] is not None
+            assert result.adapter_response["job_id"] is not None
+            assert result.adapter_response["project_id"] is not None
+
         test_results = self.run_dbt(['test'])
 
         self.assertTrue(len(test_results) > 0)

From f1b529100665e7d33188617b5ed37e572de5ba9e Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 4 Aug 2022 14:12:44 -0500
Subject: [PATCH 466/860] Revert "Add job_id/project_id to adapter response to
 enable easy job linking (#225)" (#249)

This reverts commit e2273d44da4326d2296311324e883b096ab2aae7.

Co-authored-by: Jeremy Cohen 
---
 CHANGELOG.md                                  |  4 --
 dbt/adapters/bigquery/connections.py          | 56 ++++++-------------
 .../test_bigquery_adapter_functions.py        |  6 --
 3 files changed, 16 insertions(+), 50 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index e765c8127..f6a823c3d 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -8,11 +8,7 @@
 
 ### Under the hood
 - Implement minimal changes to support dbt Core incremental materialization refactor. ([#232](http://github.com/dbt-labs/dbt-bigquery/issues/232), [#223](https://github.com/dbt-labs/dbt-biquery/pull/223))
-- Add job_id/project_id to adapter response to enable easy job linking ([#225](https://github.com/dbt-labs/dbt-bigquery/pull/225))
 
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#225](https://github.com/dbt-labs/dbt-bigquery/pull/225))
--
 ## dbt-bigquery 1.2.0rc1 (June 12, 2022)
 
 ### Features
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 3972f613b..da7b0f687 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -86,9 +86,6 @@ class BigQueryConnectionMethod(StrEnum):
 @dataclass
 class BigQueryAdapterResponse(AdapterResponse):
     bytes_processed: Optional[int] = None
-    location: Optional[str] = None
-    project_id: Optional[str] = None
-    job_id: Optional[str] = None
 
 
 @dataclass
@@ -191,12 +188,6 @@ class BigQueryConnectionManager(BaseConnectionManager):
     @classmethod
     def handle_error(cls, error, message):
         error_msg = "\n".join([item["message"] for item in error.errors])
-        if hasattr(error, "query_job"):
-            logger.error(
-                cls._bq_job_link(
-                    error.query_job.location, error.query_job.project_id, error.query_job.job_id
-                )
-            )
         raise DatabaseException(error_msg)
 
     def clear_transaction(self):
@@ -455,70 +446,55 @@ def execute(
         code = None
         num_rows = None
         bytes_processed = None
-        location = None
-        job_id = None
-        project_id = None
-        num_rows_formatted = None
-        processed_bytes = None
 
         if query_job.statement_type == "CREATE_VIEW":
             code = "CREATE VIEW"
 
         elif query_job.statement_type == "CREATE_TABLE_AS_SELECT":
-            code = "CREATE TABLE"
             conn = self.get_thread_connection()
             client = conn.handle
             query_table = client.get_table(query_job.destination)
+            code = "CREATE TABLE"
             num_rows = query_table.num_rows
+            num_rows_formated = self.format_rows_number(num_rows)
+            bytes_processed = query_job.total_bytes_processed
+            processed_bytes = self.format_bytes(bytes_processed)
+            message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
 
         elif query_job.statement_type == "SCRIPT":
             code = "SCRIPT"
+            bytes_processed = query_job.total_bytes_processed
+            message = f"{code} ({self.format_bytes(bytes_processed)} processed)"
 
         elif query_job.statement_type in ["INSERT", "DELETE", "MERGE", "UPDATE"]:
             code = query_job.statement_type
             num_rows = query_job.num_dml_affected_rows
+            num_rows_formated = self.format_rows_number(num_rows)
+            bytes_processed = query_job.total_bytes_processed
+            processed_bytes = self.format_bytes(bytes_processed)
+            message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
 
         elif query_job.statement_type == "SELECT":
-            code = "SELECT"
             conn = self.get_thread_connection()
             client = conn.handle
             # use anonymous table for num_rows
             query_table = client.get_table(query_job.destination)
+            code = "SELECT"
             num_rows = query_table.num_rows
-
-        # set common attributes
-        bytes_processed = query_job.total_bytes_processed
-        processed_bytes = self.format_bytes(bytes_processed)
-        location = query_job.location
-        job_id = query_job.job_id
-        project_id = query_job.project
-        if num_rows is not None:
-            num_rows_formatted = self.format_rows_number(num_rows)
-            message = f"{code} ({num_rows_formatted} rows, {processed_bytes} processed)"
-        elif bytes_processed is not None:
-            message = f"{code} ({processed_bytes} processed)"
-        else:
-            message = f"{code}"
-
-        if location is not None and job_id is not None and project_id is not None:
-            logger.debug(self._bq_job_link(job_id, project_id, location))
+            num_rows_formated = self.format_rows_number(num_rows)
+            bytes_processed = query_job.total_bytes_processed
+            processed_bytes = self.format_bytes(bytes_processed)
+            message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
 
         response = BigQueryAdapterResponse(  # type: ignore[call-arg]
             _message=message,
             rows_affected=num_rows,
             code=code,
             bytes_processed=bytes_processed,
-            location=location,
-            project_id=project_id,
-            job_id=job_id,
         )
 
         return response, table
 
-    @staticmethod
-    def _bq_job_link(location, project_id, job_id) -> str:
-        return f"https://console.cloud.google.com/bigquery?project={project_id}&j=bq:{location}:{job_id}&page=queryresults"
-
     def get_partitions_metadata(self, table):
         def standard_to_legacy(table):
             return table.project + ":" + table.dataset + "." + table.identifier
diff --git a/tests/integration/bigquery_test/test_bigquery_adapter_functions.py b/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
index c0793e9c7..06e230f30 100644
--- a/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
+++ b/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
@@ -21,12 +21,6 @@ def test__bigquery_adapter_functions(self):
         results = self.run_dbt()
         self.assertEqual(len(results), 3)
 
-        for result in results:
-            # all queries in adapter models are jobs that are expected to have a job_id
-            assert result.adapter_response["location"] is not None
-            assert result.adapter_response["job_id"] is not None
-            assert result.adapter_response["project_id"] is not None
-
         test_results = self.run_dbt(['test'])
 
         self.assertTrue(len(test_results) > 0)

From 64e9da7bde99cc89e4755d6156c6728ab1eadf50 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Fri, 5 Aug 2022 13:01:40 -0500
Subject: [PATCH 467/860] [Feature ] implement changie in dbt-bigquery (#253)

* init pr for changie swap over in dbt-bigquery

* typo fix

* convert 1.3 changelog including batch and merge

* changelog entry
---
 .changes/0.0.0.md                             |   5 +
 .changes/1.3.0-b1.md                          |   6 +
 .changes/1.3.0/Features-20220804-154944.yaml  |   8 +
 .changes/1.3.0/Features-20220804-155057.yaml  |   7 +
 .../1.3.0/Under the Hood-20220804-155219.yaml |   7 +
 .changes/README.md                            |   3 +
 .changes/header.tpl.md                        |   6 +
 .changes/unreleased/.gitkeep                  |   0
 .../Under the Hood-20220804-155719.yaml       |   7 +
 .changie.yaml                                 |  60 +++++++
 .github/pull_request_template.md              |   2 +-
 .github/workflows/bot-changelog.yml           |  61 +++++++
 .github/workflows/changelog-existence.yml     |  41 +++++
 CHANGELOG.md                                  | 150 ++----------------
 CONTRIBUTING.md                               |   9 ++
 15 files changed, 237 insertions(+), 135 deletions(-)
 create mode 100644 .changes/0.0.0.md
 create mode 100644 .changes/1.3.0-b1.md
 create mode 100644 .changes/1.3.0/Features-20220804-154944.yaml
 create mode 100644 .changes/1.3.0/Features-20220804-155057.yaml
 create mode 100644 .changes/1.3.0/Under the Hood-20220804-155219.yaml
 create mode 100644 .changes/README.md
 create mode 100644 .changes/header.tpl.md
 create mode 100644 .changes/unreleased/.gitkeep
 create mode 100644 .changes/unreleased/Under the Hood-20220804-155719.yaml
 create mode 100644 .changie.yaml
 create mode 100644 .github/workflows/bot-changelog.yml
 create mode 100644 .github/workflows/changelog-existence.yml

diff --git a/.changes/0.0.0.md b/.changes/0.0.0.md
new file mode 100644
index 000000000..9c545f562
--- /dev/null
+++ b/.changes/0.0.0.md
@@ -0,0 +1,5 @@
+## Previous Releases
+For information on prior major and minor releases, see their changelogs:
+- [1.2](https://github.com/dbt-labs/dbt-bigquery/blob/1.2.latest/CHANGELOG.md)
+- [1.1](https://github.com/dbt-labs/dbt-bigquery/blob/1.1.latest/CHANGELOG.md)
+- [1.0](https://github.com/dbt-labs/dbt-bigquery/blob/1.0.latest/CHANGELOG.md)
diff --git a/.changes/1.3.0-b1.md b/.changes/1.3.0-b1.md
new file mode 100644
index 000000000..9d296a9bb
--- /dev/null
+++ b/.changes/1.3.0-b1.md
@@ -0,0 +1,6 @@
+## dbt-bigquery 1.3.0-b1 - August 04, 2022
+### Features
+- Implement `create_schema` via SQL, instead of Python method, allowing users to override if desired. drop_schema remains a Python method for the time being.  ([#182](https://github.com/dbt-labs/dbt-bigquery/issues/182), [#183](https://github.com/dbt-labs/dbt-bigquery/pull/183))
+- Added table and incrementail materializations for python models via DataProc. ([#209](https://github.com/dbt-labs/dbt-bigquery/issues/209), [#226](https://github.com/dbt-labs/dbt-bigquery/pull/226))
+### Under the Hood
+- Implement minimal changes to support dbt Core incremental materialization refactor. ([#232](https://github.com/dbt-labs/dbt-bigquery/issues/232), [#223](https://github.com/dbt-labs/dbt-bigquery/pull/223))
diff --git a/.changes/1.3.0/Features-20220804-154944.yaml b/.changes/1.3.0/Features-20220804-154944.yaml
new file mode 100644
index 000000000..65bd2b32e
--- /dev/null
+++ b/.changes/1.3.0/Features-20220804-154944.yaml
@@ -0,0 +1,8 @@
+kind: Features
+body: 'Implement `create_schema` via SQL, instead of Python method, allowing users
+  to override if desired. drop_schema remains a Python method for the time being. '
+time: 2022-08-04T15:49:44.409354-05:00
+custom:
+  Author: jtcohen6
+  Issue: "182"
+  PR: "183"
diff --git a/.changes/1.3.0/Features-20220804-155057.yaml b/.changes/1.3.0/Features-20220804-155057.yaml
new file mode 100644
index 000000000..39bd6ad45
--- /dev/null
+++ b/.changes/1.3.0/Features-20220804-155057.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: Added table and incrementail materializations for python models via DataProc.
+time: 2022-08-04T15:50:57.234209-05:00
+custom:
+  Author: stu-k
+  Issue: "209"
+  PR: "226"
diff --git a/.changes/1.3.0/Under the Hood-20220804-155219.yaml b/.changes/1.3.0/Under the Hood-20220804-155219.yaml
new file mode 100644
index 000000000..f5e6b2d9c
--- /dev/null
+++ b/.changes/1.3.0/Under the Hood-20220804-155219.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: Implement minimal changes to support dbt Core incremental materialization refactor.
+time: 2022-08-04T15:52:19.342843-05:00
+custom:
+  Author: gshank
+  Issue: "232"
+  PR: "223"
diff --git a/.changes/README.md b/.changes/README.md
new file mode 100644
index 000000000..d82835028
--- /dev/null
+++ b/.changes/README.md
@@ -0,0 +1,3 @@
+# CHANGELOG
+
+To view information about the changelog operation we suggest reading this [README](https://github.com/dbt-labs/dbt-core/blob/main/.changes/README.md) found in `dbt-core`.
diff --git a/.changes/header.tpl.md b/.changes/header.tpl.md
new file mode 100644
index 000000000..b0468a97e
--- /dev/null
+++ b/.changes/header.tpl.md
@@ -0,0 +1,6 @@
+# dbt-bigquery Changelog
+
+- This file provides a full account of all changes to `dbt-bigquery`.
+- Changes are listed under the (pre)release in which they first appear. Subsequent releases include changes from previous releases.
+- "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
+- Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
diff --git a/.changes/unreleased/.gitkeep b/.changes/unreleased/.gitkeep
new file mode 100644
index 000000000..e69de29bb
diff --git a/.changes/unreleased/Under the Hood-20220804-155719.yaml b/.changes/unreleased/Under the Hood-20220804-155719.yaml
new file mode 100644
index 000000000..631333c02
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20220804-155719.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: Add changie to dbt-bigquery
+time: 2022-08-04T15:57:19.060995-05:00
+custom:
+  Author: mcknight-42
+  Issue: "254"
+  PR: "253"
diff --git a/.changie.yaml b/.changie.yaml
new file mode 100644
index 000000000..4a58d32e0
--- /dev/null
+++ b/.changie.yaml
@@ -0,0 +1,60 @@
+changesDir: .changes
+unreleasedDir: unreleased
+headerPath: header.tpl.md
+versionHeaderPath: ""
+changelogPath: CHANGELOG.md
+versionExt: md
+versionFormat: '## dbt-bigquery {{.Version}} - {{.Time.Format "January 02, 2006"}}'
+kindFormat: '### {{.Kind}}'
+changeFormat: '- {{.Body}} ([#{{.Custom.Issue}}](https://github.com/dbt-labs/dbt-bigquery/issues/{{.Custom.Issue}}), [#{{.Custom.PR}}](https://github.com/dbt-labs/dbt-bigquery/pull/{{.Custom.PR}}))'
+kinds:
+- label: Breaking Changes
+- label: Features
+- label: Fixes
+- label: Under the Hood
+- label: Dependencies
+- label: Security
+custom:
+- key: Author
+  label: GitHub Username(s) (separated by a single space if multiple)
+  type: string
+  minLength: 3
+- key: Issue
+  label: GitHub Issue Number
+  type: int
+  minLength: 4
+- key: PR
+  label: GitHub Pull Request Number
+  type: int
+  minLength: 4
+footerFormat: |
+  {{- $contributorDict := dict }}
+  {{- /* any names added to this list should be all lowercase for later matching purposes */}}
+  {{- $core_team := list "emmyoop" "nathaniel-may" "gshank" "leahwicz" "chenyulinx" "stu-k" "iknox-fa" "versusfacit" "mcknight-42" "jtcohen6" "dependabot[bot]" "snyk-bot" }}
+  {{- range $change := .Changes }}
+    {{- $authorList := splitList " " $change.Custom.Author }}
+    {{- /* loop through all authors for a PR */}}
+    {{- range $author := $authorList }}
+      {{- $authorLower := lower $author }}
+      {{- /* we only want to include non-core team contributors */}}
+      {{- if not (has $authorLower $core_team)}}
+        {{- $pr := $change.Custom.PR }}
+        {{- /* check if this contributor has other PRs associated with them already */}}
+        {{- if hasKey $contributorDict $author }}
+          {{- $prList := get $contributorDict $author }}
+          {{- $prList = append $prList $pr  }}
+          {{- $contributorDict := set $contributorDict $author $prList }}
+        {{- else }}
+          {{- $prList := list $change.Custom.PR }}
+          {{- $contributorDict := set $contributorDict $author $prList }}
+        {{- end }}
+      {{- end}}
+    {{- end}}
+  {{- end }}
+  {{- /* no indentation here for formatting so the final markdown doesn't have unneeded indentations */}}
+  {{- if $contributorDict}}
+  ### Contributors
+  {{- range $k,$v := $contributorDict }}
+  - [@{{$k}}](https://github.com/{{$k}}) ({{ range $index, $element := $v }}{{if $index}}, {{end}}[#{{$element}}](https://github.com/dbt-labs/dbt-bigquery/pull/{{$element}}){{end}})
+  {{- end }}
+  {{- end }}
diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md
index beedab7ad..e15849215 100644
--- a/.github/pull_request_template.md
+++ b/.github/pull_request_template.md
@@ -18,4 +18,4 @@ resolves #
 - [ ] I have signed the [CLA](https://docs.getdbt.com/docs/contributor-license-agreements)
 - [ ] I have run this code in development and it appears to resolve the stated issue
 - [ ] This PR includes tests, or tests are not required/relevant for this PR
-- [ ] I have updated the `CHANGELOG.md` and added information about my change to the "dbt-bigquery next" section.
+- [ ] I have run `changie new` to [create a changelog entry](https://github.com/dbt-labs/dbt-core/blob/main/CONTRIBUTING.md#Adding-CHANGELOG-Entry)
diff --git a/.github/workflows/bot-changelog.yml b/.github/workflows/bot-changelog.yml
new file mode 100644
index 000000000..2a141f849
--- /dev/null
+++ b/.github/workflows/bot-changelog.yml
@@ -0,0 +1,61 @@
+# **what?**
+# When bots create a PR, this action will add a corresponding changie yaml file to that
+# PR when a specific label is added.
+#
+# The file is created off a template:
+#
+# kind: 
+# body: 
+# time: 
+# custom:
+#   Author: 
+#   Issue: 4904
+#   PR: 
+#
+# **why?**
+# Automate changelog generation for more visability with automated bot PRs.
+#
+# **when?**
+# Once a PR is created, label should be added to PR before or after creation. You can also
+#  manually trigger this by adding the appropriate label at any time.
+#
+# **how to add another bot?**
+# Add the label and changie kind to the include matrix.  That's it!
+#
+
+name: Bot Changelog
+
+on:
+  pull_request:
+    # catch when the PR is opened with the label or when the label is added
+    types: [opened, labeled]
+
+permissions:
+  contents: write
+  pull-requests: read
+
+jobs:
+  generate_changelog:
+    strategy:
+      matrix:
+        include:
+          - label: "dependencies"
+            changie_kind: "Dependency"
+          - label: "snyk"
+            changie_kind: "Security"
+    runs-on: ubuntu-latest
+
+    steps:
+
+    - name: Create and commit changelog on bot PR
+      if: "contains(github.event.pull_request.labels.*.name, ${{ matrix.label }})"
+      id: bot_changelog
+      uses: emmyoop/changie_bot@v1.0
+      with:
+        GITHUB_TOKEN: ${{ secrets.FISHTOWN_BOT_PAT }}
+        commit_author_name: "Github Build Bot"
+        commit_author_email: ""
+        commit_message: "Add automated changelog yaml from template for bot PR"
+        changie_kind: ${{ matrix.changie_kind }}
+        label: ${{ matrix.label }}
+        custom_changelog_string: "custom:\n  Author: ${{ github.event.pull_request.user.login }}\n  Issue: 216\n  PR: ${{ github.event.pull_request.number }}\n"
diff --git a/.github/workflows/changelog-existence.yml b/.github/workflows/changelog-existence.yml
new file mode 100644
index 000000000..ea5e0dffd
--- /dev/null
+++ b/.github/workflows/changelog-existence.yml
@@ -0,0 +1,41 @@
+# **what?**
+# Checks that a file has been committed under the /.changes directory
+# as a new CHANGELOG entry.  Cannot check for a specific filename as
+# it is dynamically generated by change type and timestamp.
+# This workflow should not require any secrets since it runs for PRs
+# from forked repos.
+# By default, secrets are not passed to workflows running from
+# a forked repo.
+
+# **why?**
+# Ensure code change gets reflected in the CHANGELOG.
+
+# **when?**
+# This will run for all PRs going into main and *.latest.  It will
+# run when they are opened, reopened, when any label is added or removed
+# and when new code is pushed to the branch.  The action will then get
+# skipped if the 'Skip Changelog' label is present is any of the labels.
+
+name: Check Changelog Entry
+
+on:
+  pull_request:
+    types: [opened, reopened, labeled, unlabeled, synchronize]
+  workflow_dispatch:
+
+defaults:
+  run:
+    shell: bash
+
+permissions:
+  contents: read
+  pull-requests: write
+
+
+jobs:
+  changelog:
+    uses: dbt-labs/actions/.github/workflows/changelog-existence.yml@main
+    with:
+      changelog_comment: 'Thank you for your pull request! We could not find a changelog entry for this change. For details on how to document a change, see [the dbt-core contributing guide](https://github.com/dbt-labs/dbt-core/blob/main/CONTRIBUTING.md#adding-changelog-entry), and the [dbt-bigquery contriubuting guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.MD).'
+      skip_label: 'Skip Changelog'
+    secrets: inherit # this is only acceptable because we own the action we're calling
diff --git a/CHANGELOG.md b/CHANGELOG.md
index f6a823c3d..93ea208a1 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,137 +1,19 @@
-## dbt-bigquery 1.3.0b2 (Release TBD)
+# dbt-bigquery Changelog
 
-## dbt-bigquery 1.3.0b1 (July 29, 2020)
+- This file provides a full account of all changes to `dbt-bigquery`.
+- Changes are listed under the (pre)release in which they first appear. Subsequent releases include changes from previous releases.
+- "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
+- Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.3.0-b1 - August 04, 2022
 ### Features
-- Implement `create_schema` via SQL, instead of Python method, allowing users to override if desired. `drop_schema` remains a Python method for the time being. ([#182](https://github.com/dbt-labs/dbt-bigquery/issues/182), [#183](https://github.com/dbt-labs/dbt-bigquery/pull/183))
-- Added table and incrementail materializations for python models via DataProc. ([#209](https://github.com/dbt-labs/dbt-bigquery/pull/209), [#226](https://github.com/dbt-labs/dbt-bigquery/pull/226))
-
-### Under the hood
-- Implement minimal changes to support dbt Core incremental materialization refactor. ([#232](http://github.com/dbt-labs/dbt-bigquery/issues/232), [#223](https://github.com/dbt-labs/dbt-biquery/pull/223))
-
-## dbt-bigquery 1.2.0rc1 (June 12, 2022)
-
-### Features
-- Add grants to materializations ([#198](https://github.com/dbt-labs/dbt-bigquery/issues/198), [#212](https://github.com/dbt-labs/dbt-bigquery/pull/212))
-
-### Under the hood
-- Modify `BigQueryColumn.numeric_type` to always exclude precision + scale, since the functionality of ["parametrized data types on BigQuery](https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types#parameterized_data_types) is highly constrained ([#214](https://github.com/dbt-labs/dbt-bigquery/pull/214))
-
-## dbt-bigquery 1.2.0b1 (June 24, 2022)
-
-### Fixes
-- Incorrect parameter is passed to the bq_insert_overwrite macro call ([#172](https://github.com/dbt-labs/dbt-bigquery/pull/172))
-- Fix validate connection failing ([#168](https://github.com/dbt-labs/dbt-bigquery/pull/168))
-
-### Under the hood
-- Adding Python 3.10 testing and enabling mypy ([#177](https://github.com/dbt-labs/dbt-bigquery/pull/177))
-- Lift and shift cross-database macros from dbt-utils ([#192](https://github.com/dbt-labs/dbt-bigquery/pull/192)
-- Implement TestDocsGenerateBigQuery test ([#190](https://github.com/dbt-labs/dbt-bigquery/pull/190))
-- Init push for pytest conversion of override_database for bigquery ([#165](https://github.com/dbt-labs/dbt-bigquery/pull/165), [#188](https://github.com/dbt-labs/dbt-bigquery/pull/188))
-- Add oauth test target as default ([#166](https://github.com/dbt-labs/dbt-bigquery/pull/166))
-
-### Contributors
-- [@robomill](https://github.com/robomill) ([#172](https://github.com/dbt-labs/dbt-bigquery/pull/172))
-- [@dbeatty10](https://github.com/dbeatty10) ([#192](https://github.com/dbt-labs/dbt-bigquery/pull/192)
-
-## dbt-bigquery 1.1.0 (April 28, 2022)
-
-## dbt-bigquery 1.1.0rc2 (April 20, 2022)
-
-### Fixes
-- Restore default behavior for query timeout. Set default `job_execution_timeout` to `None` by default. Keep 300 seconds as query timeout where previously used.
-
-## dbt-bigquery 1.1.0rc1 (April 13, 2022)
-
-### Under the hood
-- Use dbt.tests.adapter.basic in tests (new test framework) ([#135](https://github.com/dbt-labs/dbt-bigquery/issues/135), [#142](https://github.com/dbt-labs/dbt-bigquery/pull/142))
-- Adding pre-commit and black formatter hooks ([#147](https://github.com/dbt-labs/dbt-bigquery/pull/147))
-- Adding pre-commit code changes ([#148](https://github.com/dbt-labs/dbt-bigquery/pull/148))
-
-## dbt-bigquery 1.1.0b1 (March 23, 2022)
-### Features
-- Provide a fine-grained control of the timeout and retry of BigQuery query with four new dbt profile configs: `job_creation_timeout_seconds`, `job_execution_timeout_seconds`, `job_retry_deadline_seconds`, and `job_retries` ([#45](https://github.com/dbt-labs/dbt-bigquery/issues/45), [#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
-- Adds new integration test to check against new ability to allow unique_key to be a list. [#112](https://github.com/dbt-labs/dbt-bigquery/issues/112)
-- Added upload_file macro to upload a local file to a table. [#102](https://github.com/dbt-labs/dbt-bigquery/issues/102)
-- Add logic to BigQueryConnectionManager to add fuctionality for UPDATE and SELECT statements. [#79](https://github.com/dbt-labs/dbt-bigquery/pull/79)
-
-### Fixes
-- Fix test related to preventing coercion of boolean values (True, False) to numeric values (0, 1) in query results ([#93](https://github.com/dbt-labs/dbt-bigquery/issues/93))
-- Add a check in `get_table_options` to check that the table has a `partition_by` in the config.
-This will prevent BigQuery from throwing an error since non-partitioned tables cannot have `require_partition_filter` ([#107](https://github.com/dbt-labs/dbt-bigquery/issues/107))
-- Ignore errors of the lack of permissions in `list_relations_without_caching` ([#104](https://github.com/dbt-labs/dbt-bigquery/issues/104))
-
-### Under the hood
-- Address BigQuery API deprecation warning and simplify usage of `TableReference` and `DatasetReference` objects ([#97](https://github.com/dbt-labs/dbt-bigquery/issues/97)),([#98](https://github.com/dbt-labs/dbt-bigquery/pull/98))
-- Add contributing.md file for adapter repo [#73](https://github.com/dbt-labs/dbt-bigquery/pull/73)
-- Add stale messaging workflow to Github Actions [#103](https://github.com/dbt-labs/dbt-bigquery/pull/103)
-- Add unique_key to go in parity with unique_key as a list chagnes made in core [#119](https://github.com/dbt-labs/dbt-bigquery/pull/119/files)
-- Adding new Enviornment variable for integration testing puproses [#116](https://github.com/dbt-labs/dbt-bigquery/pull/116)
-
-### Contributors
-- [@hui-zheng](https://github.com/hui-zheng)([#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
-- [@oliverrmaa](https://github.com/oliverrmaa)([#109](https://github.com/dbt-labs/dbt-bigquery/pull/109))
-- [@yu-iskw](https://github.com/yu-iskw)([#108](https://github.com/dbt-labs/dbt-bigquery/pull/108))
-- [@pgoslatara](https://github.com/pgoslatara) ([#66](https://github.com/dbt-labs/dbt-bigquery/pull/121))
-- [@drewmcdonald](https://github.com/drewmcdonald)([#98](https://github.com/dbt-labs/dbt-bigquery/pull/98))
-- [@rjh336](https://github.com/rjh336)([#79](https://github.com/dbt-labs/dbt-bigquery/pull/79))
-
-## dbt-bigquery 1.0.0 (December 3, 2021)
-
-## dbt-bigquery 1.0.0rc2 (November 24, 2021)
-
-### Features
-- Add optional `scopes` profile configuration argument to reduce the BigQuery OAuth scopes down to the minimal set needed. ([#23](https://github.com/dbt-labs/dbt-bigquery/issues/23), [#63](https://github.com/dbt-labs/dbt-bigquery/pull/63))
-
-### Fixes
-- Don't apply `require_partition_filter` to temporary tables, thereby fixing `insert_overwrite` strategy when partition filter is required ([#64](https://github.com/dbt-labs/dbt-bigquery/issues/64)), ([#65](https://github.com/dbt-labs/dbt-bigquery/pull/65))
-
-### Under the hood
-- Adding `execution_project` to `target` object ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))
-
-### Contributors
-- [@pgoslatara](https://github.com/pgoslatara) ([#66](https://github.com/dbt-labs/dbt-bigquery/issues/66))
-- [@bborysenko](https://github.com/bborysenko) ([#63](https://github.com/dbt-labs/dbt-bigquery/pull/63))
-- [@hui-zheng](https://github.com/hui-zheng)([#50](https://github.com/dbt-labs/dbt-bigquery/pull/50))
-- [@yu-iskw](https://github.com/yu-iskw) ([#65](https://github.com/dbt-labs/dbt-bigquery/pull/65))
-
-## dbt-bigquery 1.0.0rc1 (November 10, 2021)
-
-### Fixes
-- Fix problem with bytes processed return None value when the service account used to connect DBT in bigquery had a row policy access.
-([#47](https://github.com/dbt-labs/dbt-bigquery/issues/47), [#48](https://github.com/dbt-labs/dbt-bigquery/pull/48))
-- When on_schema_change is set, pass common columns as dest_columns in incremental merge macros ([#4144](https://github.com/dbt-labs/dbt-core/issues/4144))
-
-### Under the hood
-- Capping `google-api-core` to version `1.31.3` due to `protobuf` dependency conflict ([#53](https://github.com/dbt-labs/dbt-bigquery/pull/53))
-- Bump `google-cloud-core` and `google-api-core` upper bounds to `<3`, thereby removing `<1.31.3` limit on the latter. Remove explicit dependency on `six` ([#57](https://github.com/dbt-labs/dbt-bigquery/pull/57))
-- Remove official support for python 3.6, which is reaching end of life on December 23, 2021 ([dbt-core#4134](https://github.com/dbt-labs/dbt-core/issues/4134), [#59](https://github.com/dbt-labs/dbt-bigquery/pull/59))
-- Add support for structured logging [#55](https://github.com/dbt-labs/dbt-bigquery/pull/55)
-
-### Contributors
-- [@imartynetz](https://github.com/imartynetz) ([#48](https://github.com/dbt-labs/dbt-bigquery/pull/48))
-- [@Kayrnt](https://github.com/Kayrnt) ([#51](https://github.com/dbt-labs/dbt-bigquery/pull/51))
-
-## dbt-bigquery 1.0.0b2 (October 25, 2021)
-
-### Features
-
-- Rework `_dbt_max_partition` logic in dynamic `insert_overwrite` incremental strategy. Make the default logic compatible with `on_schema_change`, and make it possible to disable or reimplement that logic by defining a custom macro `declare_dbt_max_partition` ([#17](https://github.com/dbt-labs/dbt-bigquery/issues/17), [#39](https://github.com/dbt-labs/dbt-bigquery/issues/39), [#41](https://github.com/dbt-labs/dbt-bigquery/pull/41))
-
-### Fixes
-- Reimplement the `unique` test to handle column expressions and naming overlaps ([#33](https://github.com/dbt-labs/dbt-bigquery/issues/33), [#35](https://github.com/dbt-labs/dbt-bigquery/issues/35), [#10](https://github.com/dbt-labs/dbt-bigquery/pull/10))
-- Avoid error in `dbt deps` + `dbt clean` if default project is missing ([#27](https://github.com/dbt-labs/dbt-bigquery/issues/27), [#40](https://github.com/dbt-labs/dbt-bigquery/pull/40))
-
-### Under the hood
-- Replace `sample_profiles.yml` with `profile_template.yml`, for use with new `dbt init` ([#43](https://github.com/dbt-labs/dbt-bigquery/pull/43))
-
-### Contributors
-
-- [@DigUpTheHatchet](https://github.com/DigUpTheHatchet) ([#10](https://github.com/dbt-labs/dbt-bigquery/pull/10))
-- [@jeremyyeo](https://github.com/jeremyyeo) ([#40](https://github.com/dbt-labs/dbt-bigquery/pull/40))
-- [@NiallRees](https://github.com/NiallRees) ([#43](https://github.com/dbt-labs/dbt-bigquery/pull/43))
-
-## dbt-bigquery 1.0.0b1 (October 11, 2021)
-
-### Under the hood
-- Initial adapter split out
+- Implement `create_schema` via SQL, instead of Python method, allowing users to override if desired. drop_schema remains a Python method for the time being.  ([#182](https://github.com/dbt-labs/dbt-bigquery/issues/182), [#183](https://github.com/dbt-labs/dbt-bigquery/pull/183))
+- Added table and incrementail materializations for python models via DataProc. ([#209](https://github.com/dbt-labs/dbt-bigquery/issues/209), [#226](https://github.com/dbt-labs/dbt-bigquery/pull/226))
+### Under the Hood
+- Implement minimal changes to support dbt Core incremental materialization refactor. ([#232](https://github.com/dbt-labs/dbt-bigquery/issues/232), [#223](https://github.com/dbt-labs/dbt-bigquery/pull/223))
+
+## Previous Releases
+For information on prior major and minor releases, see their changelogs:
+- [1.2](https://github.com/dbt-labs/dbt-bigquery/blob/1.2.latest/CHANGELOG.md)
+- [1.1](https://github.com/dbt-labs/dbt-bigquery/blob/1.1.latest/CHANGELOG.md)
+- [1.0](https://github.com/dbt-labs/dbt-bigquery/blob/1.0.latest/CHANGELOG.md)
diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md
index aec184e46..9b765784e 100644
--- a/CONTRIBUTING.md
+++ b/CONTRIBUTING.md
@@ -89,6 +89,15 @@ Many changes will require and update to the `dbt-bigquery` docs here are some us
 - The changes made are likely to impact one or both of [BigQuery Profile](https://docs.getdbt.com/reference/warehouse-profiles/bigquery-profile), or [BigQuery Configs](https://docs.getdbt.com/reference/resource-configs/bigquery-configs).
 - We ask every community member who makes a user-facing change to open an issue or PR regarding doc changes.
 
+## Adding CHANGELOG Entry
+
+We use [changie](https://changie.dev) to generate `CHANGELOG` entries. **Note:** Do not edit the `CHANGELOG.md` directly. Your modifications will be lost.
+
+Follow the steps to [install `changie`](https://changie.dev/guide/installation/) for your system.
+
+Once changie is installed and your PR is created, simply run `changie new` and changie will walk you through the process of creating a changelog entry.  Commit the file that's created and your changelog entry is complete!
+
+You don't need to worry about which `dbt-bigquery` version your change will go into. Just create the changelog entry with `changie`, and open your PR against the `main` branch. All merged changes will be included in the next minor version of `dbt-bigquery`. The Core maintainers _may_ choose to "backport" specific changes in order to patch older minor versions. In that case, a maintainer will take care of that backport after merging your PR, before releasing the new version of `dbt-bigquery`.
 
 
 ## Submitting a Pull Request

From 764f5fe6370f9f7ed941a387104f57207dd562c6 Mon Sep 17 00:00:00 2001
From: Christophe Oudar 
Date: Mon, 8 Aug 2022 16:50:56 +0200
Subject: [PATCH 468/860] Add location/job_id/project_id to adapter response to
 enable easy job linking (#250)

---
 .../Under the Hood-20220806-142912.yaml       |  7 +++
 dbt/adapters/bigquery/connections.py          | 56 +++++++++++++------
 .../test_bigquery_adapter_functions.py        |  6 ++
 3 files changed, 53 insertions(+), 16 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20220806-142912.yaml

diff --git a/.changes/unreleased/Under the Hood-20220806-142912.yaml b/.changes/unreleased/Under the Hood-20220806-142912.yaml
new file mode 100644
index 000000000..6581f8e22
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20220806-142912.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: Add location/job_id/project_id to adapter response to enable easy job linking
+time: 2022-08-06T14:29:12.271054+02:00
+custom:
+  Author: Kayrnt
+  Issue: "92"
+  PR: "250"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index da7b0f687..502a7b9dd 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -86,6 +86,9 @@ class BigQueryConnectionMethod(StrEnum):
 @dataclass
 class BigQueryAdapterResponse(AdapterResponse):
     bytes_processed: Optional[int] = None
+    location: Optional[str] = None
+    project_id: Optional[str] = None
+    job_id: Optional[str] = None
 
 
 @dataclass
@@ -188,6 +191,12 @@ class BigQueryConnectionManager(BaseConnectionManager):
     @classmethod
     def handle_error(cls, error, message):
         error_msg = "\n".join([item["message"] for item in error.errors])
+        if hasattr(error, "query_job"):
+            logger.error(
+                cls._bq_job_link(
+                    error.query_job.location, error.query_job.project, error.query_job.job_id
+                )
+            )
         raise DatabaseException(error_msg)
 
     def clear_transaction(self):
@@ -446,55 +455,70 @@ def execute(
         code = None
         num_rows = None
         bytes_processed = None
+        location = None
+        job_id = None
+        project_id = None
+        num_rows_formatted = None
+        processed_bytes = None
 
         if query_job.statement_type == "CREATE_VIEW":
             code = "CREATE VIEW"
 
         elif query_job.statement_type == "CREATE_TABLE_AS_SELECT":
+            code = "CREATE TABLE"
             conn = self.get_thread_connection()
             client = conn.handle
             query_table = client.get_table(query_job.destination)
-            code = "CREATE TABLE"
             num_rows = query_table.num_rows
-            num_rows_formated = self.format_rows_number(num_rows)
-            bytes_processed = query_job.total_bytes_processed
-            processed_bytes = self.format_bytes(bytes_processed)
-            message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
 
         elif query_job.statement_type == "SCRIPT":
             code = "SCRIPT"
-            bytes_processed = query_job.total_bytes_processed
-            message = f"{code} ({self.format_bytes(bytes_processed)} processed)"
 
         elif query_job.statement_type in ["INSERT", "DELETE", "MERGE", "UPDATE"]:
             code = query_job.statement_type
             num_rows = query_job.num_dml_affected_rows
-            num_rows_formated = self.format_rows_number(num_rows)
-            bytes_processed = query_job.total_bytes_processed
-            processed_bytes = self.format_bytes(bytes_processed)
-            message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
 
         elif query_job.statement_type == "SELECT":
+            code = "SELECT"
             conn = self.get_thread_connection()
             client = conn.handle
             # use anonymous table for num_rows
             query_table = client.get_table(query_job.destination)
-            code = "SELECT"
             num_rows = query_table.num_rows
-            num_rows_formated = self.format_rows_number(num_rows)
-            bytes_processed = query_job.total_bytes_processed
-            processed_bytes = self.format_bytes(bytes_processed)
-            message = f"{code} ({num_rows_formated} rows, {processed_bytes} processed)"
+
+        # set common attributes
+        bytes_processed = query_job.total_bytes_processed
+        processed_bytes = self.format_bytes(bytes_processed)
+        location = query_job.location
+        job_id = query_job.job_id
+        project_id = query_job.project
+        if num_rows is not None:
+            num_rows_formatted = self.format_rows_number(num_rows)
+            message = f"{code} ({num_rows_formatted} rows, {processed_bytes} processed)"
+        elif bytes_processed is not None:
+            message = f"{code} ({processed_bytes} processed)"
+        else:
+            message = f"{code}"
+
+        if location is not None and job_id is not None and project_id is not None:
+            logger.debug(self._bq_job_link(job_id, project_id, location))
 
         response = BigQueryAdapterResponse(  # type: ignore[call-arg]
             _message=message,
             rows_affected=num_rows,
             code=code,
             bytes_processed=bytes_processed,
+            location=location,
+            project_id=project_id,
+            job_id=job_id,
         )
 
         return response, table
 
+    @staticmethod
+    def _bq_job_link(location, project_id, job_id) -> str:
+        return f"https://console.cloud.google.com/bigquery?project={project_id}&j=bq:{location}:{job_id}&page=queryresults"
+
     def get_partitions_metadata(self, table):
         def standard_to_legacy(table):
             return table.project + ":" + table.dataset + "." + table.identifier
diff --git a/tests/integration/bigquery_test/test_bigquery_adapter_functions.py b/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
index 06e230f30..8ad4a27fc 100644
--- a/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
+++ b/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
@@ -21,6 +21,12 @@ def test__bigquery_adapter_functions(self):
         results = self.run_dbt()
         self.assertEqual(len(results), 3)
 
+        for result in results:
+            # all queries in adapter models are jobs that are expected to have a location/project_id/job_id
+            assert result.adapter_response["location"] is not None
+            assert result.adapter_response["project_id"] is not None
+            assert result.adapter_response["job_id"] is not None
+
         test_results = self.run_dbt(['test'])
 
         self.assertTrue(len(test_results) > 0)

From 9c36aa239fb674faebfadab0a2b6a38cd201da86 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Mon, 8 Aug 2022 16:01:14 -0500
Subject: [PATCH 469/860] minor fixes for changie  (#261)

* init pr for changie swap over in dbt-bigquery

* typo fix

* convert 1.3 changelog including batch and merge

* changelog entry

* typo fix and change of ref in .changes readme

* Empty-Commit

* update to main, fix eof issue
---
 .changes/README.md                        | 2 +-
 .github/pull_request_template.md          | 2 +-
 .github/workflows/changelog-existence.yml | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/.changes/README.md b/.changes/README.md
index d82835028..c8c37e3f5 100644
--- a/.changes/README.md
+++ b/.changes/README.md
@@ -1,3 +1,3 @@
 # CHANGELOG
 
-To view information about the changelog operation we suggest reading this [README](https://github.com/dbt-labs/dbt-core/blob/main/.changes/README.md) found in `dbt-core`.
+To view information about the changelog operation we suggest reading this [README](https://github.com/dbt-labs/dbt-bigquery/blob/main/.changes/README.md) found in `dbt-bigquery`.
diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md
index e15849215..5d8734bd5 100644
--- a/.github/pull_request_template.md
+++ b/.github/pull_request_template.md
@@ -18,4 +18,4 @@ resolves #
 - [ ] I have signed the [CLA](https://docs.getdbt.com/docs/contributor-license-agreements)
 - [ ] I have run this code in development and it appears to resolve the stated issue
 - [ ] This PR includes tests, or tests are not required/relevant for this PR
-- [ ] I have run `changie new` to [create a changelog entry](https://github.com/dbt-labs/dbt-core/blob/main/CONTRIBUTING.md#Adding-CHANGELOG-Entry)
+- [ ] I have run `changie new` to [create a changelog entry](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#Adding-CHANGELOG-Entry)
diff --git a/.github/workflows/changelog-existence.yml b/.github/workflows/changelog-existence.yml
index ea5e0dffd..1284bfe46 100644
--- a/.github/workflows/changelog-existence.yml
+++ b/.github/workflows/changelog-existence.yml
@@ -36,6 +36,6 @@ jobs:
   changelog:
     uses: dbt-labs/actions/.github/workflows/changelog-existence.yml@main
     with:
-      changelog_comment: 'Thank you for your pull request! We could not find a changelog entry for this change. For details on how to document a change, see [the dbt-core contributing guide](https://github.com/dbt-labs/dbt-core/blob/main/CONTRIBUTING.md#adding-changelog-entry), and the [dbt-bigquery contriubuting guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.MD).'
+      changelog_comment: 'Thank you for your pull request! We could not find a changelog entry for this change. For details on how to document a change, see the [dbt-bigquery contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.MD).'
       skip_label: 'Skip Changelog'
     secrets: inherit # this is only acceptable because we own the action we're calling

From 2c4041752d556b57f0e15f8711c950a7c296f0b9 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Tue, 9 Aug 2022 15:35:06 -0500
Subject: [PATCH 470/860] Change no issue reference in bot-changelog.yml and
 add condional logic for kinds (#262)

* init pr for changie swap over in dbt-bigquery

* typo fix

* convert 1.3 changelog including batch and merge

* changelog entry

* typo fix and change of ref in .changes readme

* Empty-Commit

* update to main, fix eof issue

* change ref for no issue issue

* updating to main

* add conditional logic for dpendency and security kinds
---
 .changie.yaml                       | 2 ++
 .github/workflows/bot-changelog.yml | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)

diff --git a/.changie.yaml b/.changie.yaml
index 4a58d32e0..88670f09c 100644
--- a/.changie.yaml
+++ b/.changie.yaml
@@ -13,7 +13,9 @@ kinds:
 - label: Fixes
 - label: Under the Hood
 - label: Dependencies
+  changeFormat: '- {{.Body}} ({{if ne .Custom.Issue ""}}[#{{.Custom.Issue}}](https://github.com/dbt-labs/dbt-bigquery/issues/{{.Custom.Issue}}), {{end}}[#{{.Custom.PR}}](https://github.com/dbt-labs/dbt-bigquery/pull/{{.Custom.PR}}))'
 - label: Security
+  changeFormat: '- {{.Body}} ({{if ne .Custom.Issue ""}}[#{{.Custom.Issue}}](https://github.com/dbt-labs/dbt-bigquery/issues/{{.Custom.Issue}}), {{end}}[#{{.Custom.PR}}](https://github.com/dbt-labs/dbt-bigquery/pull/{{.Custom.PR}}))'
 custom:
 - key: Author
   label: GitHub Username(s) (separated by a single space if multiple)
diff --git a/.github/workflows/bot-changelog.yml b/.github/workflows/bot-changelog.yml
index 2a141f849..2ac7091bf 100644
--- a/.github/workflows/bot-changelog.yml
+++ b/.github/workflows/bot-changelog.yml
@@ -58,4 +58,4 @@ jobs:
         commit_message: "Add automated changelog yaml from template for bot PR"
         changie_kind: ${{ matrix.changie_kind }}
         label: ${{ matrix.label }}
-        custom_changelog_string: "custom:\n  Author: ${{ github.event.pull_request.user.login }}\n  Issue: 216\n  PR: ${{ github.event.pull_request.number }}\n"
+        custom_changelog_string: "custom:\n  Author: ${{ github.event.pull_request.user.login }}\n  Issue: 254\n  PR: ${{ github.event.pull_request.number }}\n"

From 540bd7cb0708af980e75b0d1a55ef6fe2d072837 Mon Sep 17 00:00:00 2001
From: Yu Ishikawa 
Date: Fri, 19 Aug 2022 23:29:14 +0900
Subject: [PATCH 471/860] Fix a typo in `changelog-existence.md` (#276)

---
 .github/workflows/changelog-existence.yml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/.github/workflows/changelog-existence.yml b/.github/workflows/changelog-existence.yml
index 1284bfe46..02bce4582 100644
--- a/.github/workflows/changelog-existence.yml
+++ b/.github/workflows/changelog-existence.yml
@@ -36,6 +36,6 @@ jobs:
   changelog:
     uses: dbt-labs/actions/.github/workflows/changelog-existence.yml@main
     with:
-      changelog_comment: 'Thank you for your pull request! We could not find a changelog entry for this change. For details on how to document a change, see the [dbt-bigquery contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.MD).'
+      changelog_comment: 'Thank you for your pull request! We could not find a changelog entry for this change. For details on how to document a change, see the [dbt-bigquery contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md).'
       skip_label: 'Skip Changelog'
     secrets: inherit # this is only acceptable because we own the action we're calling

From 56a95e27d9b249c4f3d3a2e75c719306c1da5fb3 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 24 Aug 2022 09:40:03 -0500
Subject: [PATCH 472/860] version bump, changie. and backports (#282)

---
 .github/workflows/backport.yml     | 40 +++++++++++++
 .github/workflows/version-bump.yml | 96 ++++--------------------------
 2 files changed, 51 insertions(+), 85 deletions(-)
 create mode 100644 .github/workflows/backport.yml

diff --git a/.github/workflows/backport.yml b/.github/workflows/backport.yml
new file mode 100644
index 000000000..d5c7fffed
--- /dev/null
+++ b/.github/workflows/backport.yml
@@ -0,0 +1,40 @@
+# **what?**
+# When a PR is merged, if it has the backport label, it will create
+# a new PR to backport those changes to the given branch. If it can't
+# cleanly do a backport, it will comment on the merged PR of the failure.
+#
+# Label naming convention: "backport "
+# Example: backport 1.0.latest
+#
+# You MUST "Squash and merge" the original PR or this won't work.
+
+# **why?**
+# Changes sometimes need to be backported to release branches.
+# This automates the backporting process
+
+# **when?**
+# Once a PR is "Squash and merge"'d, by adding a backport label, this is triggered
+
+name: Backport
+on:
+  pull_request:
+    types:
+      - labeled
+
+permissions:
+  contents: write
+  pull-requests: write
+
+jobs:
+  backport:
+    name: Backport
+    runs-on: ubuntu-latest
+    # Only react to merged PRs for security reasons.
+    # See https://docs.github.com/en/actions/using-workflows/events-that-trigger-workflows#pull_request_target.
+    if: >
+      github.event.pull_request.merged
+      && contains(github.event.label.name, 'backport')
+    steps:
+      - uses: tibdex/backport@v2.0.2
+        with:
+          github_token: ${{ secrets.GITHUB_TOKEN }}
diff --git a/.github/workflows/version-bump.yml b/.github/workflows/version-bump.yml
index d9df91c52..bde34d683 100644
--- a/.github/workflows/version-bump.yml
+++ b/.github/workflows/version-bump.yml
@@ -1,18 +1,15 @@
 # **what?**
-# This workflow will take a version number and a dry run flag. With that
+# This workflow will take the new version number to bump to. With that
 # it will run versionbump to update the version number everywhere in the
-# code base and then generate an update Docker requirements file. If this
-# is a dry run, a draft PR will open with the changes. If this isn't a dry
-# run, the changes will be committed to the branch this is run on.
+# code base and then run changie to create the corresponding changelog.
+# A PR will be created with the changes that can be reviewed before committing.
 
 # **why?**
 # This is to aid in releasing dbt and making sure we have updated
-# the versions and Docker requirements in all places.
+# the version in all places and generated the changelog.
 
 # **when?**
-# This is triggered either manually OR
-# from the repository_dispatch event "version-bump" which is sent from
-# the dbt-release repo Action
+# This is triggered manually
 
 name: Version Bump
 
@@ -20,83 +17,12 @@ on:
   workflow_dispatch:
     inputs:
       version_number:
-       description: 'The version number to bump to'
+       description: 'The version number to bump to (ex. 1.2.0, 1.3.0b1)'
        required: true
-      is_dry_run:
-       description: 'Creates a draft PR to allow testing instead of committing to a branch'
-       required: true
-       default: 'true'
-  repository_dispatch:
-    types: [version-bump]
 
 jobs:
-  bump:
-    runs-on: ubuntu-latest
-    steps:
-      - name: Check out the repository
-        uses: actions/checkout@v2
-
-      - name: Set version and dry run values
-        id: variables
-        env:
-          VERSION_NUMBER: "${{ github.event.client_payload.version_number == '' && github.event.inputs.version_number || github.event.client_payload.version_number }}"
-          IS_DRY_RUN: "${{ github.event.client_payload.is_dry_run == '' && github.event.inputs.is_dry_run || github.event.client_payload.is_dry_run }}"
-        run: |
-          echo Repository dispatch event version: ${{ github.event.client_payload.version_number }}
-          echo Repository dispatch event dry run: ${{ github.event.client_payload.is_dry_run }}
-          echo Workflow dispatch event version: ${{ github.event.inputs.version_number }}
-          echo Workflow dispatch event dry run: ${{ github.event.inputs.is_dry_run }}
-          echo ::set-output name=VERSION_NUMBER::$VERSION_NUMBER
-          echo ::set-output name=IS_DRY_RUN::$IS_DRY_RUN
-
-      - uses: actions/setup-python@v2
-        with:
-          python-version: "3.8"
-
-      - name: Install python dependencies
-        run: |
-          python3 -m venv env
-          source env/bin/activate
-          python -m pip install --upgrade pip
-
-      - name: Create PR branch
-        if: ${{ steps.variables.outputs.IS_DRY_RUN  == 'true' }}
-        run: |
-          git checkout -b bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_$GITHUB_RUN_ID
-          git push origin bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_$GITHUB_RUN_ID
-          git branch --set-upstream-to=origin/bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_$GITHUB_RUN_ID bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_$GITHUB_RUN_ID
-
-      - name: Bumping version
-        run: |
-          source env/bin/activate
-          python -m pip install -r dev-requirements.txt
-          env/bin/bumpversion --allow-dirty --new-version ${{steps.variables.outputs.VERSION_NUMBER}} major
-          git status
-
-      - name: Commit version bump directly
-        uses: EndBug/add-and-commit@v7
-        if: ${{ steps.variables.outputs.IS_DRY_RUN == 'false' }}
-        with:
-          author_name: 'Github Build Bot'
-          author_email: 'buildbot@fishtownanalytics.com'
-          message: 'Bumping version to ${{steps.variables.outputs.VERSION_NUMBER}}'
-
-      - name: Commit version bump to branch
-        uses: EndBug/add-and-commit@v7
-        if: ${{ steps.variables.outputs.IS_DRY_RUN == 'true' }}
-        with:
-          author_name: 'Github Build Bot'
-          author_email: 'buildbot@fishtownanalytics.com'
-          message: 'Bumping version to ${{steps.variables.outputs.VERSION_NUMBER}}'
-          branch: 'bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_${{GITHUB.RUN_ID}}'
-          push: 'origin origin/bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_${{GITHUB.RUN_ID}}'
-
-      - name: Create Pull Request
-        uses: peter-evans/create-pull-request@v3
-        if: ${{ steps.variables.outputs.IS_DRY_RUN == 'true' }}
-        with:
-          author: 'Github Build Bot '
-          draft: true
-          base: ${{github.ref}}
-          title: 'Bumping version to ${{steps.variables.outputs.VERSION_NUMBER}}'
-          branch: 'bumping-version/${{steps.variables.outputs.VERSION_NUMBER}}_${{GITHUB.RUN_ID}}'
+  version_bump_and_changie:
+    uses: dbt-labs/actions/.github/workflows/version-bump.yml@main
+    with:
+      version_number: ${{ inputs.version_number }}
+    secrets: inherit  # ok since what we are calling is internally maintained

From 8b47eb8efe69b114b3c63f05dd1f7cf470954204 Mon Sep 17 00:00:00 2001
From: Chenyu Li 
Date: Thu, 25 Aug 2022 09:18:23 -0700
Subject: [PATCH 473/860] add supported language (#290)

* add supported language

* add changelog
---
 .changes/unreleased/Under the Hood-20220825-073235.yaml    | 7 +++++++
 .../bigquery/macros/materializations/incremental.sql       | 2 +-
 dbt/include/bigquery/macros/materializations/table.sql     | 2 +-
 3 files changed, 9 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20220825-073235.yaml

diff --git a/.changes/unreleased/Under the Hood-20220825-073235.yaml b/.changes/unreleased/Under the Hood-20220825-073235.yaml
new file mode 100644
index 000000000..f33d77173
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20220825-073235.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: specify supported_languages for materialization that support python models
+time: 2022-08-25T07:32:35.820396-07:00
+custom:
+  Author: ChenyuLInx
+  Issue: "288"
+  PR: "290"
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index c699e816e..a594e993e 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -136,7 +136,7 @@
 
 {% endmacro %}
 
-{% materialization incremental, adapter='bigquery' -%}
+{% materialization incremental, adapter='bigquery', supported_languages=['sql', 'python'] -%}
 
   {%- set unique_key = config.get('unique_key') -%}
   {%- set full_refresh_mode = (should_full_refresh()) -%}
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index 886ba3fa9..75751f331 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -1,4 +1,4 @@
-{% materialization table, adapter='bigquery' -%}
+{% materialization table, adapter='bigquery', supported_languages=['sql', 'python']-%}
 
   {%- set language = model['language'] -%}
   {%- set identifier = model['alias'] -%}

From ad885cd6db8b3a26278cb16d452038b061aa88e8 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 25 Aug 2022 15:51:23 -0500
Subject: [PATCH 474/860] Add `slot_ms` to `BigQueryAdapterResponse` (#271)

* Add `slot_millis` to `BigQueryAdapterResponse`

* Update CHANGELOG.md

* Rename `slot_millis` to `slot_ms`

* Rename the tile of the pull request

* add assert check of slot_ms to test__bigquery_adapter_functions

Co-authored-by: Yu ISHIKAWA 
---
 .changes/unreleased/Under the Hood-20220817-154151.yaml    | 7 +++++++
 dbt/adapters/bigquery/connections.py                       | 4 ++++
 .../bigquery_test/test_bigquery_adapter_functions.py       | 1 +
 3 files changed, 12 insertions(+)
 create mode 100644 .changes/unreleased/Under the Hood-20220817-154151.yaml

diff --git a/.changes/unreleased/Under the Hood-20220817-154151.yaml b/.changes/unreleased/Under the Hood-20220817-154151.yaml
new file mode 100644
index 000000000..d3de5a992
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20220817-154151.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: Adding `slot_ms` go `BigQueryAdapterResponse`
+time: 2022-08-17T15:41:51.080936-05:00
+custom:
+  Author: yu-iskw
+  Issue: "194"
+  PR: "195"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 502a7b9dd..50437622b 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -89,6 +89,7 @@ class BigQueryAdapterResponse(AdapterResponse):
     location: Optional[str] = None
     project_id: Optional[str] = None
     job_id: Optional[str] = None
+    slot_ms: Optional[int] = None
 
 
 @dataclass
@@ -460,6 +461,7 @@ def execute(
         project_id = None
         num_rows_formatted = None
         processed_bytes = None
+        slot_ms = None
 
         if query_job.statement_type == "CREATE_VIEW":
             code = "CREATE VIEW"
@@ -488,6 +490,7 @@ def execute(
 
         # set common attributes
         bytes_processed = query_job.total_bytes_processed
+        slot_ms = query_job.slot_millis
         processed_bytes = self.format_bytes(bytes_processed)
         location = query_job.location
         job_id = query_job.job_id
@@ -511,6 +514,7 @@ def execute(
             location=location,
             project_id=project_id,
             job_id=job_id,
+            slot_ms=slot_ms,
         )
 
         return response, table
diff --git a/tests/integration/bigquery_test/test_bigquery_adapter_functions.py b/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
index 8ad4a27fc..427470f1f 100644
--- a/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
+++ b/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
@@ -26,6 +26,7 @@ def test__bigquery_adapter_functions(self):
             assert result.adapter_response["location"] is not None
             assert result.adapter_response["project_id"] is not None
             assert result.adapter_response["job_id"] is not None
+            assert result.adapter_response["slot_ms"] is not None
 
         test_results = self.run_dbt(['test'])
 

From 170c65e08250017e1413dfb3f14df4bcbb11c113 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 25 Aug 2022 23:34:12 -0500
Subject: [PATCH 475/860] expanding test env example (#120)

* expanding test env example

* minor changes after mila review

* minor wording changes

* updating

* update after feedback from @VersusFacit

* updating based on suggestions from @kwigley

* updating test.env.example file

* reformat descriptions

* changing : to = in example so that users can c/p without making any changes
---
 test.env.example | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)

diff --git a/test.env.example b/test.env.example
index d49649307..58893f5af 100644
--- a/test.env.example
+++ b/test.env.example
@@ -1,6 +1,12 @@
-BIGQUERY_TEST_ALT_DATABASE=
+# Note: These values will come from your BigQuery account and GCP projects.
+
+# Test Environment field definitions
+# Name of a GCP project you don't have access to query.
 BIGQUERY_TEST_NO_ACCESS_DATABASE=
+# Authentication method required to hookup to BigQuery via client library.
 BIGQUERY_TEST_SERVICE_ACCOUNT_JSON='{}'
+
+# tests for local ci/cd
 DBT_TEST_USER_1="group:buildbot@dbtlabs.com"
 DBT_TEST_USER_2="group:dev-core@dbtlabs.com"
 DBT_TEST_USER_3="serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserviceaccount.com"

From 597aaa4667320a2f0e21abdf6ac10d523757183e Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Mon, 29 Aug 2022 17:05:49 -0400
Subject: [PATCH 476/860] Bumping version to 1.3.0b2 and generate CHANGELOG
 (#292)

Co-authored-by: Github Build Bot 
---
 .bumpversion.cfg                                       |  2 +-
 .changes/1.3.0-b2.md                                   | 10 ++++++++++
 .../Under the Hood-20220804-155719.yaml                |  0
 .../Under the Hood-20220806-142912.yaml                |  0
 .../Under the Hood-20220817-154151.yaml                |  0
 .../Under the Hood-20220825-073235.yaml                |  0
 CHANGELOG.md                                           | 10 +++++++++-
 dbt/adapters/bigquery/__version__.py                   |  2 +-
 setup.py                                               |  2 +-
 9 files changed, 22 insertions(+), 4 deletions(-)
 create mode 100644 .changes/1.3.0-b2.md
 rename .changes/{unreleased => 1.3.0}/Under the Hood-20220804-155719.yaml (100%)
 rename .changes/{unreleased => 1.3.0}/Under the Hood-20220806-142912.yaml (100%)
 rename .changes/{unreleased => 1.3.0}/Under the Hood-20220817-154151.yaml (100%)
 rename .changes/{unreleased => 1.3.0}/Under the Hood-20220825-073235.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index f0da053eb..d3aa82250 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.3.0b1
+current_version = 1.3.0b2
 parse = (?P\d+)
 	\.(?P\d+)
 	\.(?P\d+)
diff --git a/.changes/1.3.0-b2.md b/.changes/1.3.0-b2.md
new file mode 100644
index 000000000..a62c278cc
--- /dev/null
+++ b/.changes/1.3.0-b2.md
@@ -0,0 +1,10 @@
+## dbt-bigquery 1.3.0-b2 - August 29, 2022
+### Under the Hood
+- Add changie to dbt-bigquery ([#254](https://github.com/dbt-labs/dbt-bigquery/issues/254), [#253](https://github.com/dbt-labs/dbt-bigquery/pull/253))
+- Add location/job_id/project_id to adapter response to enable easy job linking ([#92](https://github.com/dbt-labs/dbt-bigquery/issues/92), [#250](https://github.com/dbt-labs/dbt-bigquery/pull/250))
+- Adding `slot_ms` go `BigQueryAdapterResponse` ([#194](https://github.com/dbt-labs/dbt-bigquery/issues/194), [#195](https://github.com/dbt-labs/dbt-bigquery/pull/195))
+- specify supported_languages for materialization that support python models ([#288](https://github.com/dbt-labs/dbt-bigquery/issues/288), [#290](https://github.com/dbt-labs/dbt-bigquery/pull/290))
+
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#250](https://github.com/dbt-labs/dbt-bigquery/pull/250))
+- [@yu-iskw](https://github.com/yu-iskw) ([#195](https://github.com/dbt-labs/dbt-bigquery/pull/195))
diff --git a/.changes/unreleased/Under the Hood-20220804-155719.yaml b/.changes/1.3.0/Under the Hood-20220804-155719.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20220804-155719.yaml
rename to .changes/1.3.0/Under the Hood-20220804-155719.yaml
diff --git a/.changes/unreleased/Under the Hood-20220806-142912.yaml b/.changes/1.3.0/Under the Hood-20220806-142912.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20220806-142912.yaml
rename to .changes/1.3.0/Under the Hood-20220806-142912.yaml
diff --git a/.changes/unreleased/Under the Hood-20220817-154151.yaml b/.changes/1.3.0/Under the Hood-20220817-154151.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20220817-154151.yaml
rename to .changes/1.3.0/Under the Hood-20220817-154151.yaml
diff --git a/.changes/unreleased/Under the Hood-20220825-073235.yaml b/.changes/1.3.0/Under the Hood-20220825-073235.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20220825-073235.yaml
rename to .changes/1.3.0/Under the Hood-20220825-073235.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 93ea208a1..a9038a76d 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -4,14 +4,22 @@
 - Changes are listed under the (pre)release in which they first appear. Subsequent releases include changes from previous releases.
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
+## dbt-bigquery 1.3.0-b2 - August 29, 2022
+### Under the Hood
+- Add changie to dbt-bigquery ([#254](https://github.com/dbt-labs/dbt-bigquery/issues/254), [#253](https://github.com/dbt-labs/dbt-bigquery/pull/253))
+- Add location/job_id/project_id to adapter response to enable easy job linking ([#92](https://github.com/dbt-labs/dbt-bigquery/issues/92), [#250](https://github.com/dbt-labs/dbt-bigquery/pull/250))
+- Adding `slot_ms` go `BigQueryAdapterResponse` ([#194](https://github.com/dbt-labs/dbt-bigquery/issues/194), [#195](https://github.com/dbt-labs/dbt-bigquery/pull/195))
+- specify supported_languages for materialization that support python models ([#288](https://github.com/dbt-labs/dbt-bigquery/issues/288), [#290](https://github.com/dbt-labs/dbt-bigquery/pull/290))
 
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#250](https://github.com/dbt-labs/dbt-bigquery/pull/250))
+- [@yu-iskw](https://github.com/yu-iskw) ([#195](https://github.com/dbt-labs/dbt-bigquery/pull/195))
 ## dbt-bigquery 1.3.0-b1 - August 04, 2022
 ### Features
 - Implement `create_schema` via SQL, instead of Python method, allowing users to override if desired. drop_schema remains a Python method for the time being.  ([#182](https://github.com/dbt-labs/dbt-bigquery/issues/182), [#183](https://github.com/dbt-labs/dbt-bigquery/pull/183))
 - Added table and incrementail materializations for python models via DataProc. ([#209](https://github.com/dbt-labs/dbt-bigquery/issues/209), [#226](https://github.com/dbt-labs/dbt-bigquery/pull/226))
 ### Under the Hood
 - Implement minimal changes to support dbt Core incremental materialization refactor. ([#232](https://github.com/dbt-labs/dbt-bigquery/issues/232), [#223](https://github.com/dbt-labs/dbt-bigquery/pull/223))
-
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 - [1.2](https://github.com/dbt-labs/dbt-bigquery/blob/1.2.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 4b49b750d..e2c1a233c 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.3.0b1"
+version = "1.3.0b2"
diff --git a/setup.py b/setup.py
index 175fa2f1a..69919fe1b 100644
--- a/setup.py
+++ b/setup.py
@@ -50,7 +50,7 @@ def _get_dbt_core_version():
 
 
 package_name = "dbt-bigquery"
-package_version = "1.3.0b1"
+package_version = "1.3.0b2"
 dbt_core_version = _get_dbt_core_version()
 description = """The BigQuery adapter plugin for dbt"""
 

From 53f8b907915cf0d45a4265d08c655b1568014792 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 2 Sep 2022 11:36:13 -0400
Subject: [PATCH 477/860] Bump black from 22.6.0 to 22.8.0 (#294)

* Bump black from 22.6.0 to 22.8.0

Bumps [black](https://github.com/psf/black) from 22.6.0 to 22.8.0.
- [Release notes](https://github.com/psf/black/releases)
- [Changelog](https://github.com/psf/black/blob/main/CHANGES.md)
- [Commits](https://github.com/psf/black/compare/22.6.0...22.8.0)

---
updated-dependencies:
- dependency-name: black
  dependency-type: direct:development
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] 

* Add automated changelog yaml from template for bot PR

* Remove newline

Signed-off-by: dependabot[bot] 
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot 
Co-authored-by: leahwicz <60146280+leahwicz@users.noreply.github.com>
---
 .changes/unreleased/Dependency-20220902-152810.yaml | 7 +++++++
 dev-requirements.txt                                | 2 +-
 2 files changed, 8 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependency-20220902-152810.yaml

diff --git a/.changes/unreleased/Dependency-20220902-152810.yaml b/.changes/unreleased/Dependency-20220902-152810.yaml
new file mode 100644
index 000000000..2d7fb11ef
--- /dev/null
+++ b/.changes/unreleased/Dependency-20220902-152810.yaml
@@ -0,0 +1,7 @@
+kind: Dependency
+body: "Bump black from 22.6.0 to 22.8.0"
+time: 2022-09-02T15:28:10.00000Z
+custom:
+  Author: dependabot[bot]
+  Issue: 254
+  PR: 294
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 5ff54ac51..129dbbe64 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -3,7 +3,7 @@
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
 
-black==22.6.0
+black==22.8.0
 bumpversion
 flake8
 flaky

From 84c20fcef86d9777b6ce6a4745b6415ec288ed93 Mon Sep 17 00:00:00 2001
From: Chenyu Li 
Date: Mon, 12 Sep 2022 11:33:56 -0700
Subject: [PATCH 478/860] Enhancement/enable serverless (#303)

* Experiment with Dataproc Serverless

* add serverless as another submission method

* add changelog and run tests against target core branch

* fix syntax

* fix schema overwrite

* use 0.21 version of connector

Co-authored-by: Jeremy Cohen 
---
 .../unreleased/Features-20220909-122924.yaml  |   7 +
 dbt/adapters/bigquery/impl.py                 | 128 +++------------
 dbt/adapters/bigquery/python_submissions.py   | 152 ++++++++++++++++++
 .../macros/materializations/table.sql         |   2 +-
 dev-requirements.txt                          |   4 +-
 tests/conftest.py                             |   2 +-
 tests/functional/adapter/test_python_model.py |   2 +-
 7 files changed, 187 insertions(+), 110 deletions(-)
 create mode 100644 .changes/unreleased/Features-20220909-122924.yaml
 create mode 100644 dbt/adapters/bigquery/python_submissions.py

diff --git a/.changes/unreleased/Features-20220909-122924.yaml b/.changes/unreleased/Features-20220909-122924.yaml
new file mode 100644
index 000000000..cde9bbb43
--- /dev/null
+++ b/.changes/unreleased/Features-20220909-122924.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: Add support for Dataproc Serverless
+time: 2022-09-09T12:29:24.993388-07:00
+custom:
+  Author: ChenyuLInx
+  Issue: "248"
+  PR: "303"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index dde6f865d..602aafd19 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -1,5 +1,5 @@
 from dataclasses import dataclass
-from typing import Dict, List, Optional, Any, Set, Union
+from typing import Dict, List, Optional, Any, Set, Union, Type
 from dbt.dataclass_schema import dbtClassMixin, ValidationError
 
 import dbt.deprecations
@@ -7,14 +7,24 @@
 import dbt.clients.agate_helper
 
 from dbt import ui  # type: ignore
-from dbt.adapters.base import BaseAdapter, available, RelationType, SchemaSearchMap, AdapterConfig
-from dbt.adapters.base.impl import log_code_execution
+from dbt.adapters.base import (
+    BaseAdapter,
+    available,
+    RelationType,
+    SchemaSearchMap,
+    AdapterConfig,
+    PythonJobHelper,
+)
 
 from dbt.adapters.cache import _make_key
 
 from dbt.adapters.bigquery.relation import BigQueryRelation
 from dbt.adapters.bigquery import BigQueryColumn
 from dbt.adapters.bigquery import BigQueryConnectionManager
+from dbt.adapters.bigquery.python_submissions import (
+    ClusterDataprocHelper,
+    ServerlessDataProcHelper,
+)
 from dbt.adapters.bigquery.connections import BigQueryAdapterResponse
 from dbt.contracts.graph.manifest import Manifest
 from dbt.events import AdapterLogger
@@ -835,108 +845,16 @@ def run_sql_for_tests(self, sql, fetch, conn=None):
         else:
             return list(res)
 
-    @available.parse_none
-    @log_code_execution
-    def submit_python_job(self, parsed_model: dict, compiled_code: str):
-        # TODO improve the typing here.  N.B. Jinja returns a `jinja2.runtime.Undefined` instead
-        # of `None` which evaluates to True!
-
-        # TODO limit this function to run only when doing the materialization of python nodes
-        # TODO should we also to timeout here?
-
-        # validate all additional stuff for python is set
-        schema = getattr(parsed_model, "schema", self.config.credentials.schema)
-        identifier = parsed_model["alias"]
-        python_required_configs = [
-            "dataproc_region",
-            "dataproc_cluster_name",
-            "gcs_bucket",
-        ]
-        for required_config in python_required_configs:
-            if not getattr(self.connections.profile.credentials, required_config):
-                raise ValueError(
-                    f"Need to supply {required_config} in profile to submit python job"
-                )
-        if not hasattr(self, "dataproc_helper"):
-            self.dataproc_helper = DataProcHelper(self.connections.profile.credentials)
-        model_file_name = f"{schema}/{identifier}.py"
-        # upload python file to GCS
-        self.dataproc_helper.upload_to_gcs(model_file_name, compiled_code)
-        # submit dataproc job
-        self.dataproc_helper.submit_dataproc_job(model_file_name)
-
-        # TODO proper result for this
-        message = "OK"
-        code = None
-        num_rows = None
-        bytes_processed = None
-        return BigQueryAdapterResponse(  # type: ignore[call-arg]
-            _message=message,
-            rows_affected=num_rows,
-            code=code,
-            bytes_processed=bytes_processed,
-        )
-
+    def generate_python_submission_response(self, submission_result) -> BigQueryAdapterResponse:
+        return BigQueryAdapterResponse(_message="OK")  # type: ignore[call-arg]
 
-class DataProcHelper:
-    def __init__(self, credential):
-        """_summary_
+    @property
+    def default_python_submission_method(self) -> str:
+        return "serverless"
 
-        Args:
-            credential (_type_): _description_
-        """
-        try:
-            # Library only needed for python models
-            from google.cloud import dataproc_v1
-            from google.cloud import storage
-        except ImportError:
-            raise RuntimeError(
-                "You need to install [dataproc] extras to run python model in dbt-bigquery"
-            )
-        self.credential = credential
-        self.GoogleCredentials = BigQueryConnectionManager.get_credentials(credential)
-        self.storage_client = storage.Client(
-            project=self.credential.database, credentials=self.GoogleCredentials
-        )
-        self.job_client = dataproc_v1.JobControllerClient(
-            client_options={
-                "api_endpoint": "{}-dataproc.googleapis.com:443".format(
-                    self.credential.dataproc_region
-                )
-            },
-            credentials=self.GoogleCredentials,
-        )
-
-    def upload_to_gcs(self, filename: str, compiled_code: str):
-        bucket = self.storage_client.get_bucket(self.credential.gcs_bucket)
-        blob = bucket.blob(filename)
-        blob.upload_from_string(compiled_code)
-
-    def submit_dataproc_job(self, filename: str):
-        # Create the job config.
-        job = {
-            "placement": {"cluster_name": self.credential.dataproc_cluster_name},
-            "pyspark_job": {
-                "main_python_file_uri": "gs://{}/{}".format(self.credential.gcs_bucket, filename)
-            },
+    @property
+    def python_submission_helpers(self) -> Dict[str, Type[PythonJobHelper]]:
+        return {
+            "cluster": ClusterDataprocHelper,
+            "serverless": ServerlessDataProcHelper,
         }
-        operation = self.job_client.submit_job_as_operation(
-            request={
-                "project_id": self.credential.database,
-                "region": self.credential.dataproc_region,
-                "job": job,
-            }
-        )
-        response = operation.result()
-        return response
-
-        # TODO: there might be useful results here that we can parse and return
-        # Dataproc job output is saved to the Cloud Storage bucket
-        # allocated to the job. Use regex to obtain the bucket and blob info.
-        # matches = re.match("gs://(.*?)/(.*)", response.driver_output_resource_uri)
-        # output = (
-        #     self.storage_client
-        #     .get_bucket(matches.group(1))
-        #     .blob(f"{matches.group(2)}.000000000")
-        #     .download_as_string()
-        # )
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
new file mode 100644
index 000000000..408984c2f
--- /dev/null
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -0,0 +1,152 @@
+from typing import Dict, Union
+
+from dbt.adapters.base import PythonJobHelper
+from dbt.adapters.bigquery import BigQueryConnectionManager, BigQueryCredentials
+from google.api_core import retry
+from google.api_core.client_options import ClientOptions
+
+try:
+    # library only needed for python models
+    from google.cloud import storage, dataproc_v1  # type: ignore
+except ImportError:
+    _has_dataproc_lib = False
+else:
+    _has_dataproc_lib = True
+
+
+class BaseDataProcHelper(PythonJobHelper):
+    def __init__(self, parsed_model: Dict, credential: BigQueryCredentials) -> None:
+        """_summary_
+
+        Args:
+            credential (_type_): _description_
+        """
+        if not _has_dataproc_lib:
+            raise RuntimeError(
+                "You need to install [dataproc] extras to run python model in dbt-bigquery"
+            )
+        # validate all additional stuff for python is set
+        schema = parsed_model["schema"]
+        identifier = parsed_model["alias"]
+        self.parsed_model = parsed_model
+        python_required_configs = [
+            "dataproc_region",
+            "gcs_bucket",
+        ]
+        for required_config in python_required_configs:
+            if not getattr(credential, required_config):
+                raise ValueError(
+                    f"Need to supply {required_config} in profile to submit python job"
+                )
+        self.model_file_name = f"{schema}/{identifier}.py"
+        self.credential = credential
+        self.GoogleCredentials = BigQueryConnectionManager.get_credentials(credential)
+        self.storage_client = storage.Client(
+            project=self.credential.database, credentials=self.GoogleCredentials
+        )
+        self.gcs_location = "gs://{}/{}".format(self.credential.gcs_bucket, self.model_file_name)
+
+        # set retry policy, default to timeout after 24 hours
+        self.timeout = self.parsed_model["config"].get(
+            "timeout", self.credential.job_execution_timeout_seconds or 60 * 60 * 24
+        )
+        self.retry = retry.Retry(maximum=10.0, deadline=self.timeout)
+        self.client_options = ClientOptions(
+            api_endpoint="{}-dataproc.googleapis.com:443".format(self.credential.dataproc_region)
+        )
+        self.job_client = self._get_job_client()
+
+    def _upload_to_gcs(self, filename: str, compiled_code: str) -> None:
+        bucket = self.storage_client.get_bucket(self.credential.gcs_bucket)
+        blob = bucket.blob(filename)
+        blob.upload_from_string(compiled_code)
+
+    def submit(self, compiled_code: str) -> dataproc_v1.types.jobs.Job:
+        # upload python file to GCS
+        self._upload_to_gcs(self.model_file_name, compiled_code)
+        # submit dataproc job
+        return self._submit_dataproc_job()
+
+    def _get_job_client(
+        self,
+    ) -> Union[dataproc_v1.JobControllerClient, dataproc_v1.BatchControllerClient]:
+        raise NotImplementedError("_get_job_client not implemented")
+
+    def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
+        raise NotImplementedError("_submit_dataproc_job not implemented")
+
+
+class ClusterDataprocHelper(BaseDataProcHelper):
+    def _get_job_client(self) -> dataproc_v1.JobControllerClient:
+        if not self._get_cluster_name():
+            raise ValueError(
+                "Need to supply dataproc_cluster_name in profile or config to submit python job with cluster submission method"
+            )
+        return dataproc_v1.JobControllerClient(  # type: ignore
+            client_options=self.client_options, credentials=self.GoogleCredentials
+        )
+
+    def _get_cluster_name(self) -> str:
+        return self.parsed_model["config"].get(
+            "dataproc_cluster_name", self.credential.dataproc_cluster_name
+        )
+
+    def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
+        job = {
+            "placement": {"cluster_name": self._get_cluster_name()},
+            "pyspark_job": {
+                "main_python_file_uri": self.gcs_location,
+            },
+        }
+        operation = self.job_client.submit_job_as_operation(  # type: ignore
+            request={
+                "project_id": self.credential.database,
+                "region": self.credential.dataproc_region,
+                "job": job,
+            }
+        )
+        response = operation.result(retry=self.retry)
+        return response
+
+
+class ServerlessDataProcHelper(BaseDataProcHelper):
+    def _get_job_client(self) -> dataproc_v1.BatchControllerClient:
+        return dataproc_v1.BatchControllerClient(
+            client_options=self.client_options, credentials=self.GoogleCredentials
+        )
+
+    def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
+        # create the Dataproc Serverless job config
+        batch = dataproc_v1.Batch()
+        batch.pyspark_batch.main_python_file_uri = self.gcs_location
+        # how to keep this up to date?
+        # we should probably also open this up to be configurable
+        batch.pyspark_batch.jar_file_uris = [
+            "gs://spark-lib/bigquery/spark-bigquery-with-dependencies_2.12-0.21.1.jar"
+        ]
+        # should we make all of these spark/dataproc properties configurable?
+        # https://cloud.google.com/dataproc-serverless/docs/concepts/properties
+        # https://cloud.google.com/dataproc-serverless/docs/reference/rest/v1/projects.locations.batches#runtimeconfig
+        batch.runtime_config.properties = {
+            "spark.executor.instances": "2",
+        }
+        parent = f"projects/{self.credential.database}/locations/{self.credential.dataproc_region}"
+        request = dataproc_v1.CreateBatchRequest(
+            parent=parent,
+            batch=batch,
+        )
+        # make the request
+        operation = self.job_client.create_batch(request=request)  # type: ignore
+        # this takes quite a while, waiting on GCP response to resolve
+        response = operation.result(retry=self.retry)
+        return response
+        # there might be useful results here that we can parse and return
+        # Dataproc job output is saved to the Cloud Storage bucket
+        # allocated to the job. Use regex to obtain the bucket and blob info.
+        # matches = re.match("gs://(.*?)/(.*)", response.driver_output_resource_uri)
+        # output = (
+        #     self.storage_client
+        #     .get_bucket(matches.group(1))
+        #     .blob(f"{matches.group(2)}.000000000")
+        #     .download_as_string()
+        # )
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index 75751f331..5ca735aa1 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -67,6 +67,6 @@ df = model(dbt, spark)
 df.write \
   .mode("overwrite") \
   .format("bigquery") \
-  .option("writeMethod", "direct") \
+  .option("writeMethod", "direct").option("writeDisposition", 'WRITE_TRUNCATE') \
   .save("{{target_relation}}")
 {% endmacro %}
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 129dbbe64..23418bf3f 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -1,7 +1,7 @@
 # install latest changes in dbt-core
 # TODO: how to automate switching from develop to version branches?
-git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
-git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
+git+https://github.com/dbt-labs/dbt-core.git@enhancement/python_submission_helper#egg=dbt-core&subdirectory=core
+git+https://github.com/dbt-labs/dbt-core.git@enhancement/python_submission_helper#egg=dbt-tests-adapter&subdirectory=tests/adapter
 
 black==22.8.0
 bumpversion
diff --git a/tests/conftest.py b/tests/conftest.py
index 7b0c69fc3..e74fa424b 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -44,6 +44,6 @@ def service_account_target():
         'keyfile_json': credentials,
         # following 3 for python model
         'dataproc_region': os.getenv("DATAPROC_REGION"),
-        'dataproc_cluster_name': os.getenv("DATAPROC_CLUSTER_NAME"),
+        'dataproc_cluster_name': os.getenv("DATAPROC_CLUSTER_NAME"), # only needed for cluster submission method
         'gcs_bucket': os.getenv("GCS_BUCKET")
     }
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
index 03ad871e2..68bb90e68 100644
--- a/tests/functional/adapter/test_python_model.py
+++ b/tests/functional/adapter/test_python_model.py
@@ -3,7 +3,7 @@
 from dbt.tests.util import run_dbt, write_file
 import dbt.tests.adapter.python_model.test_python_model as dbt_tests
 
-@pytest.skip("cluster unstable", allow_module_level=True)
+
 class TestPythonIncrementalMatsDataproc(dbt_tests.BasePythonIncrementalTests):
     pass
 

From 7daebe845d62d7f9e5c286e9ece50cab25b44a24 Mon Sep 17 00:00:00 2001
From: Chenyu Li 
Date: Mon, 12 Sep 2022 14:02:24 -0700
Subject: [PATCH 479/860] fix dev-req (#304)

---
 dbt/adapters/bigquery/python_submissions.py | 8 +++++---
 dev-requirements.txt                        | 4 ++--
 2 files changed, 7 insertions(+), 5 deletions(-)

diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 408984c2f..1cdbf438f 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -121,9 +121,11 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         batch.pyspark_batch.main_python_file_uri = self.gcs_location
         # how to keep this up to date?
         # we should probably also open this up to be configurable
-        batch.pyspark_batch.jar_file_uris = [
-            "gs://spark-lib/bigquery/spark-bigquery-with-dependencies_2.12-0.21.1.jar"
-        ]
+        jar_file_uri = self.parsed_model["config"].get(
+            "jar_file_uri",
+            "gs://spark-lib/bigquery/spark-bigquery-with-dependencies_2.12-0.21.1.jar",
+        )
+        batch.pyspark_batch.jar_file_uris = [jar_file_uri]
         # should we make all of these spark/dataproc properties configurable?
         # https://cloud.google.com/dataproc-serverless/docs/concepts/properties
         # https://cloud.google.com/dataproc-serverless/docs/reference/rest/v1/projects.locations.batches#runtimeconfig
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 23418bf3f..129dbbe64 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -1,7 +1,7 @@
 # install latest changes in dbt-core
 # TODO: how to automate switching from develop to version branches?
-git+https://github.com/dbt-labs/dbt-core.git@enhancement/python_submission_helper#egg=dbt-core&subdirectory=core
-git+https://github.com/dbt-labs/dbt-core.git@enhancement/python_submission_helper#egg=dbt-tests-adapter&subdirectory=tests/adapter
+git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
+git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
 
 black==22.8.0
 bumpversion

From 6f4fd0165d33ec79f4e71f0a9b6d977cddcfac25 Mon Sep 17 00:00:00 2001
From: Chenyu Li 
Date: Mon, 12 Sep 2022 16:01:05 -0700
Subject: [PATCH 480/860] Make storage and dataproc required Lib and disable
 unstable test (#305)

* mock optional library

* skip python tests again and remove type hint for optional lib

* add dataproc and storage as required

* remove try except for import
---
 dbt/adapters/bigquery/python_submissions.py   | 13 +------------
 setup.py                                      |  8 ++------
 tests/functional/adapter/test_python_model.py |  2 +-
 3 files changed, 4 insertions(+), 19 deletions(-)

diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 1cdbf438f..8d12fa2cc 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -4,14 +4,7 @@
 from dbt.adapters.bigquery import BigQueryConnectionManager, BigQueryCredentials
 from google.api_core import retry
 from google.api_core.client_options import ClientOptions
-
-try:
-    # library only needed for python models
-    from google.cloud import storage, dataproc_v1  # type: ignore
-except ImportError:
-    _has_dataproc_lib = False
-else:
-    _has_dataproc_lib = True
+from google.cloud import storage, dataproc_v1  # type: ignore
 
 
 class BaseDataProcHelper(PythonJobHelper):
@@ -21,10 +14,6 @@ def __init__(self, parsed_model: Dict, credential: BigQueryCredentials) -> None:
         Args:
             credential (_type_): _description_
         """
-        if not _has_dataproc_lib:
-            raise RuntimeError(
-                "You need to install [dataproc] extras to run python model in dbt-bigquery"
-            )
         # validate all additional stuff for python is set
         schema = parsed_model["schema"]
         identifier = parsed_model["alias"]
diff --git a/setup.py b/setup.py
index 69919fe1b..81a267711 100644
--- a/setup.py
+++ b/setup.py
@@ -72,13 +72,9 @@ def _get_dbt_core_version():
         "google-cloud-bigquery>=1.25.0,<3",
         "google-api-core>=1.16.0,<3",
         "googleapis-common-protos>=1.6.0,<2",
+        "google-cloud-storage>=2.4.0",
+        "google-cloud-dataproc>=4.0.3",
     ],
-    extras_require={
-        "dataproc": [
-            "google-cloud-storage[dataproc]>=2.4.0",
-            "google-cloud-dataproc[dataproc]>=4.0.3",
-        ],
-    },
     zip_safe=False,
     classifiers=[
         "Development Status :: 5 - Production/Stable",
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
index 68bb90e68..03ad871e2 100644
--- a/tests/functional/adapter/test_python_model.py
+++ b/tests/functional/adapter/test_python_model.py
@@ -3,7 +3,7 @@
 from dbt.tests.util import run_dbt, write_file
 import dbt.tests.adapter.python_model.test_python_model as dbt_tests
 
-
+@pytest.skip("cluster unstable", allow_module_level=True)
 class TestPythonIncrementalMatsDataproc(dbt_tests.BasePythonIncrementalTests):
     pass
 

From b4cc9b3ec1cddfba2b2d58d08af5a84f6d72b1b6 Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Tue, 13 Sep 2022 18:43:09 -0400
Subject: [PATCH 481/860] Update repo templates (#295)

---
 .github/ISSUE_TEMPLATE/bug-report.yml        | 84 ++++++++++++++++++++
 .github/ISSUE_TEMPLATE/bug_report.md         | 33 --------
 .github/ISSUE_TEMPLATE/config.yml            | 14 ++++
 .github/ISSUE_TEMPLATE/feature-request.yml   | 59 ++++++++++++++
 .github/ISSUE_TEMPLATE/feature_request.md    | 23 ------
 .github/ISSUE_TEMPLATE/regression-report.yml | 82 +++++++++++++++++++
 .github/ISSUE_TEMPLATE/release.md            | 10 ---
 .github/pull_request_template.md             |  7 +-
 8 files changed, 245 insertions(+), 67 deletions(-)
 create mode 100644 .github/ISSUE_TEMPLATE/bug-report.yml
 delete mode 100644 .github/ISSUE_TEMPLATE/bug_report.md
 create mode 100644 .github/ISSUE_TEMPLATE/config.yml
 create mode 100644 .github/ISSUE_TEMPLATE/feature-request.yml
 delete mode 100644 .github/ISSUE_TEMPLATE/feature_request.md
 create mode 100644 .github/ISSUE_TEMPLATE/regression-report.yml
 delete mode 100644 .github/ISSUE_TEMPLATE/release.md

diff --git a/.github/ISSUE_TEMPLATE/bug-report.yml b/.github/ISSUE_TEMPLATE/bug-report.yml
new file mode 100644
index 000000000..b785e189b
--- /dev/null
+++ b/.github/ISSUE_TEMPLATE/bug-report.yml
@@ -0,0 +1,84 @@
+name: 🐞 Bug
+description: Report a bug or an issue you've found with dbt-bigquery
+title: "[Bug] "
+labels: ["bug", "triage"]
+body:
+  - type: markdown
+    attributes:
+      value: |
+        Thanks for taking the time to fill out this bug report!
+  - type: checkboxes
+    attributes:
+      label: Is this a new bug in dbt-bigquery?
+      description: >
+        In other words, is this an error, flaw, failure or fault in our software?
+
+        If this is a bug that broke existing functionality that used to work, please open a regression issue.
+        If this is a bug in the dbt-core logic, please open an issue in the dbt-core repository.
+        If this is a bug experienced while using dbt Cloud, please report to [support](mailto:support@getdbt.com).
+        If this is a request for help or troubleshooting code in your own dbt project, please join our [dbt Community Slack](https://www.getdbt.com/community/join-the-community/) or open a [Discussion question](https://github.com/dbt-labs/docs.getdbt.com/discussions).
+
+        Please search to see if an issue already exists for the bug you encountered.
+      options:
+        - label: I believe this is a new bug in dbt-bigquery
+          required: true
+        - label: I have searched the existing issues, and I could not find an existing issue for this bug
+          required: true
+  - type: textarea
+    attributes:
+      label: Current Behavior
+      description: A concise description of what you're experiencing.
+    validations:
+      required: true
+  - type: textarea
+    attributes:
+      label: Expected Behavior
+      description: A concise description of what you expected to happen.
+    validations:
+      required: true
+  - type: textarea
+    attributes:
+      label: Steps To Reproduce
+      description: Steps to reproduce the behavior.
+      placeholder: |
+        1. In this environment...
+        2. With this config...
+        3. Run '...'
+        4. See error...
+    validations:
+      required: true
+  - type: textarea
+    id: logs
+    attributes:
+      label: Relevant log output
+      description: |
+        If applicable, log output to help explain your problem.
+      render: shell
+    validations:
+      required: false
+  - type: textarea
+    attributes:
+      label: Environment
+      description: |
+        examples:
+          - **OS**: Ubuntu 20.04
+          - **Python**: 3.9.12 (`python3 --version`)
+          - **dbt-core**: 1.1.1 (`dbt --version`)
+          - **dbt-bigquery**: 1.1.0 (`dbt --version`)
+      value: |
+        - OS:
+        - Python:
+        - dbt-core:
+        - dbt-bigquery:
+      render: markdown
+    validations:
+      required: false
+  - type: textarea
+    attributes:
+      label: Additional Context
+      description: |
+        Links? References? Anything that will give us more context about the issue you are encountering!
+
+        Tip: You can attach images or log files by clicking this area to highlight it and then dragging files in.
+    validations:
+      required: false
diff --git a/.github/ISSUE_TEMPLATE/bug_report.md b/.github/ISSUE_TEMPLATE/bug_report.md
deleted file mode 100644
index c2cb2a061..000000000
--- a/.github/ISSUE_TEMPLATE/bug_report.md
+++ /dev/null
@@ -1,33 +0,0 @@
----
-name: Bug report
-about: Report a bug or an issue you've found with dbt-bigquery
-title: ''
-labels: bug, triage
-assignees: ''
-
----
-
-### Describe the bug
-A clear and concise description of what the bug is. What command did you run? What happened?
-
-### 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
-A clear and concise description of what you expected to happen.
-
-### Screenshots and log output
-If applicable, add screenshots or log output to help explain your problem.
-
-### System information
-**The output of `dbt --version`:**
-```
-<output goes here>
-```
-
-**The operating system you're using:**
-
-**The output of `python --version`:**
-
-### Additional context
-Add any other context about the problem here.
diff --git a/.github/ISSUE_TEMPLATE/config.yml b/.github/ISSUE_TEMPLATE/config.yml
new file mode 100644
index 000000000..afd2ea18c
--- /dev/null
+++ b/.github/ISSUE_TEMPLATE/config.yml
@@ -0,0 +1,14 @@
+blank_issues_enabled: false
+contact_links:
+  - name: Ask the community for help
+    url: https://github.com/dbt-labs/docs.getdbt.com/discussions
+    about: Need help troubleshooting? Check out our guide on how to ask
+  - name: Contact dbt Cloud support
+    url: mailto:support@getdbt.com
+    about: Are you using dbt Cloud? Contact our support team for help!
+  - name: Participate in Discussions
+    url: https://github.com/dbt-labs/dbt-bigquery/discussions
+    about: Do you have a Big Idea for dbt-bigquery? Read open discussions, or start a new one
+  - name: Create an issue for dbt-core
+    url: https://github.com/dbt-labs/dbt-core/issues/new/choose
+    about: Report a bug or request a feature for dbt-core
diff --git a/.github/ISSUE_TEMPLATE/feature-request.yml b/.github/ISSUE_TEMPLATE/feature-request.yml
new file mode 100644
index 000000000..3cd55868b
--- /dev/null
+++ b/.github/ISSUE_TEMPLATE/feature-request.yml
@@ -0,0 +1,59 @@
+name: ✨ Feature
+description: Propose a straightforward extension of dbt-bigquery functionality
+title: "[Feature] <title>"
+labels: ["enhancement", "triage"]
+body:
+  - type: markdown
+    attributes:
+      value: |
+        Thanks for taking the time to fill out this feature request!
+  - type: checkboxes
+    attributes:
+      label: Is this your first time submitting a feature request?
+      description: >
+        We want to make sure that features are distinct and discoverable,
+        so that other members of the community can find them and offer their thoughts.
+
+        Issues are the right place to request straightforward extensions of existing dbt-bigquery functionality.
+        For "big ideas" about future capabilities of dbt-bigquery, we ask that you open a
+        [discussion](https://github.com/dbt-labs/dbt-bigquery/discussions) in the "Ideas" category instead.
+      options:
+        - label: I have read the [expectations for open source contributors](https://docs.getdbt.com/docs/contributing/oss-expectations)
+          required: true
+        - label: I have searched the existing issues, and I could not find an existing issue for this feature
+          required: true
+        - label: I am requesting a straightforward extension of existing dbt-bigquery functionality, rather than a Big Idea better suited to a discussion
+          required: true
+  - type: textarea
+    attributes:
+      label: Describe the feature
+      description: A clear and concise description of what you want to happen.
+    validations:
+      required: true
+  - type: textarea
+    attributes:
+      label: Describe alternatives you've considered
+      description: |
+        A clear and concise description of any alternative solutions or features you've considered.
+    validations:
+      required: false
+  - type: textarea
+    attributes:
+      label: Who will this benefit?
+      description: |
+        What kind of use case will this feature be useful for? Please be specific and provide examples, this will help us prioritize properly.
+    validations:
+      required: false
+  - type: input
+    attributes:
+      label: Are you interested in contributing this feature?
+      description: Let us know if you want to write some code, and how we can help.
+    validations:
+      required: false
+  - type: textarea
+    attributes:
+      label: Anything else?
+      description: |
+        Links? References? Anything that will give us more context about the feature you are suggesting!
+    validations:
+      required: false
diff --git a/.github/ISSUE_TEMPLATE/feature_request.md b/.github/ISSUE_TEMPLATE/feature_request.md
deleted file mode 100644
index 5c0a9545f..000000000
--- a/.github/ISSUE_TEMPLATE/feature_request.md
+++ /dev/null
@@ -1,23 +0,0 @@
----
-name: Feature request
-about: Suggest an idea for dbt-bigquery
-title: ''
-labels: enhancement, triage
-assignees: ''
-
----
-
-### Describe the feature
-A clear and concise description of what you want to happen.
-
-### Describe alternatives you've considered
-A clear and concise description of any alternative solutions or features you've considered.
-
-### Additional context
-Please include any other relevant context here.
-
-### Who will this benefit?
-What kind of use case will this feature be useful for? Please be specific and provide examples, this will help us prioritize properly.
-
-### Are you interested in contributing this feature?
-Let us know if you want to write some code, and how we can help.
diff --git a/.github/ISSUE_TEMPLATE/regression-report.yml b/.github/ISSUE_TEMPLATE/regression-report.yml
new file mode 100644
index 000000000..c9f4853cd
--- /dev/null
+++ b/.github/ISSUE_TEMPLATE/regression-report.yml
@@ -0,0 +1,82 @@
+name: ☣️ Regression
+description: Report a regression you've observed in a newer version of dbt-bigquery
+title: "[Regression] <title>"
+labels: ["bug", "regression", "triage"]
+body:
+  - type: markdown
+    attributes:
+      value: |
+        Thanks for taking the time to fill out this regression report!
+  - type: checkboxes
+    attributes:
+      label: Is this a regression in a recent version of dbt-bigquery?
+      description: >
+        A regression is when documented functionality works as expected in an older version of dbt-bigquery,
+        and no longer works after upgrading to a newer version of dbt-bigquery
+      options:
+        - label: I believe this is a regression in dbt-bigquery functionality
+          required: true
+        - label: I have searched the existing issues, and I could not find an existing issue for this regression
+          required: true
+  - type: textarea
+    attributes:
+      label: Current Behavior
+      description: A concise description of what you're experiencing.
+    validations:
+      required: true
+  - type: textarea
+    attributes:
+      label: Expected/Previous Behavior
+      description: A concise description of what you expected to happen.
+    validations:
+      required: true
+  - type: textarea
+    attributes:
+      label: Steps To Reproduce
+      description: Steps to reproduce the behavior.
+      placeholder: |
+        1. In this environment...
+        2. With this config...
+        3. Run '...'
+        4. See error...
+    validations:
+      required: true
+  - type: textarea
+    id: logs
+    attributes:
+      label: Relevant log output
+      description: |
+        If applicable, log output to help explain your problem.
+      render: shell
+    validations:
+      required: false
+  - type: textarea
+    attributes:
+      label: Environment
+      description: |
+        examples:
+          - **OS**: Ubuntu 20.04
+          - **Python**: 3.9.12 (`python3 --version`)
+          - **dbt-core (working version)**: 1.1.1 (`dbt --version`)
+          - **dbt-bigquery (working version)**: 1.1.0 (`dbt --version`)
+          - **dbt-core (regression version)**: 1.2.0 (`dbt --version`)
+          - **dbt-bigquery (regression version)**: 1.2.0 (`dbt --version`)
+      value: |
+        - OS:
+        - Python:
+        - dbt-core (working version):
+        - dbt-bigquery (working version):
+        - dbt-core (regression version):
+        - dbt-bigquery (regression version):
+      render: markdown
+    validations:
+      required: true
+  - type: textarea
+    attributes:
+      label: Additional Context
+      description: |
+        Links? References? Anything that will give us more context about the issue you are encountering!
+
+        Tip: You can attach images or log files by clicking this area to highlight it and then dragging files in.
+    validations:
+      required: false
diff --git a/.github/ISSUE_TEMPLATE/release.md b/.github/ISSUE_TEMPLATE/release.md
deleted file mode 100644
index c3cb05c3c..000000000
--- a/.github/ISSUE_TEMPLATE/release.md
+++ /dev/null
@@ -1,10 +0,0 @@
----
-name: Release
-about: Release a new version of dbt-bigquery
-title: ''
-labels: release
-assignees: ''
-
----
-
-### TBD
diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md
index 5d8734bd5..33a1e4538 100644
--- a/.github/pull_request_template.md
+++ b/.github/pull_request_template.md
@@ -11,11 +11,16 @@ resolves #
 
 ### Description
 
-<!--- Describe the Pull Request here -->
+<!---
+  Describe the Pull Request here. Add any references and info to help reviewers
+  understand your changes. Include any tradeoffs you considered.
+-->
 
 ### Checklist
 
+- [ ] I have read [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md) and understand what's expected of me
 - [ ] I have signed the [CLA](https://docs.getdbt.com/docs/contributor-license-agreements)
 - [ ] I have run this code in development and it appears to resolve the stated issue
 - [ ] This PR includes tests, or tests are not required/relevant for this PR
+- [ ] I have [opened an issue to add/update docs](https://github.com/dbt-labs/docs.getdbt.com/issues/new/choose), or docs changes are not required/relevant for this PR
 - [ ] I have run `changie new` to [create a changelog entry](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#Adding-CHANGELOG-Entry)

From 61ed3b02a999995fe547dcc263ad1e0fd7e575f8 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Fri, 16 Sep 2022 12:00:28 -0700
Subject: [PATCH 482/860] update retry settings experiment (#310)

* update retry settings

* changelog entry
---
 .changes/unreleased/Under the Hood-20220915-145212.yaml | 7 +++++++
 dbt/adapters/bigquery/connections.py                    | 4 +---
 tests/conftest.py                                       | 2 ++
 3 files changed, 10 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20220915-145212.yaml

diff --git a/.changes/unreleased/Under the Hood-20220915-145212.yaml b/.changes/unreleased/Under the Hood-20220915-145212.yaml
new file mode 100644
index 000000000..6ab035256
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20220915-145212.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: Update BQ job and call retry settings
+time: 2022-09-15T14:52:12.902965-07:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "311"
+  PR: "310"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 50437622b..336db3fd1 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -184,10 +184,8 @@ def __pre_deserialize__(cls, d: Dict[Any, Any]) -> Dict[Any, Any]:
 class BigQueryConnectionManager(BaseConnectionManager):
     TYPE = "bigquery"
 
-    QUERY_TIMEOUT = 300
-    RETRIES = 1
     DEFAULT_INITIAL_DELAY = 1.0  # Seconds
-    DEFAULT_MAXIMUM_DELAY = 1.0  # Seconds
+    DEFAULT_MAXIMUM_DELAY = 3.0  # Seconds
 
     @classmethod
     def handle_error(cls, error, message):
diff --git a/tests/conftest.py b/tests/conftest.py
index e74fa424b..6ca033f8d 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -29,6 +29,7 @@ def oauth_target():
         'type': 'bigquery',
         'method': 'oauth',
         'threads': 1,
+        'job_retries': 2,
     }
 
 
@@ -40,6 +41,7 @@ def service_account_target():
         'type': 'bigquery',
         'method': 'service-account-json',
         'threads': 1,
+        'job_retries': 2,
         'project': project_id,
         'keyfile_json': credentials,
         # following 3 for python model

From 9699a48c3efd923f95d1fc13ea77827fa222d7db Mon Sep 17 00:00:00 2001
From: Matteo Ferrando <matteo.ferrando2@gmail.com>
Date: Mon, 19 Sep 2022 12:46:58 -0400
Subject: [PATCH 483/860] Convert df to pyspark DataFrame if it is pandas
 before writing (#301)

---
 .changes/unreleased/Under the Hood-20220908-143218.yaml | 7 +++++++
 dbt/include/bigquery/macros/materializations/table.sql  | 8 ++++++++
 2 files changed, 15 insertions(+)
 create mode 100644 .changes/unreleased/Under the Hood-20220908-143218.yaml

diff --git a/.changes/unreleased/Under the Hood-20220908-143218.yaml b/.changes/unreleased/Under the Hood-20220908-143218.yaml
new file mode 100644
index 000000000..7053f96b6
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20220908-143218.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: Convert df to pyspark DataFrame if it is pandas before writing
+time: 2022-09-08T14:32:18.243412-04:00
+custom:
+  Author: chamini2
+  Issue: "301"
+  PR: "301"
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index 5ca735aa1..519b8f994 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -63,6 +63,14 @@ df = model(dbt, spark)
 # COMMAND ----------
 # this is materialization code dbt generated, please do not modify
 
+# 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):
+      # convert to pyspark.DataFrame
+      df = spark.createDataFrame(df)
 
 df.write \
   .mode("overwrite") \

From 353e724b4666fa4201e525439c4efcdff3fe9464 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 20 Sep 2022 16:41:56 -0400
Subject: [PATCH 484/860] Bump mypy from 0.942 to 0.971 (#296)

* Bump mypy from 0.942 to 0.971

Bumps [mypy](https://github.com/python/mypy) from 0.942 to 0.971.
- [Release notes](https://github.com/python/mypy/releases)
- [Commits](https://github.com/python/mypy/compare/v0.942...v0.971)

---
updated-dependencies:
- dependency-name: mypy
  dependency-type: direct:development
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

* Remove newline

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: leahwicz <60146280+leahwicz@users.noreply.github.com>
---
 .changes/unreleased/Dependency-20220920-200205.yaml | 7 +++++++
 dev-requirements.txt                                | 2 +-
 2 files changed, 8 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependency-20220920-200205.yaml

diff --git a/.changes/unreleased/Dependency-20220920-200205.yaml b/.changes/unreleased/Dependency-20220920-200205.yaml
new file mode 100644
index 000000000..6dcfa5924
--- /dev/null
+++ b/.changes/unreleased/Dependency-20220920-200205.yaml
@@ -0,0 +1,7 @@
+kind: Dependency
+body: "Bump mypy from 0.942 to 0.971"
+time: 2022-09-20T20:02:05.00000Z
+custom:
+  Author: dependabot[bot]
+  Issue: 254
+  PR: 296
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 129dbbe64..676703d3e 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -9,7 +9,7 @@ flake8
 flaky
 freezegun==1.1.0
 ipdb
-mypy==0.942
+mypy==0.971
 pip-tools
 pre-commit
 pytest

From 1e6558126e3d2cd761d8114804cd5e00ae3b6f93 Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Wed, 21 Sep 2022 12:13:41 -0600
Subject: [PATCH 485/860] Convert df to Spark DataFrame if it is a pandas or
 pandas-on-Spark DataFrame before writing (#317)

* Convert df to Spark DataFrame if it is a pandas or pandas-on-Spark DataFrame before writing

* Changelog entry
---
 .../Under the Hood-20220920-151057.yaml       |  7 ++++
 .../macros/materializations/table.sql         | 37 +++++++++++++++----
 2 files changed, 37 insertions(+), 7 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20220920-151057.yaml

diff --git a/.changes/unreleased/Under the Hood-20220920-151057.yaml b/.changes/unreleased/Under the Hood-20220920-151057.yaml
new file mode 100644
index 000000000..86914c4f3
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20220920-151057.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: Enable pandas-on-Spark DataFrames for dbt python models
+time: 2022-09-20T15:10:57.712169-06:00
+custom:
+  Author: dbeatty10
+  Issue: "316"
+  PR: "317"
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index 519b8f994..3ecdf37a5 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -63,14 +63,37 @@ df = model(dbt, spark)
 # COMMAND ----------
 # this is materialization code dbt generated, please do not modify
 
-# 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):
-      # convert to pyspark.DataFrame
-      df = spark.createDataFrame(df)
+
+pandas_available = False
+pyspark_available = False
+
+# make sure pandas exists before using it
+if importlib.util.find_spec("pandas"):
+  import pandas
+  pandas_available = True
+
+# make sure pyspark.pandas exists before using it
+if importlib.util.find_spec("pyspark.pandas"):
+  import pyspark.pandas
+  pyspark_available = True
+
+# preferentially convert pandas DataFrames to pandas-on-Spark DataFrames first
+# since they know how to convert pandas DataFrames better than `spark.createDataFrame(df)`
+# and converting from pandas-on-Spark to Spark DataFrame has no overhead
+if pyspark_available and pandas_available and isinstance(df, pandas.core.frame.DataFrame):
+  df = pyspark.pandas.frame.DataFrame(df)
+
+# convert to pyspark.sql.dataframe.DataFrame
+if isinstance(df, pyspark.sql.dataframe.DataFrame):
+  pass  # since it is already a Spark DataFrame
+elif pyspark_available and isinstance(df, pyspark.pandas.frame.DataFrame):
+  df = df.to_spark()
+elif pandas_available and isinstance(df, pandas.core.frame.DataFrame):
+  df = spark.createDataFrame(df)
+else:
+  msg = f"{type(df)} is not a supported type for dbt Python materialization"
+  raise Exception(msg)
 
 df.write \
   .mode("overwrite") \

From 24a787fae36aa30a6c5dc29c9944a91a671fb78a Mon Sep 17 00:00:00 2001
From: James McNeill <55981540+jpmmcneill@users.noreply.github.com>
Date: Thu, 22 Sep 2022 14:15:51 +0100
Subject: [PATCH 486/860] add type boolean (#313)

* add type boolean

* changie result

* dont pull main

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
---
 .changes/unreleased/Features-20220919-232721.yaml | 7 +++++++
 dbt/adapters/bigquery/column.py                   | 1 +
 tests/functional/adapter/utils/test_data_types.py | 5 +++++
 3 files changed, 13 insertions(+)
 create mode 100644 .changes/unreleased/Features-20220919-232721.yaml

diff --git a/.changes/unreleased/Features-20220919-232721.yaml b/.changes/unreleased/Features-20220919-232721.yaml
new file mode 100644
index 000000000..4f5c90dd0
--- /dev/null
+++ b/.changes/unreleased/Features-20220919-232721.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: add type_boolean as a data type macro
+time: 2022-09-19T23:27:21.864912+01:00
+custom:
+  Author: jpmmcneill
+  Issue: "315"
+  PR: "313"
diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py
index 1a6d042b6..0f9269283 100644
--- a/dbt/adapters/bigquery/column.py
+++ b/dbt/adapters/bigquery/column.py
@@ -15,6 +15,7 @@ class BigQueryColumn(Column):
         "TIMESTAMP": "TIMESTAMP",
         "FLOAT": "FLOAT64",
         "INTEGER": "INT64",
+        "BOOLEAN": "BOOLEAN",
         "RECORD": "RECORD",
     }
     fields: List[Self]  # type: ignore
diff --git a/tests/functional/adapter/utils/test_data_types.py b/tests/functional/adapter/utils/test_data_types.py
index af084a0ef..e1669910c 100644
--- a/tests/functional/adapter/utils/test_data_types.py
+++ b/tests/functional/adapter/utils/test_data_types.py
@@ -5,6 +5,7 @@
 from dbt.tests.adapter.utils.data_types.test_type_numeric import BaseTypeNumeric
 from dbt.tests.adapter.utils.data_types.test_type_string import BaseTypeString
 from dbt.tests.adapter.utils.data_types.test_type_timestamp import BaseTypeTimestamp
+from dbt.tests.adapter.utils.data_types.test_type_boolean import BaseTypeBoolean
 
 
 class TestTypeBigInt(BaseTypeBigInt):
@@ -31,4 +32,8 @@ class TestTypeString(BaseTypeString):
 class TestTypeTimestamp(BaseTypeTimestamp):
     pass
 
+
+class TestTypeBoolean(BaseTypeBoolean):
+    pass
+
     
\ No newline at end of file

From e110bd6124e6be21f5603d196b7438958a976352 Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Mon, 26 Sep 2022 14:29:00 -0600
Subject: [PATCH 487/860] Tests for `current_timestamp` (#322)

* Test failure will confirm that it is not a timezone-naive data type

* Temporarily update dev-requirements.text

* Tests for current_timestamp

* Restore original dev-requirements.txt
---
 tests/functional/adapter/utils/test_utils.py | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/tests/functional/adapter/utils/test_utils.py b/tests/functional/adapter/utils/test_utils.py
index 2a37ac6b7..ae1aadcbc 100644
--- a/tests/functional/adapter/utils/test_utils.py
+++ b/tests/functional/adapter/utils/test_utils.py
@@ -4,6 +4,7 @@
 from dbt.tests.adapter.utils.test_bool_or import BaseBoolOr
 from dbt.tests.adapter.utils.test_cast_bool_to_text import BaseCastBoolToText
 from dbt.tests.adapter.utils.test_concat import BaseConcat
+from dbt.tests.adapter.utils.test_current_timestamp import BaseCurrentTimestampAware
 from dbt.tests.adapter.utils.test_dateadd import BaseDateAdd
 from dbt.tests.adapter.utils.test_datediff import BaseDateDiff
 from dbt.tests.adapter.utils.test_date_trunc import BaseDateTrunc
@@ -39,6 +40,11 @@ class TestConcat(BaseConcat):
     pass
 
 
+# Use either BaseCurrentTimestampAware or BaseCurrentTimestampNaive but not both
+class TestCurrentTimestamp(BaseCurrentTimestampAware):
+    pass
+
+
 class TestDateAdd(BaseDateAdd):
     pass
 

From 164478c20cfff851c40351f3f9e0ab0a67936099 Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Mon, 26 Sep 2022 16:49:46 -0600
Subject: [PATCH 488/860] Array macros (#308)

* Temporary dev-requirements

* Changelog entry

* Implementations and tests for array macros

* Remove `cast_array_to_string` macro

* ARRAY in BigQuery does not support set operation comparisons using EXCEPT

* Restore original dev-requirements.txt
---
 .../unreleased/Features-20220913-084836.yaml  |  7 +++
 .../bigquery/macros/utils/array_append.sql    |  3 +
 .../bigquery/macros/utils/array_concat.sql    |  3 +
 .../bigquery/macros/utils/array_construct.sql |  7 +++
 .../adapter/utils/fixture_array_append.py     | 13 ++++
 .../adapter/utils/fixture_array_concat.py     | 15 +++++
 .../adapter/utils/fixture_array_construct.py  | 20 ++++++
 tests/functional/adapter/utils/test_utils.py  | 63 ++++++++++++++++++-
 8 files changed, 130 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Features-20220913-084836.yaml
 create mode 100644 dbt/include/bigquery/macros/utils/array_append.sql
 create mode 100644 dbt/include/bigquery/macros/utils/array_concat.sql
 create mode 100644 dbt/include/bigquery/macros/utils/array_construct.sql
 create mode 100644 tests/functional/adapter/utils/fixture_array_append.py
 create mode 100644 tests/functional/adapter/utils/fixture_array_concat.py
 create mode 100644 tests/functional/adapter/utils/fixture_array_construct.py

diff --git a/.changes/unreleased/Features-20220913-084836.yaml b/.changes/unreleased/Features-20220913-084836.yaml
new file mode 100644
index 000000000..0a4bd275d
--- /dev/null
+++ b/.changes/unreleased/Features-20220913-084836.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: Array macros
+time: 2022-09-13T08:48:36.255365-06:00
+custom:
+  Author: graciegoheen dbeatty10
+  Issue: "307"
+  PR: "308"
diff --git a/dbt/include/bigquery/macros/utils/array_append.sql b/dbt/include/bigquery/macros/utils/array_append.sql
new file mode 100644
index 000000000..78bd5cc43
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/array_append.sql
@@ -0,0 +1,3 @@
+{% macro bigquery__array_append(array, new_element) -%}
+    {{ array_concat(array, array_construct([new_element])) }}
+{%- endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/array_concat.sql b/dbt/include/bigquery/macros/utils/array_concat.sql
new file mode 100644
index 000000000..eff8f524a
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/array_concat.sql
@@ -0,0 +1,3 @@
+{% macro bigquery__array_concat(array_1, array_2) -%}
+    array_concat({{ array_1 }}, {{ array_2 }})
+{%- endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/array_construct.sql b/dbt/include/bigquery/macros/utils/array_construct.sql
new file mode 100644
index 000000000..270b1f785
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/array_construct.sql
@@ -0,0 +1,7 @@
+{% macro bigquery__array_construct(inputs, data_type) -%}
+    {% if inputs|length > 0 %}
+    [ {{ inputs|join(' , ') }} ]
+    {% else %}
+    ARRAY<{{data_type}}>[]
+    {% endif %}
+{%- endmacro %}
diff --git a/tests/functional/adapter/utils/fixture_array_append.py b/tests/functional/adapter/utils/fixture_array_append.py
new file mode 100644
index 000000000..0558d66e1
--- /dev/null
+++ b/tests/functional/adapter/utils/fixture_array_append.py
@@ -0,0 +1,13 @@
+# array_append
+
+# EXCEPT can't be used with ARRAYs in BigQuery, so convert to a string
+models__array_append_expected_sql = """
+select 1 as id, {{ array_to_string(array_construct([1,2,3,4])) }} as array_col union all
+select 2 as id, {{ array_to_string(array_construct([4])) }} as array_col
+"""
+
+
+models__array_append_actual_sql = """
+select 1 as id, {{ array_to_string(array_append(array_construct([1,2,3]), 4)) }} as array_col union all
+select 2 as id, {{ array_to_string(array_append(array_construct([]), 4)) }} as array_col
+"""
diff --git a/tests/functional/adapter/utils/fixture_array_concat.py b/tests/functional/adapter/utils/fixture_array_concat.py
new file mode 100644
index 000000000..51af8bf12
--- /dev/null
+++ b/tests/functional/adapter/utils/fixture_array_concat.py
@@ -0,0 +1,15 @@
+# array_concat
+
+# EXCEPT can't be used with ARRAYs in BigQuery, so convert to a string
+models__array_concat_expected_sql = """
+select 1 as id, {{ array_to_string(array_construct([1,2,3,4,5,6])) }} as array_col union all
+select 2 as id, {{ array_to_string(array_construct([2])) }} as array_col union all
+select 3 as id, {{ array_to_string(array_construct([3])) }} as array_col
+"""
+
+
+models__array_concat_actual_sql = """
+select 1 as id, {{ array_to_string(array_concat(array_construct([1,2,3]), array_construct([4,5,6]))) }} as array_col union all
+select 2 as id, {{ array_to_string(array_concat(array_construct([]), array_construct([2]))) }} as array_col union all
+select 3 as id, {{ array_to_string(array_concat(array_construct([3]), array_construct([]))) }} as array_col
+"""
diff --git a/tests/functional/adapter/utils/fixture_array_construct.py b/tests/functional/adapter/utils/fixture_array_construct.py
new file mode 100644
index 000000000..13d0bb2f3
--- /dev/null
+++ b/tests/functional/adapter/utils/fixture_array_construct.py
@@ -0,0 +1,20 @@
+# array_construct
+
+# EXCEPT can't be used with ARRAYs in BigQuery, so convert to a string
+models__array_construct_expected_sql = """
+select 1 as id, {{ array_to_string(array_construct([1,2,3])) }} as array_col union all
+select 2 as id, {{ array_to_string(array_construct([])) }} as array_col
+"""
+
+
+models__array_construct_actual_sql = """
+select 1 as id, {{ array_to_string(array_construct([1,2,3])) }} as array_col union all
+select 2 as id, {{ array_to_string(array_construct([])) }} as array_col
+"""
+
+
+macros__array_to_string_sql = """
+{% macro array_to_string(array) %}
+    (select string_agg(cast(element as string), ',') from unnest({{ array }}) element)
+{% endmacro %}
+"""
diff --git a/tests/functional/adapter/utils/test_utils.py b/tests/functional/adapter/utils/test_utils.py
index ae1aadcbc..5f5c5d83b 100644
--- a/tests/functional/adapter/utils/test_utils.py
+++ b/tests/functional/adapter/utils/test_utils.py
@@ -1,5 +1,8 @@
 import pytest
-from dbt.tests.adapter.utils.base_utils import BaseUtils
+
+from dbt.tests.adapter.utils.test_array_append import BaseArrayAppend
+from dbt.tests.adapter.utils.test_array_concat import BaseArrayConcat
+from dbt.tests.adapter.utils.test_array_construct import BaseArrayConstruct
 from dbt.tests.adapter.utils.test_any_value import BaseAnyValue
 from dbt.tests.adapter.utils.test_bool_or import BaseBoolOr
 from dbt.tests.adapter.utils.test_cast_bool_to_text import BaseCastBoolToText
@@ -22,12 +25,70 @@
 from dbt.tests.adapter.utils.test_safe_cast import BaseSafeCast
 from dbt.tests.adapter.utils.test_split_part import BaseSplitPart
 from dbt.tests.adapter.utils.test_string_literal import BaseStringLiteral
+from tests.functional.adapter.utils.fixture_array_append import (
+    models__array_append_actual_sql,
+    models__array_append_expected_sql,
+)
+from tests.functional.adapter.utils.fixture_array_concat import (
+    models__array_concat_actual_sql,
+    models__array_concat_expected_sql,
+)
+from tests.functional.adapter.utils.fixture_array_construct import (
+    models__array_construct_actual_sql,
+    models__array_construct_expected_sql,
+    macros__array_to_string_sql,
+)
 
 
 class TestAnyValue(BaseAnyValue):
     pass
 
 
+class TestArrayAppend(BaseArrayAppend):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "actual.sql": models__array_append_actual_sql,
+            "expected.sql": models__array_append_expected_sql,
+        }
+
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {
+            "array_to_string.sql": macros__array_to_string_sql,
+        }
+
+
+class TestArrayConcat(BaseArrayConcat):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "actual.sql": models__array_concat_actual_sql,
+            "expected.sql": models__array_concat_expected_sql,
+        }
+
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {
+            "array_to_string.sql": macros__array_to_string_sql,
+        }
+
+
+class TestArrayConstruct(BaseArrayConstruct):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "actual.sql": models__array_construct_actual_sql,
+            "expected.sql": models__array_construct_expected_sql,
+        }
+
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {
+            "array_to_string.sql": macros__array_to_string_sql,
+        }
+
+
 class TestBoolOr(BaseBoolOr):
     pass
 

From 8d0c3bbedf647ef7e4110e47ac3626cfd9b5ae9c Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Tue, 27 Sep 2022 06:00:14 -0600
Subject: [PATCH 489/860] Convert df to pyspark DataFrame if it is koalas
 before writing (#321)

* Convert df to pyspark DataFrame if it is koalas before writing

* Changelog entry

* Preferentially convert Koalas DataFrames to pandas-on-Spark DataFrames first

* Update changelog entry

* Fix explanation

Co-authored-by: Chenyu Li <chenyu.li@dbtlabs.com>
---
 .../unreleased/Under the Hood-20220924-134858.yaml   |  7 +++++++
 .../bigquery/macros/materializations/table.sql       | 12 +++++++++++-
 2 files changed, 18 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Under the Hood-20220924-134858.yaml

diff --git a/.changes/unreleased/Under the Hood-20220924-134858.yaml b/.changes/unreleased/Under the Hood-20220924-134858.yaml
new file mode 100644
index 000000000..a4fbceb3f
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20220924-134858.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: Convert df to pyspark DataFrame if it is koalas before writing
+time: 2022-09-24T13:48:58.614746-06:00
+custom:
+  Author: dbeatty10 ueshin
+  Issue: "320"
+  PR: "321"
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index 3ecdf37a5..1bea3f487 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -67,6 +67,7 @@ import importlib.util
 
 pandas_available = False
 pyspark_available = False
+koalas_available = False
 
 # make sure pandas exists before using it
 if importlib.util.find_spec("pandas"):
@@ -78,17 +79,26 @@ if importlib.util.find_spec("pyspark.pandas"):
   import pyspark.pandas
   pyspark_available = True
 
-# preferentially convert pandas DataFrames to pandas-on-Spark DataFrames first
+# make sure databricks.koalas exists before using it
+if importlib.util.find_spec("databricks.koalas"):
+  import databricks.koalas
+  koalas_available = True
+
+# preferentially convert pandas DataFrames to pandas-on-Spark or Koalas DataFrames first
 # since they know how to convert pandas DataFrames better than `spark.createDataFrame(df)`
 # and converting from pandas-on-Spark to Spark DataFrame has no overhead
 if pyspark_available and pandas_available and isinstance(df, pandas.core.frame.DataFrame):
   df = pyspark.pandas.frame.DataFrame(df)
+elif koalas_available and pandas_available and isinstance(df, pandas.core.frame.DataFrame):
+  df = databricks.koalas.frame.DataFrame(df)
 
 # convert to pyspark.sql.dataframe.DataFrame
 if isinstance(df, pyspark.sql.dataframe.DataFrame):
   pass  # since it is already a Spark DataFrame
 elif pyspark_available and isinstance(df, pyspark.pandas.frame.DataFrame):
   df = df.to_spark()
+elif koalas_available and isinstance(df, databricks.koalas.frame.DataFrame):
+  df = df.to_spark()
 elif pandas_available and isinstance(df, pandas.core.frame.DataFrame):
   df = spark.createDataFrame(df)
 else:

From 8b354f9f8d6d7cf47ac6f9deb16204394316254a Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 28 Sep 2022 11:52:10 -0500
Subject: [PATCH 490/860] CT-1166 bigquery conversion of test_alias  (#318)

* init pr, ipmorting test from core

* referencing new Base class in core instead of tests

* readding ref to MACROS__EXPECT_VALUE_SQL as seems to be overwritten when adding new bigquery macro

* remove old integration test

* swapping schema ref to database ref

* minor import change, rechanging back to uique_schema version of test pulled in from core

* rolling back test to unforked state will document thoughts on what problem is to raise as this may tie into other tests having hanging tables

* change core pointer

* change core pointer

Co-authored-by: Chenyu Li <chenyu.li@dbtlabs.com>
---
 tests/functional/adapter/test_aliases.py      | 48 +++++++++++++
 .../integration/aliases_test/macros/cast.sql  | 13 ----
 .../aliases_test/macros/expect_value.sql      | 10 ---
 .../models-dupe-custom-database/README.md     |  2 -
 .../models-dupe-custom-database/model_a.sql   |  1 -
 .../models-dupe-custom-database/model_b.sql   |  1 -
 .../models-dupe-custom-database/schema.yml    | 12 ----
 .../aliases_test/models/alias_in_project.sql  |  2 -
 .../models/alias_in_project_with_override.sql |  4 --
 .../aliases_test/models/foo_alias.sql         |  9 ---
 .../aliases_test/models/ref_foo_alias.sql     | 16 -----
 .../aliases_test/models/schema.yml            | 22 ------
 .../integration/aliases_test/test_aliases.py  | 70 -------------------
 13 files changed, 48 insertions(+), 162 deletions(-)
 create mode 100644 tests/functional/adapter/test_aliases.py
 delete mode 100644 tests/integration/aliases_test/macros/cast.sql
 delete mode 100644 tests/integration/aliases_test/macros/expect_value.sql
 delete mode 100644 tests/integration/aliases_test/models-dupe-custom-database/README.md
 delete mode 100644 tests/integration/aliases_test/models-dupe-custom-database/model_a.sql
 delete mode 100644 tests/integration/aliases_test/models-dupe-custom-database/model_b.sql
 delete mode 100644 tests/integration/aliases_test/models-dupe-custom-database/schema.yml
 delete mode 100644 tests/integration/aliases_test/models/alias_in_project.sql
 delete mode 100644 tests/integration/aliases_test/models/alias_in_project_with_override.sql
 delete mode 100644 tests/integration/aliases_test/models/foo_alias.sql
 delete mode 100644 tests/integration/aliases_test/models/ref_foo_alias.sql
 delete mode 100644 tests/integration/aliases_test/models/schema.yml
 delete mode 100644 tests/integration/aliases_test/test_aliases.py

diff --git a/tests/functional/adapter/test_aliases.py b/tests/functional/adapter/test_aliases.py
new file mode 100644
index 000000000..2b4cdd381
--- /dev/null
+++ b/tests/functional/adapter/test_aliases.py
@@ -0,0 +1,48 @@
+import pytest
+import os
+from dbt.tests.adapter.aliases.test_aliases import BaseAliases, BaseSameAliasDifferentDatabases
+
+MACROS__BIGQUERY_CAST_SQL = """
+
+{% macro bigquery__string_literal(s) %}
+    cast('{{ s }}' as string)
+{% endmacro %}
+
+"""
+
+MACROS__EXPECT_VALUE_SQL = """
+
+-- cross-db compatible test, similar to accepted_values
+
+{% test expect_value(model, field, value) %}
+
+select *
+from {{ model }}
+where {{ field }} != '{{ value }}'
+
+{% endtest %}
+
+"""
+
+class TestAliasesBigQuery(BaseAliases):
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {"bigquery_cast.sql": MACROS__BIGQUERY_CAST_SQL, "expect_value.sql": MACROS__EXPECT_VALUE_SQL}
+
+
+class TestSameTestSameAliasDifferentDatabasesBigQuery(BaseSameAliasDifferentDatabases):
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            "config-version": 2,
+            "macro-paths": ["macros"],
+            "models": {
+                "test": {
+                    "alias": "duped_alias",
+                    "model_b": {"database": os.getenv("BIGQUERY_TEST_ALT_DATABASE")},
+                },
+            },
+        }
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {"bigquery_cast.sql": MACROS__BIGQUERY_CAST_SQL, "expect_value.sql": MACROS__EXPECT_VALUE_SQL}
diff --git a/tests/integration/aliases_test/macros/cast.sql b/tests/integration/aliases_test/macros/cast.sql
deleted file mode 100644
index 584492077..000000000
--- a/tests/integration/aliases_test/macros/cast.sql
+++ /dev/null
@@ -1,13 +0,0 @@
-
-
-{% macro string_literal(s) -%}
-  {{ adapter.dispatch('string_literal', macro_namespace='test')(s) }}
-{%- endmacro %}
-
-{% macro default__string_literal(s) %}
-    '{{ s }}'::text
-{% endmacro %}
-
-{% macro bigquery__string_literal(s) %}
-    cast('{{ s }}' as string)
-{% endmacro %}
diff --git a/tests/integration/aliases_test/macros/expect_value.sql b/tests/integration/aliases_test/macros/expect_value.sql
deleted file mode 100644
index 0ee66151f..000000000
--- a/tests/integration/aliases_test/macros/expect_value.sql
+++ /dev/null
@@ -1,10 +0,0 @@
-
--- cross-db compatible test, similar to accepted_values
-
-{% test expect_value(model, field, value) %}
-
-select *
-from {{ model }}
-where {{ field }} != '{{ value }}'
-
-{% endtest %}
diff --git a/tests/integration/aliases_test/models-dupe-custom-database/README.md b/tests/integration/aliases_test/models-dupe-custom-database/README.md
deleted file mode 100644
index 7d0add229..000000000
--- a/tests/integration/aliases_test/models-dupe-custom-database/README.md
+++ /dev/null
@@ -1,2 +0,0 @@
-these should succeed, as both models have the same alias,
-but they are configured to be built in _different_ schemas
diff --git a/tests/integration/aliases_test/models-dupe-custom-database/model_a.sql b/tests/integration/aliases_test/models-dupe-custom-database/model_a.sql
deleted file mode 100644
index 9ce91382c..000000000
--- a/tests/integration/aliases_test/models-dupe-custom-database/model_a.sql
+++ /dev/null
@@ -1 +0,0 @@
-select {{ string_literal(this.name) }} as tablename
diff --git a/tests/integration/aliases_test/models-dupe-custom-database/model_b.sql b/tests/integration/aliases_test/models-dupe-custom-database/model_b.sql
deleted file mode 100644
index 9ce91382c..000000000
--- a/tests/integration/aliases_test/models-dupe-custom-database/model_b.sql
+++ /dev/null
@@ -1 +0,0 @@
-select {{ string_literal(this.name) }} as tablename
diff --git a/tests/integration/aliases_test/models-dupe-custom-database/schema.yml b/tests/integration/aliases_test/models-dupe-custom-database/schema.yml
deleted file mode 100644
index a7613882c..000000000
--- a/tests/integration/aliases_test/models-dupe-custom-database/schema.yml
+++ /dev/null
@@ -1,12 +0,0 @@
-version: 2
-models:
-- name: model_a
-  tests:
-  - expect_value:
-      field: tablename
-      value: duped_alias
-- name: model_b
-  tests:
-  - expect_value:
-      field: tablename
-      value: duped_alias
diff --git a/tests/integration/aliases_test/models/alias_in_project.sql b/tests/integration/aliases_test/models/alias_in_project.sql
deleted file mode 100644
index aa9ecd0bf..000000000
--- a/tests/integration/aliases_test/models/alias_in_project.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-
-select {{ string_literal(this.name) }} as tablename
diff --git a/tests/integration/aliases_test/models/alias_in_project_with_override.sql b/tests/integration/aliases_test/models/alias_in_project_with_override.sql
deleted file mode 100644
index 67cb93d93..000000000
--- a/tests/integration/aliases_test/models/alias_in_project_with_override.sql
+++ /dev/null
@@ -1,4 +0,0 @@
-
-{{ config(alias='override_alias') }}
-
-select {{ string_literal(this.name) }} as tablename
diff --git a/tests/integration/aliases_test/models/foo_alias.sql b/tests/integration/aliases_test/models/foo_alias.sql
deleted file mode 100644
index 3f7d16467..000000000
--- a/tests/integration/aliases_test/models/foo_alias.sql
+++ /dev/null
@@ -1,9 +0,0 @@
-
-{{
-    config(
-        alias='foo',
-        materialized='table'
-    )
-}}
-
-select {{ string_literal(this.name) }} as tablename
diff --git a/tests/integration/aliases_test/models/ref_foo_alias.sql b/tests/integration/aliases_test/models/ref_foo_alias.sql
deleted file mode 100644
index e01463bb7..000000000
--- a/tests/integration/aliases_test/models/ref_foo_alias.sql
+++ /dev/null
@@ -1,16 +0,0 @@
-
-{{
-    config(
-        materialized='table'
-    )
-}}
-
-with trigger_ref as (
-
-  -- we should still be able to ref a model by its filepath
-  select * from {{ ref('foo_alias') }}
-
-)
-
--- this name should still be the filename
-select {{ string_literal(this.name) }} as tablename
diff --git a/tests/integration/aliases_test/models/schema.yml b/tests/integration/aliases_test/models/schema.yml
deleted file mode 100644
index b3a82faad..000000000
--- a/tests/integration/aliases_test/models/schema.yml
+++ /dev/null
@@ -1,22 +0,0 @@
-version: 2
-models:
-- name: foo_alias
-  tests:
-  - expect_value:
-      field: tablename
-      value: foo
-- name: ref_foo_alias
-  tests:
-  - expect_value:
-      field: tablename
-      value: ref_foo_alias
-- name: alias_in_project
-  tests:
-  - expect_value:
-      field: tablename
-      value: project_alias
-- name: alias_in_project_with_override
-  tests:
-  - expect_value:
-      field: tablename
-      value: override_alias
diff --git a/tests/integration/aliases_test/test_aliases.py b/tests/integration/aliases_test/test_aliases.py
deleted file mode 100644
index 404c89499..000000000
--- a/tests/integration/aliases_test/test_aliases.py
+++ /dev/null
@@ -1,70 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-
-class TestAliases(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "aliases"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            "macro-paths": ['macros'],
-            "models": {
-                "test": {
-                    "alias_in_project": {
-                        "alias": 'project_alias',
-                    },
-                    "alias_in_project_with_override": {
-                        "alias": 'project_alias',
-                    },
-                }
-            }
-        }
-
-    @use_profile('bigquery')
-    def test__alias_model_name_bigquery(self):
-        results = self.run_dbt(['run'])
-        self.assertEqual(len(results), 4)
-        self.run_dbt(['test'])
-
-
-class TestSameAliasDifferentDatabases(DBTIntegrationTest):
-    setup_alternate_db = True
-
-    @property
-    def schema(self):
-        return "aliases_026"
-
-    @property
-    def models(self):
-        return "models-dupe-custom-database"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            "macro-paths": ['macros'],
-            'models': {
-                'test': {
-                    'alias': 'duped_alias',
-                    'model_b': {
-                        'database': self.alternative_database,
-                    },
-                },
-            }
-        }
-
-    @use_profile('bigquery')
-    def test__bigquery_same_alias_succeeds_in_different_schemas(self):
-        results = self.run_dbt(['run'])
-        self.assertEqual(len(results), 2)
-        res = self.run_dbt(['test'])
-
-        # Make extra sure the tests ran
-        self.assertTrue(len(res) > 0)

From 4d4ef84661fb570a75ae3a893f813972c9f90b94 Mon Sep 17 00:00:00 2001
From: Christophe Oudar <kayrnt@gmail.com>
Date: Wed, 28 Sep 2022 19:40:55 +0200
Subject: [PATCH 491/860] Update Readme to link to the repository
 CONTRIBUTING.md (#252)

* Update Readme to link to the repository CONTRIBUTING.md

* review changes

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 README.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/README.md b/README.md
index 976bdf918..6ec77bd2e 100644
--- a/README.md
+++ b/README.md
@@ -32,7 +32,7 @@ more information on using dbt with BigQuery, consult [the docs](https://docs.get
 ## Reporting bugs and contributing code
 
 - Want to report a bug or request a feature? Let us know on [Slack](http://community.getdbt.com/), or open [an issue](https://github.com/dbt-labs/dbt-bigquery/issues/new)
-- Want to help us build dbt? Check out the [Contributing Guide](https://github.com/dbt-labs/dbt/blob/HEAD/CONTRIBUTING.md)
+- Want to help us build dbt? Check out the [Contributing Guide](https://github.com/dbt-labs/dbt-bigquery/blob/HEAD/CONTRIBUTING.md)
 
 ## Code of Conduct
 

From ac4cee86a7217952c7ce864267890a99534f6f09 Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Wed, 28 Sep 2022 14:36:57 -0600
Subject: [PATCH 492/860] try/except rather than find_spec for optional imports
 (#328)

* try/except rather than find_spec for optional imports

* Remove unused import
---
 .../bigquery/macros/materializations/table.sql | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)

diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index 1bea3f487..441b97d28 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -63,26 +63,26 @@ df = model(dbt, spark)
 # COMMAND ----------
 # this is materialization code dbt generated, please do not modify
 
-import importlib.util
-
-pandas_available = False
-pyspark_available = False
-koalas_available = False
-
 # make sure pandas exists before using it
-if importlib.util.find_spec("pandas"):
+try:
   import pandas
   pandas_available = True
+except ImportError:
+  pandas_available = False
 
 # make sure pyspark.pandas exists before using it
-if importlib.util.find_spec("pyspark.pandas"):
+try:
   import pyspark.pandas
   pyspark_available = True
+except ImportError:
+  pyspark_available = False
 
 # make sure databricks.koalas exists before using it
-if importlib.util.find_spec("databricks.koalas"):
+try:
   import databricks.koalas
   koalas_available = True
+except ImportError:
+  koalas_available = False
 
 # preferentially convert pandas DataFrames to pandas-on-Spark or Koalas DataFrames first
 # since they know how to convert pandas DataFrames better than `spark.createDataFrame(df)`

From e7658fda1b9de15d6269715e5282f66b7c3d914f Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Thu, 29 Sep 2022 09:18:14 -0400
Subject: [PATCH 493/860] Bumping version to 1.4.0a1 and generate changelog
 (#331)

* Bumping version to 1.4.0a1 and generate CHANGELOG

* Fix up changelog

Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Leah Antkiewicz <leah.antkiewicz@fishtownanalytics.com>
---
 .bumpversion.cfg                                 |  2 +-
 .changes/1.3.0-b1.md                             |  6 ------
 .changes/1.3.0-b2.md                             | 10 ----------
 .changes/1.3.0/Features-20220804-154944.yaml     |  8 --------
 .changes/1.3.0/Features-20220804-155057.yaml     |  7 -------
 .../1.3.0/Under the Hood-20220804-155219.yaml    |  7 -------
 .../1.3.0/Under the Hood-20220804-155719.yaml    |  7 -------
 .../1.3.0/Under the Hood-20220806-142912.yaml    |  7 -------
 .../1.3.0/Under the Hood-20220817-154151.yaml    |  7 -------
 .../1.3.0/Under the Hood-20220825-073235.yaml    |  7 -------
 .../unreleased/Dependency-20220902-152810.yaml   |  7 -------
 .../unreleased/Dependency-20220920-200205.yaml   |  7 -------
 .../unreleased/Features-20220909-122924.yaml     |  7 -------
 .../unreleased/Features-20220913-084836.yaml     |  7 -------
 .../unreleased/Features-20220919-232721.yaml     |  7 -------
 .../Under the Hood-20220908-143218.yaml          |  7 -------
 .../Under the Hood-20220915-145212.yaml          |  7 -------
 .../Under the Hood-20220920-151057.yaml          |  7 -------
 .../Under the Hood-20220924-134858.yaml          |  7 -------
 CHANGELOG.md                                     | 16 +---------------
 dbt/adapters/bigquery/__version__.py             |  2 +-
 setup.py                                         |  2 +-
 22 files changed, 4 insertions(+), 147 deletions(-)
 delete mode 100644 .changes/1.3.0-b1.md
 delete mode 100644 .changes/1.3.0-b2.md
 delete mode 100644 .changes/1.3.0/Features-20220804-154944.yaml
 delete mode 100644 .changes/1.3.0/Features-20220804-155057.yaml
 delete mode 100644 .changes/1.3.0/Under the Hood-20220804-155219.yaml
 delete mode 100644 .changes/1.3.0/Under the Hood-20220804-155719.yaml
 delete mode 100644 .changes/1.3.0/Under the Hood-20220806-142912.yaml
 delete mode 100644 .changes/1.3.0/Under the Hood-20220817-154151.yaml
 delete mode 100644 .changes/1.3.0/Under the Hood-20220825-073235.yaml
 delete mode 100644 .changes/unreleased/Dependency-20220902-152810.yaml
 delete mode 100644 .changes/unreleased/Dependency-20220920-200205.yaml
 delete mode 100644 .changes/unreleased/Features-20220909-122924.yaml
 delete mode 100644 .changes/unreleased/Features-20220913-084836.yaml
 delete mode 100644 .changes/unreleased/Features-20220919-232721.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20220908-143218.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20220915-145212.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20220920-151057.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20220924-134858.yaml

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index d3aa82250..047a5cd7d 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.3.0b2
+current_version = 1.4.0a1
 parse = (?P<major>\d+)
 	\.(?P<minor>\d+)
 	\.(?P<patch>\d+)
diff --git a/.changes/1.3.0-b1.md b/.changes/1.3.0-b1.md
deleted file mode 100644
index 9d296a9bb..000000000
--- a/.changes/1.3.0-b1.md
+++ /dev/null
@@ -1,6 +0,0 @@
-## dbt-bigquery 1.3.0-b1 - August 04, 2022
-### Features
-- Implement `create_schema` via SQL, instead of Python method, allowing users to override if desired. drop_schema remains a Python method for the time being.  ([#182](https://github.com/dbt-labs/dbt-bigquery/issues/182), [#183](https://github.com/dbt-labs/dbt-bigquery/pull/183))
-- Added table and incrementail materializations for python models via DataProc. ([#209](https://github.com/dbt-labs/dbt-bigquery/issues/209), [#226](https://github.com/dbt-labs/dbt-bigquery/pull/226))
-### Under the Hood
-- Implement minimal changes to support dbt Core incremental materialization refactor. ([#232](https://github.com/dbt-labs/dbt-bigquery/issues/232), [#223](https://github.com/dbt-labs/dbt-bigquery/pull/223))
diff --git a/.changes/1.3.0-b2.md b/.changes/1.3.0-b2.md
deleted file mode 100644
index a62c278cc..000000000
--- a/.changes/1.3.0-b2.md
+++ /dev/null
@@ -1,10 +0,0 @@
-## dbt-bigquery 1.3.0-b2 - August 29, 2022
-### Under the Hood
-- Add changie to dbt-bigquery ([#254](https://github.com/dbt-labs/dbt-bigquery/issues/254), [#253](https://github.com/dbt-labs/dbt-bigquery/pull/253))
-- Add location/job_id/project_id to adapter response to enable easy job linking ([#92](https://github.com/dbt-labs/dbt-bigquery/issues/92), [#250](https://github.com/dbt-labs/dbt-bigquery/pull/250))
-- Adding `slot_ms` go `BigQueryAdapterResponse` ([#194](https://github.com/dbt-labs/dbt-bigquery/issues/194), [#195](https://github.com/dbt-labs/dbt-bigquery/pull/195))
-- specify supported_languages for materialization that support python models ([#288](https://github.com/dbt-labs/dbt-bigquery/issues/288), [#290](https://github.com/dbt-labs/dbt-bigquery/pull/290))
-
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#250](https://github.com/dbt-labs/dbt-bigquery/pull/250))
-- [@yu-iskw](https://github.com/yu-iskw) ([#195](https://github.com/dbt-labs/dbt-bigquery/pull/195))
diff --git a/.changes/1.3.0/Features-20220804-154944.yaml b/.changes/1.3.0/Features-20220804-154944.yaml
deleted file mode 100644
index 65bd2b32e..000000000
--- a/.changes/1.3.0/Features-20220804-154944.yaml
+++ /dev/null
@@ -1,8 +0,0 @@
-kind: Features
-body: 'Implement `create_schema` via SQL, instead of Python method, allowing users
-  to override if desired. drop_schema remains a Python method for the time being. '
-time: 2022-08-04T15:49:44.409354-05:00
-custom:
-  Author: jtcohen6
-  Issue: "182"
-  PR: "183"
diff --git a/.changes/1.3.0/Features-20220804-155057.yaml b/.changes/1.3.0/Features-20220804-155057.yaml
deleted file mode 100644
index 39bd6ad45..000000000
--- a/.changes/1.3.0/Features-20220804-155057.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: Added table and incrementail materializations for python models via DataProc.
-time: 2022-08-04T15:50:57.234209-05:00
-custom:
-  Author: stu-k
-  Issue: "209"
-  PR: "226"
diff --git a/.changes/1.3.0/Under the Hood-20220804-155219.yaml b/.changes/1.3.0/Under the Hood-20220804-155219.yaml
deleted file mode 100644
index f5e6b2d9c..000000000
--- a/.changes/1.3.0/Under the Hood-20220804-155219.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: Implement minimal changes to support dbt Core incremental materialization refactor.
-time: 2022-08-04T15:52:19.342843-05:00
-custom:
-  Author: gshank
-  Issue: "232"
-  PR: "223"
diff --git a/.changes/1.3.0/Under the Hood-20220804-155719.yaml b/.changes/1.3.0/Under the Hood-20220804-155719.yaml
deleted file mode 100644
index 631333c02..000000000
--- a/.changes/1.3.0/Under the Hood-20220804-155719.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: Add changie to dbt-bigquery
-time: 2022-08-04T15:57:19.060995-05:00
-custom:
-  Author: mcknight-42
-  Issue: "254"
-  PR: "253"
diff --git a/.changes/1.3.0/Under the Hood-20220806-142912.yaml b/.changes/1.3.0/Under the Hood-20220806-142912.yaml
deleted file mode 100644
index 6581f8e22..000000000
--- a/.changes/1.3.0/Under the Hood-20220806-142912.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: Add location/job_id/project_id to adapter response to enable easy job linking
-time: 2022-08-06T14:29:12.271054+02:00
-custom:
-  Author: Kayrnt
-  Issue: "92"
-  PR: "250"
diff --git a/.changes/1.3.0/Under the Hood-20220817-154151.yaml b/.changes/1.3.0/Under the Hood-20220817-154151.yaml
deleted file mode 100644
index d3de5a992..000000000
--- a/.changes/1.3.0/Under the Hood-20220817-154151.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: Adding `slot_ms` go `BigQueryAdapterResponse`
-time: 2022-08-17T15:41:51.080936-05:00
-custom:
-  Author: yu-iskw
-  Issue: "194"
-  PR: "195"
diff --git a/.changes/1.3.0/Under the Hood-20220825-073235.yaml b/.changes/1.3.0/Under the Hood-20220825-073235.yaml
deleted file mode 100644
index f33d77173..000000000
--- a/.changes/1.3.0/Under the Hood-20220825-073235.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: specify supported_languages for materialization that support python models
-time: 2022-08-25T07:32:35.820396-07:00
-custom:
-  Author: ChenyuLInx
-  Issue: "288"
-  PR: "290"
diff --git a/.changes/unreleased/Dependency-20220902-152810.yaml b/.changes/unreleased/Dependency-20220902-152810.yaml
deleted file mode 100644
index 2d7fb11ef..000000000
--- a/.changes/unreleased/Dependency-20220902-152810.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Dependency
-body: "Bump black from 22.6.0 to 22.8.0"
-time: 2022-09-02T15:28:10.00000Z
-custom:
-  Author: dependabot[bot]
-  Issue: 254
-  PR: 294
diff --git a/.changes/unreleased/Dependency-20220920-200205.yaml b/.changes/unreleased/Dependency-20220920-200205.yaml
deleted file mode 100644
index 6dcfa5924..000000000
--- a/.changes/unreleased/Dependency-20220920-200205.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Dependency
-body: "Bump mypy from 0.942 to 0.971"
-time: 2022-09-20T20:02:05.00000Z
-custom:
-  Author: dependabot[bot]
-  Issue: 254
-  PR: 296
diff --git a/.changes/unreleased/Features-20220909-122924.yaml b/.changes/unreleased/Features-20220909-122924.yaml
deleted file mode 100644
index cde9bbb43..000000000
--- a/.changes/unreleased/Features-20220909-122924.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: Add support for Dataproc Serverless
-time: 2022-09-09T12:29:24.993388-07:00
-custom:
-  Author: ChenyuLInx
-  Issue: "248"
-  PR: "303"
diff --git a/.changes/unreleased/Features-20220913-084836.yaml b/.changes/unreleased/Features-20220913-084836.yaml
deleted file mode 100644
index 0a4bd275d..000000000
--- a/.changes/unreleased/Features-20220913-084836.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: Array macros
-time: 2022-09-13T08:48:36.255365-06:00
-custom:
-  Author: graciegoheen dbeatty10
-  Issue: "307"
-  PR: "308"
diff --git a/.changes/unreleased/Features-20220919-232721.yaml b/.changes/unreleased/Features-20220919-232721.yaml
deleted file mode 100644
index 4f5c90dd0..000000000
--- a/.changes/unreleased/Features-20220919-232721.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: add type_boolean as a data type macro
-time: 2022-09-19T23:27:21.864912+01:00
-custom:
-  Author: jpmmcneill
-  Issue: "315"
-  PR: "313"
diff --git a/.changes/unreleased/Under the Hood-20220908-143218.yaml b/.changes/unreleased/Under the Hood-20220908-143218.yaml
deleted file mode 100644
index 7053f96b6..000000000
--- a/.changes/unreleased/Under the Hood-20220908-143218.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: Convert df to pyspark DataFrame if it is pandas before writing
-time: 2022-09-08T14:32:18.243412-04:00
-custom:
-  Author: chamini2
-  Issue: "301"
-  PR: "301"
diff --git a/.changes/unreleased/Under the Hood-20220915-145212.yaml b/.changes/unreleased/Under the Hood-20220915-145212.yaml
deleted file mode 100644
index 6ab035256..000000000
--- a/.changes/unreleased/Under the Hood-20220915-145212.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: Update BQ job and call retry settings
-time: 2022-09-15T14:52:12.902965-07:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "311"
-  PR: "310"
diff --git a/.changes/unreleased/Under the Hood-20220920-151057.yaml b/.changes/unreleased/Under the Hood-20220920-151057.yaml
deleted file mode 100644
index 86914c4f3..000000000
--- a/.changes/unreleased/Under the Hood-20220920-151057.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: Enable pandas-on-Spark DataFrames for dbt python models
-time: 2022-09-20T15:10:57.712169-06:00
-custom:
-  Author: dbeatty10
-  Issue: "316"
-  PR: "317"
diff --git a/.changes/unreleased/Under the Hood-20220924-134858.yaml b/.changes/unreleased/Under the Hood-20220924-134858.yaml
deleted file mode 100644
index a4fbceb3f..000000000
--- a/.changes/unreleased/Under the Hood-20220924-134858.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: Convert df to pyspark DataFrame if it is koalas before writing
-time: 2022-09-24T13:48:58.614746-06:00
-custom:
-  Author: dbeatty10 ueshin
-  Issue: "320"
-  PR: "321"
diff --git a/CHANGELOG.md b/CHANGELOG.md
index a9038a76d..73a284e72 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -4,24 +4,10 @@
 - Changes are listed under the (pre)release in which they first appear. Subsequent releases include changes from previous releases.
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
-## dbt-bigquery 1.3.0-b2 - August 29, 2022
-### Under the Hood
-- Add changie to dbt-bigquery ([#254](https://github.com/dbt-labs/dbt-bigquery/issues/254), [#253](https://github.com/dbt-labs/dbt-bigquery/pull/253))
-- Add location/job_id/project_id to adapter response to enable easy job linking ([#92](https://github.com/dbt-labs/dbt-bigquery/issues/92), [#250](https://github.com/dbt-labs/dbt-bigquery/pull/250))
-- Adding `slot_ms` go `BigQueryAdapterResponse` ([#194](https://github.com/dbt-labs/dbt-bigquery/issues/194), [#195](https://github.com/dbt-labs/dbt-bigquery/pull/195))
-- specify supported_languages for materialization that support python models ([#288](https://github.com/dbt-labs/dbt-bigquery/issues/288), [#290](https://github.com/dbt-labs/dbt-bigquery/pull/290))
 
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#250](https://github.com/dbt-labs/dbt-bigquery/pull/250))
-- [@yu-iskw](https://github.com/yu-iskw) ([#195](https://github.com/dbt-labs/dbt-bigquery/pull/195))
-## dbt-bigquery 1.3.0-b1 - August 04, 2022
-### Features
-- Implement `create_schema` via SQL, instead of Python method, allowing users to override if desired. drop_schema remains a Python method for the time being.  ([#182](https://github.com/dbt-labs/dbt-bigquery/issues/182), [#183](https://github.com/dbt-labs/dbt-bigquery/pull/183))
-- Added table and incrementail materializations for python models via DataProc. ([#209](https://github.com/dbt-labs/dbt-bigquery/issues/209), [#226](https://github.com/dbt-labs/dbt-bigquery/pull/226))
-### Under the Hood
-- Implement minimal changes to support dbt Core incremental materialization refactor. ([#232](https://github.com/dbt-labs/dbt-bigquery/issues/232), [#223](https://github.com/dbt-labs/dbt-bigquery/pull/223))
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
+- [1.3](https://github.com/dbt-labs/dbt-bigquery/blob/1.3.latest/CHANGELOG.md)
 - [1.2](https://github.com/dbt-labs/dbt-bigquery/blob/1.2.latest/CHANGELOG.md)
 - [1.1](https://github.com/dbt-labs/dbt-bigquery/blob/1.1.latest/CHANGELOG.md)
 - [1.0](https://github.com/dbt-labs/dbt-bigquery/blob/1.0.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index e2c1a233c..70ba273f5 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.3.0b2"
+version = "1.4.0a1"
diff --git a/setup.py b/setup.py
index 81a267711..3345dbc04 100644
--- a/setup.py
+++ b/setup.py
@@ -50,7 +50,7 @@ def _get_dbt_core_version():
 
 
 package_name = "dbt-bigquery"
-package_version = "1.3.0b2"
+package_version = "1.4.0a1"
 dbt_core_version = _get_dbt_core_version()
 description = """The BigQuery adapter plugin for dbt"""
 

From 86d5dbe2c3d5ee863ef0b97a47f276cf26518c83 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Fri, 30 Sep 2022 16:37:13 -0700
Subject: [PATCH 494/860] consolidate timestamps (#323)

* update retry settings

* changelog entry

* consolidate timestamps

* add changie

* whitespace fix

* fix macro name

* update expected test fixture

* Update Features-20220926-105700.yaml

* add backcompat to test fixture

* add backcompat

* update dev-requirements

* Update change log body

* update test class name

* lowercase timestamps
---
 .../unreleased/Features-20220926-105700.yaml   |  7 +++++++
 dbt/include/bigquery/macros/adapters.sql       | 11 -----------
 .../bigquery/macros/utils/timestamps.sql       | 12 ++++++++++++
 .../adapter/utils/test_timestamps.py           | 18 ++++++++++++++++++
 4 files changed, 37 insertions(+), 11 deletions(-)
 create mode 100644 .changes/unreleased/Features-20220926-105700.yaml
 create mode 100644 dbt/include/bigquery/macros/utils/timestamps.sql
 create mode 100644 tests/functional/adapter/utils/test_timestamps.py

diff --git a/.changes/unreleased/Features-20220926-105700.yaml b/.changes/unreleased/Features-20220926-105700.yaml
new file mode 100644
index 000000000..61e0ac741
--- /dev/null
+++ b/.changes/unreleased/Features-20220926-105700.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: Migrate dbt-utils current_timestamp macros into core + adapters
+time: 2022-09-26T10:57:00.942765-07:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "324"
+  PR: "323"
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index cbfba2627..07cf3c3e5 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -109,17 +109,6 @@
 {%- endmacro %}
 
 
-{% macro bigquery__current_timestamp() -%}
-  CURRENT_TIMESTAMP()
-{%- endmacro %}
-
-
-{% macro bigquery__snapshot_string_as_time(timestamp) -%}
-    {%- set result = 'TIMESTAMP("' ~ timestamp ~ '")' -%}
-    {{ return(result) }}
-{%- endmacro %}
-
-
 {% macro bigquery__list_schemas(database) -%}
   {{ return(adapter.list_schemas(database)) }}
 {% endmacro %}
diff --git a/dbt/include/bigquery/macros/utils/timestamps.sql b/dbt/include/bigquery/macros/utils/timestamps.sql
new file mode 100644
index 000000000..cdcbfd51e
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/timestamps.sql
@@ -0,0 +1,12 @@
+{% macro bigquery__current_timestamp() -%}
+  current_timestamp()
+{%- endmacro %}
+
+{% macro bigquery__snapshot_string_as_time(timestamp) -%}
+    {%- set result = 'TIMESTAMP("' ~ timestamp ~ '")' -%}
+    {{ return(result) }}
+{%- endmacro %}
+
+{% macro bigquery__current_timestamp_backcompat() -%}
+  current_timestamp
+{%- endmacro %}
diff --git a/tests/functional/adapter/utils/test_timestamps.py b/tests/functional/adapter/utils/test_timestamps.py
new file mode 100644
index 000000000..2f35e40ee
--- /dev/null
+++ b/tests/functional/adapter/utils/test_timestamps.py
@@ -0,0 +1,18 @@
+import pytest
+from dbt.tests.adapter.utils.test_timestamps import BaseCurrentTimestamps
+
+
+class TestCurrentTimestampBigQuery(BaseCurrentTimestamps):
+    @pytest.fixture(scope="class")
+    def expected_schema(self):
+        return {
+            "current_timestamp": "TIMESTAMP",
+            "current_timestamp_in_utc_backcompat": "TIMESTAMP",
+            "current_timestamp_backcompat": "TIMESTAMP",
+        }
+
+    @pytest.fixture(scope="class")
+    def expected_sql(self):
+        return """select current_timestamp() as current_timestamp,
+                current_timestamp as current_timestamp_in_utc_backcompat,
+                current_timestamp as current_timestamp_backcompat"""
\ No newline at end of file

From 0410e592a348f866c0776fd99db73b133445da4f Mon Sep 17 00:00:00 2001
From: Chenyu Li <chenyu.li@dbtlabs.com>
Date: Tue, 4 Oct 2022 16:47:31 -0700
Subject: [PATCH 495/860] fix operation wait for cluster submission (#338)

* fix operation wait for cluster

* remove unused Variable
---
 .pre-commit-config.yaml                         |  2 +-
 dbt/adapters/bigquery/python_submissions.py     | 17 ++++++++++++++++-
 .../bigquery/macros/materializations/table.sql  |  9 +++++----
 3 files changed, 22 insertions(+), 6 deletions(-)

diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index cf4ddf68a..e366015f2 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -4,7 +4,7 @@
 exclude: '^tests/.*'
 
 default_language_version:
-  python: python3.8
+  python: python3
 
 repos:
 - repo: https://github.com/pre-commit/pre-commit-hooks
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 8d12fa2cc..d0cf9c275 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -1,4 +1,5 @@
 from typing import Dict, Union
+import time
 
 from dbt.adapters.base import PythonJobHelper
 from dbt.adapters.bigquery import BigQueryConnectionManager, BigQueryCredentials
@@ -6,6 +7,8 @@
 from google.api_core.client_options import ClientOptions
 from google.cloud import storage, dataproc_v1  # type: ignore
 
+OPERATION_RETRY_TIME = 10
+
 
 class BaseDataProcHelper(PythonJobHelper):
     def __init__(self, parsed_model: Dict, credential: BigQueryCredentials) -> None:
@@ -64,6 +67,14 @@ def _get_job_client(
     def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         raise NotImplementedError("_submit_dataproc_job not implemented")
 
+    def _wait_operation(self, operation):
+        # can't use due to https://github.com/googleapis/python-api-core/issues/458
+        # response = operation.result(retry=self.retry)
+        # Temp solution to wait for the job to finish
+        start = time.time()
+        while not operation.done(retry=None) and time.time() - start < self.timeout:
+            time.sleep(OPERATION_RETRY_TIME)
+
 
 class ClusterDataprocHelper(BaseDataProcHelper):
     def _get_job_client(self) -> dataproc_v1.JobControllerClient:
@@ -94,7 +105,11 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
                 "job": job,
             }
         )
-        response = operation.result(retry=self.retry)
+        self._wait_operation(operation)
+        response = operation.metadata
+        # check if job failed
+        if response.status.state == 6:
+            raise ValueError(response.status.details)
         return response
 
 
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index 441b97d28..c8c88aa3c 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -63,6 +63,7 @@ df = model(dbt, spark)
 # COMMAND ----------
 # this is materialization code dbt generated, please do not modify
 
+import pyspark
 # make sure pandas exists before using it
 try:
   import pandas
@@ -73,9 +74,9 @@ except ImportError:
 # make sure pyspark.pandas exists before using it
 try:
   import pyspark.pandas
-  pyspark_available = True
+  pyspark_pandas_api_available = True
 except ImportError:
-  pyspark_available = False
+  pyspark_pandas_api_available = False
 
 # make sure databricks.koalas exists before using it
 try:
@@ -87,7 +88,7 @@ except ImportError:
 # preferentially convert pandas DataFrames to pandas-on-Spark or Koalas DataFrames first
 # since they know how to convert pandas DataFrames better than `spark.createDataFrame(df)`
 # and converting from pandas-on-Spark to Spark DataFrame has no overhead
-if pyspark_available and pandas_available and isinstance(df, pandas.core.frame.DataFrame):
+if pyspark_pandas_api_available and pandas_available and isinstance(df, pandas.core.frame.DataFrame):
   df = pyspark.pandas.frame.DataFrame(df)
 elif koalas_available and pandas_available and isinstance(df, pandas.core.frame.DataFrame):
   df = databricks.koalas.frame.DataFrame(df)
@@ -95,7 +96,7 @@ elif koalas_available and pandas_available and isinstance(df, pandas.core.frame.
 # convert to pyspark.sql.dataframe.DataFrame
 if isinstance(df, pyspark.sql.dataframe.DataFrame):
   pass  # since it is already a Spark DataFrame
-elif pyspark_available and isinstance(df, pyspark.pandas.frame.DataFrame):
+elif pyspark_pandas_api_available and isinstance(df, pyspark.pandas.frame.DataFrame):
   df = df.to_spark()
 elif koalas_available and isinstance(df, databricks.koalas.frame.DataFrame):
   df = df.to_spark()

From 6ce3a3dcda46528e831d09f426d4cd74f6dd6fa4 Mon Sep 17 00:00:00 2001
From: aolalere-lbg <adeola.olalere.eplus@gmail.com>
Date: Wed, 12 Oct 2022 03:01:11 +0100
Subject: [PATCH 496/860] CT-1043 Support google-cloud-bigquery 3.x.x
 dependency (#332)

* CT-1043 Support google-cloud-bigquery 3.x.x dependency

* CT-1043 Support google-cloud-bigquery 3.x.x dependency.

* CT-1043 update of google-cloud-bigquery upper bound to the current latest of 3.3.3

* changelog entry for PR 332

* updated description in changelog entry

* updated description in changelog entry

* Update .changes/unreleased/Dependencies-20221010-181501.yaml

Addressed comment from maintainer

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>

* Update .changes/unreleased/Dependencies-20221010-181501.yaml

Addressed new comment from maintainer

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>

Co-authored-by: Daniele Casal <91036734+d-casal@users.noreply.github.com>
Co-authored-by: Daniele Casal <daniele.casal@lloydsbanking.com>
Co-authored-by: Daniele Casal <daniele.casal@hotmail.com>
Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20221010-181501.yaml | 7 +++++++
 setup.py                                              | 2 +-
 2 files changed, 8 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20221010-181501.yaml

diff --git a/.changes/unreleased/Dependencies-20221010-181501.yaml b/.changes/unreleased/Dependencies-20221010-181501.yaml
new file mode 100644
index 000000000..3fff057ad
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20221010-181501.yaml
@@ -0,0 +1,7 @@
+kind: Dependencies
+body: Bump google-cloud-bigquery dependency to < 3.3.3
+time: 2022-10-10T18:15:01.113219+01:00
+custom:
+  Author: eplus-aolalere
+  Issue: "269"
+  PR: "332"
diff --git a/setup.py b/setup.py
index 3345dbc04..b24bbe16f 100644
--- a/setup.py
+++ b/setup.py
@@ -69,7 +69,7 @@ def _get_dbt_core_version():
         "dbt-core~={}".format(dbt_core_version),
         "protobuf>=3.13.0,<4",
         "google-cloud-core>=1.3.0,<3",
-        "google-cloud-bigquery>=1.25.0,<3",
+        "google-cloud-bigquery>=1.25.0,<3.3.3",
         "google-api-core>=1.16.0,<3",
         "googleapis-common-protos>=1.6.0,<2",
         "google-cloud-storage>=2.4.0",

From ec6101113be27458a69b5241ffbe1a2330854e0d Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 12 Oct 2022 14:15:25 -0500
Subject: [PATCH 497/860] [CT-1271] Query Comment test conversion (#333)

* init pr 051 test conversion in bigquery

* update to main

* remove old test

* point back to main in core

* trying to fix the custom quota error
---
 .../query_comment_test/test_query_comment.py  |  28 +++
 .../query_comments_test/macros/macro.sql      |  25 ---
 .../query_comments_test/models/x.sql          |  24 ---
 .../test_query_comments.py                    | 169 ------------------
 4 files changed, 28 insertions(+), 218 deletions(-)
 create mode 100644 tests/functional/adapter/query_comment_test/test_query_comment.py
 delete mode 100644 tests/integration/query_comments_test/macros/macro.sql
 delete mode 100644 tests/integration/query_comments_test/models/x.sql
 delete mode 100644 tests/integration/query_comments_test/test_query_comments.py

diff --git a/tests/functional/adapter/query_comment_test/test_query_comment.py b/tests/functional/adapter/query_comment_test/test_query_comment.py
new file mode 100644
index 000000000..aa7061150
--- /dev/null
+++ b/tests/functional/adapter/query_comment_test/test_query_comment.py
@@ -0,0 +1,28 @@
+import pytest
+from dbt.tests.adapter.query_comment.test_query_comment import (
+    BaseQueryComments,
+    BaseMacroQueryComments,
+    BaseMacroArgsQueryComments,
+    BaseMacroInvalidQueryComments,
+    BaseNullQueryComments,
+    BaseEmptyQueryComments,
+)
+
+
+class TestQueryCommentsBigQuery(BaseQueryComments):
+    pass
+
+class TestMacroQueryCommentsBigQuery(BaseMacroQueryComments):
+    pass
+
+class TestMacroArgsQueryCommentsBigQuery(BaseMacroArgsQueryComments):
+    pass
+
+class TestMacroInvalidQueryCommentsBigQuery(BaseMacroInvalidQueryComments):
+    pass
+
+class TestNullQueryCommentsBigQuery(BaseNullQueryComments):
+    pass
+
+class TestEmptyQueryCommentsBigQuery(BaseEmptyQueryComments):
+    pass
\ No newline at end of file
diff --git a/tests/integration/query_comments_test/macros/macro.sql b/tests/integration/query_comments_test/macros/macro.sql
deleted file mode 100644
index 6e8a1900c..000000000
--- a/tests/integration/query_comments_test/macros/macro.sql
+++ /dev/null
@@ -1,25 +0,0 @@
-{%- macro query_header_no_args() -%}
-{%- set x = "are pretty cool" -%}
-{{ "dbt macros" }}
-{{ x }}
-{%- endmacro -%}
-
-
-{%- macro query_header_args(message) -%}
-  {%- set comment_dict = dict(
-    app='dbt++',
-    macro_version='0.1.0',
-    dbt_version=dbt_version,
-    message='blah: '~ message) -%}
-  {{ return(comment_dict) }}
-{%- endmacro -%}
-
-
-{%- macro ordered_to_json(dct) -%}
-{{ tojson(dct, sort_keys=True) }}
-{%- endmacro %}
-
-
-{% macro invalid_query_header() -%}
-{{ "Here is an invalid character for you: */" }}
-{% endmacro %}
diff --git a/tests/integration/query_comments_test/models/x.sql b/tests/integration/query_comments_test/models/x.sql
deleted file mode 100644
index e719a30b5..000000000
--- a/tests/integration/query_comments_test/models/x.sql
+++ /dev/null
@@ -1,24 +0,0 @@
-{% set blacklist = ['pass', 'password', 'keyfile', 'keyfile.json', 'password', 'private_key_passphrase'] %}
-{% for key in blacklist %}
-  {% if key in blacklist and blacklist[key] %}
-  	{% do exceptions.raise_compiler_error('invalid target, found banned key "' ~ key ~ '"') %}
-  {% endif %}
-{% endfor %}
-
-{% if 'type' not in target %}
-  {% do exceptions.raise_compiler_error('invalid target, missing "type"') %}
-{% endif %}
-
-{% set required = ['name', 'schema', 'type', 'threads'] %}
-
-{# Require what we document at https://docs.getdbt.com/docs/target #}
-{% do required.extend(['project']) %}
-
-{% for value in required %}
-	{% if value not in target %}
-  		{% do exceptions.raise_compiler_error('invalid target, missing "' ~ value ~ '"') %}
-	{% endif %}
-{% endfor %}
-
-{% do run_query('select 2 as inner_id') %}
-select 1 as outer_id
diff --git a/tests/integration/query_comments_test/test_query_comments.py b/tests/integration/query_comments_test/test_query_comments.py
deleted file mode 100644
index bc813a6ae..000000000
--- a/tests/integration/query_comments_test/test_query_comments.py
+++ /dev/null
@@ -1,169 +0,0 @@
-from tests.integration.base import DBTIntegrationTest,  use_profile
-import io
-import json
-import os
-import re
-
-import dbt.exceptions
-from dbt.version import __version__ as dbt_version
-from dbt.logger import log_manager
-
-
-class TestDefaultQueryComments(DBTIntegrationTest):
-    def matches_comment(self, msg) -> bool:
-        if not msg.startswith('/* '):
-            return False
-        # our blob is the first line of the query comments, minus the comment
-        json_str = msg.split('\n')[0][3:-3]
-        data = json.loads(json_str)
-        return (
-            data['app'] == 'dbt' and
-            data['dbt_version'] == dbt_version and
-            data['node_id'] == 'model.test.x'
-        )
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'macro-paths': ['macros']
-        }
-
-    @property
-    def schema(self):
-        return 'dbt_query_comments'
-
-    @staticmethod
-    def dir(value):
-        return os.path.normpath(value)
-
-    @property
-    def models(self):
-        return self.dir('models')
-
-    def setUp(self):
-        super().setUp()
-        self.initial_stdout = log_manager.stdout
-        self.initial_stderr = log_manager.stderr
-        self.stringbuf = io.StringIO()
-        log_manager.set_output_stream(self.stringbuf)
-
-    def tearDown(self):
-        log_manager.set_output_stream(self.initial_stdout, self.initial_stderr)
-        super().tearDown()
-
-    def run_get_json(self, expect_pass=True):
-        res, raw_logs = self.run_dbt_and_capture(
-            ['--debug', '--log-format=json', 'run'],
-            expect_pass=expect_pass
-        )
-        parsed_logs = []
-        for line in raw_logs.split('\n'):
-            try:
-                log = json.loads(line)
-            except ValueError:
-                continue
-
-            parsed_logs.append(log)
-
-        # empty lists evaluate as False
-        self.assertTrue(parsed_logs)
-        return parsed_logs
-
-    def query_comment(self, model_name, log):
-        log_msg = re.sub("(?:[01]\d|2[0123]):(?:[012345]\d):(?:[012345]\d \| )", "", log['msg'])
-        prefix = 'On {}: '.format(model_name)
-
-        if log_msg.startswith(prefix):
-            msg = log_msg[len(prefix):]
-            if msg in {'COMMIT', 'BEGIN', 'ROLLBACK'}:
-                return None
-            return msg
-        return None
-
-    def run_assert_comments(self):
-        logs = self.run_get_json()
-
-        seen = False
-        for log in logs:
-            msg = self.query_comment('model.test.x', log)
-            if msg is not None and self.matches_comment(msg):
-                seen = True
-
-        self.assertTrue(seen, 'Never saw a matching log message! Logs:\n{}'.format('\n'.join(l['msg'] for l in logs)))
-
-    @use_profile('bigquery')
-    def test_bigquery_comments(self):
-        self.run_assert_comments()
-
-
-class TestQueryComments(TestDefaultQueryComments):
-    @property
-    def project_config(self):
-        cfg = super().project_config
-        cfg.update({'query-comment': 'dbt\nrules!\n'})
-        return cfg
-
-    def matches_comment(self, msg) -> bool:
-        return msg.startswith('/* dbt\nrules! */\n')
-
-
-class TestMacroQueryComments(TestDefaultQueryComments):
-    @property
-    def project_config(self):
-        cfg = super().project_config
-        cfg.update({'query-comment': '{{ query_header_no_args() }}'})
-        return cfg
-
-    def matches_comment(self, msg) -> bool:
-        start_with = '/* dbt macros\nare pretty cool */\n'
-        return msg.startswith(start_with)
-
-
-class TestMacroArgsQueryComments(TestDefaultQueryComments):
-    @property
-    def project_config(self):
-        cfg = super().project_config
-        cfg.update(
-            {'query-comment': '{{ return(ordered_to_json(query_header_args(target.name))) }}'}
-        )
-        return cfg
-
-    def matches_comment(self, msg) -> bool:
-        expected_dct = {'app': 'dbt++', 'dbt_version': dbt_version, 'macro_version': '0.1.0', 'message': 'blah: default2'}
-        expected = '/* {} */\n'.format(json.dumps(expected_dct, sort_keys=True))
-        return msg.startswith(expected)
-
-
-class TestMacroInvalidQueryComments(TestDefaultQueryComments):
-    @property
-    def project_config(self):
-        cfg = super().project_config
-        cfg.update({'query-comment': '{{ invalid_query_header() }}'})
-        return cfg
-
-    def run_assert_comments(self):
-        with self.assertRaises(dbt.exceptions.RuntimeException):
-            self.run_get_json(expect_pass=False)
-
-
-class TestNullQueryComments(TestDefaultQueryComments):
-    @property
-    def project_config(self):
-        cfg = super().project_config
-        cfg.update({'query-comment': ''})
-        return cfg
-
-    def matches_comment(self, msg) -> bool:
-        return not ('/*' in msg or '*/' in msg)
-
-
-class TestEmptyQueryComments(TestDefaultQueryComments):
-    @property
-    def project_config(self):
-        cfg = super().project_config
-        cfg.update({'query-comment': ''})
-        return cfg
-
-    def matches_comment(self, msg) -> bool:
-        return not ('/*' in msg or '*/' in msg)

From 1762ee00ef951dcdd591d34ad8ac3a54908fc519 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 13 Oct 2022 09:01:40 -0700
Subject: [PATCH 498/860] migrate changing_relation_type test (#351)

* migrate changing_relation_type test

* Add full-refresh arg

* remove dev req change
---
 .../adapter/test_changing_relation_type.py    |  8 ++++
 .../models/model.sql                          |  8 ----
 .../test_changing_relation_type.py            | 45 -------------------
 3 files changed, 8 insertions(+), 53 deletions(-)
 create mode 100644 tests/functional/adapter/test_changing_relation_type.py
 delete mode 100644 tests/integration/changing_relation_type_test/models/model.sql
 delete mode 100644 tests/integration/changing_relation_type_test/test_changing_relation_type.py

diff --git a/tests/functional/adapter/test_changing_relation_type.py b/tests/functional/adapter/test_changing_relation_type.py
new file mode 100644
index 000000000..431e6c6ac
--- /dev/null
+++ b/tests/functional/adapter/test_changing_relation_type.py
@@ -0,0 +1,8 @@
+from dbt.tests.adapter.relations.test_changing_relation_type import BaseChangeRelationTypeValidator
+
+class TestBigQueryChangeRelationTypes(BaseChangeRelationTypeValidator):
+    def test_changing_materialization_changes_relation_type(self, project):
+        self._run_and_check_materialization('view')
+        self._run_and_check_materialization('table', extra_args=['--full-refresh'])
+        self._run_and_check_materialization('view', extra_args=['--full-refresh'])
+        self._run_and_check_materialization('incremental', extra_args=['--full-refresh'])
\ No newline at end of file
diff --git a/tests/integration/changing_relation_type_test/models/model.sql b/tests/integration/changing_relation_type_test/models/model.sql
deleted file mode 100644
index c88e66ee1..000000000
--- a/tests/integration/changing_relation_type_test/models/model.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-
-{{ config(materialized=var('materialized')) }}
-
-select '{{ var("materialized") }}' as materialization
-
-{% if var('materialized') == 'incremental' and is_incremental() %}
-    where 'abc' != (select max(materialization) from {{ this }})
-{% endif %}
diff --git a/tests/integration/changing_relation_type_test/test_changing_relation_type.py b/tests/integration/changing_relation_type_test/test_changing_relation_type.py
deleted file mode 100644
index 0c7830f1e..000000000
--- a/tests/integration/changing_relation_type_test/test_changing_relation_type.py
+++ /dev/null
@@ -1,45 +0,0 @@
-from pytest import mark
-from tests.integration.base import DBTIntegrationTest, use_profile, bigquery_rate_limiter
-
-
-class TestChangingRelationType(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "changing_relation_type"
-
-    @staticmethod
-    def dir(path):
-        return path.lstrip("/")
-
-    @property
-    def models(self):
-        return self.dir("models")
-
-    @mark.flaky(rerun_filter=bigquery_rate_limiter, max_runs=3)
-    @use_profile("bigquery")
-    def test__bigquery__switch_materialization(self):
-        # BQ has a weird check that prevents the dropping of tables in the view materialization
-        # if --full-refresh is not provided. This is to prevent the clobbering of a date-sharded
-        # table with a view if a model config is accidently changed. We should probably remove that check
-        # and then remove these bq-specific tests
-
-        results = self.run_dbt(['run', '--vars', 'materialized: view'])
-        self.assertEqual(results[0].node.config.materialized, 'view')
-        self.assertEqual(len(results),  1)
-
-        results = self.run_dbt(['run', '--vars', 'materialized: table'])
-        self.assertEqual(results[0].node.config.materialized, 'table')
-        self.assertEqual(len(results),  1)
-
-        results = self.run_dbt(['run', '--vars', 'materialized: view', "--full-refresh"])
-        self.assertEqual(results[0].node.config.materialized, 'view')
-        self.assertEqual(len(results),  1)
-
-        results = self.run_dbt(['run', '--vars', 'materialized: incremental'])
-        self.assertEqual(results[0].node.config.materialized, 'incremental')
-        self.assertEqual(len(results),  1)
-
-        results = self.run_dbt(['run', '--vars', 'materialized: view', "--full-refresh"])
-        self.assertEqual(results[0].node.config.materialized, 'view')
-        self.assertEqual(len(results),  1)

From 27e70025d9beadb3a83630035a7914925532bab3 Mon Sep 17 00:00:00 2001
From: Chenyu Li <chenyu.li@dbtlabs.com>
Date: Sun, 16 Oct 2022 08:35:50 -0700
Subject: [PATCH 499/860] update following changes in CORE#5949 (#353)

* update following changes in CORE#5949

* reset custom vars in test

* reset custom vars in test

* reset vars every run

Co-authored-by: Emily Rockman <emily.rockman@dbtlabs.com>
---
 dbt/adapters/bigquery/impl.py                  |  6 +++---
 tests/integration/sources_test/test_sources.py | 10 ++++++----
 2 files changed, 9 insertions(+), 7 deletions(-)

diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 602aafd19..e9a9dfc04 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -16,7 +16,7 @@
     PythonJobHelper,
 )
 
-from dbt.adapters.cache import _make_key
+from dbt.adapters.cache import _make_ref_key_msg
 
 from dbt.adapters.bigquery.relation import BigQueryRelation
 from dbt.adapters.bigquery import BigQueryColumn
@@ -301,7 +301,7 @@ def create_schema(self, relation: BigQueryRelation) -> None:
         # use SQL 'create schema'
         relation = relation.without_identifier()  # type: ignore
 
-        fire_event(SchemaCreation(relation=_make_key(relation)))
+        fire_event(SchemaCreation(relation=_make_ref_key_msg(relation)))
         kwargs = {
             "relation": relation,
         }
@@ -315,7 +315,7 @@ def drop_schema(self, relation: BigQueryRelation) -> None:
         database = relation.database
         schema = relation.schema
         logger.debug('Dropping schema "{}.{}".', database, schema)  # in lieu of SQL
-        fire_event(SchemaDrop(relation=_make_key(relation)))
+        fire_event(SchemaDrop(relation=_make_ref_key_msg(relation)))
         self.connections.drop_dataset(database, schema)
         self.cache.drop_schema(database, schema)
 
diff --git a/tests/integration/sources_test/test_sources.py b/tests/integration/sources_test/test_sources.py
index ec4dd75f6..21dbb19bf 100644
--- a/tests/integration/sources_test/test_sources.py
+++ b/tests/integration/sources_test/test_sources.py
@@ -7,6 +7,7 @@
 from dbt.exceptions import CompilationException
 import dbt.tracking
 import dbt.version
+from dbt.events.functions import reset_metadata_vars
 from tests.integration.base import DBTIntegrationTest, use_profile, AnyFloat, \
     AnyStringWith
 
@@ -48,7 +49,7 @@ def run_dbt_with_vars(self, cmd, *args, **kwargs):
         return self.run_dbt(cmd, *args, **kwargs)
 
 
-class SuccessfulSourcesTest(BaseSourcesTest):
+class TestSourceFreshness(BaseSourcesTest):
     def setUp(self):
         super().setUp()
         self.run_dbt_with_vars(['seed'])
@@ -60,6 +61,7 @@ def setUp(self):
 
     def tearDown(self):
         super().tearDown()
+        reset_metadata_vars()
         del os.environ['DBT_ENV_CUSTOM_ENV_key']
 
     def _set_updated_at_to(self, delta):
@@ -91,9 +93,6 @@ def _set_updated_at_to(self, delta):
         self.last_inserted_time = insert_time.strftime(
             "%Y-%m-%dT%H:%M:%S+00:00")
 
-
-class TestSourceFreshness(SuccessfulSourcesTest):
-
     def _assert_freshness_results(self, path, state):
         self.assertTrue(os.path.exists(path))
         with open(path) as fp:
@@ -151,6 +150,7 @@ def _run_source_freshness(self):
         # and a freshness of warn_after: 10 hours, error_after: 18 hours
         # by default, our data set is way out of date!
         self.freshness_start_time = datetime.utcnow()
+        reset_metadata_vars()
         results = self.run_dbt_with_vars(
             ['source', 'freshness', '-o', 'target/error_source.json'],
             expect_pass=False
@@ -161,6 +161,7 @@ def _run_source_freshness(self):
 
         self._set_updated_at_to(timedelta(hours=-12))
         self.freshness_start_time = datetime.utcnow()
+        reset_metadata_vars()
         results = self.run_dbt_with_vars(
             ['source', 'freshness', '-o', 'target/warn_source.json'],
         )
@@ -170,6 +171,7 @@ def _run_source_freshness(self):
 
         self._set_updated_at_to(timedelta(hours=-2))
         self.freshness_start_time = datetime.utcnow()
+        reset_metadata_vars()
         results = self.run_dbt_with_vars(
             ['source', 'freshness', '-o', 'target/pass_source.json'],
         )

From ed2b051278e531315a120fd63bb39b1eeeb37337 Mon Sep 17 00:00:00 2001
From: Christophe Oudar <kayrnt@gmail.com>
Date: Thu, 20 Oct 2022 19:26:12 +0200
Subject: [PATCH 500/860] Support for ingestion time partition table on
 BigQuery as incremental materialization (#136)

* Support for incremental materialization with ingestion time partition tables

* Refactor incremental materialization for readibility

* add changelog entry

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .../unreleased/Features-20220807-164227.yaml  |   7 +
 dbt/adapters/bigquery/impl.py                 |  37 ++++-
 .../macros/materializations/incremental.sql   | 131 ++++--------------
 .../incremental_strategy/common.sql           |  22 +++
 .../incremental_strategy/insert_overwrite.sql |  93 +++++++++++++
 .../incremental_strategy/merge.sql            |  28 ++++
 .../time_ingestion_tables.sql                 |  69 +++++++++
 ...ncremental_time_ingestion_partitioning.sql |  38 +++++
 ...tal_time_ingestion_partitioning_target.sql |  24 ++++
 .../models/schema.yml                         |  16 ++-
 .../test_incremental_schema.py                |  20 +++
 ...ncremental_time_ingestion_partitioning.sql |   1 +
 ...tal_time_ingestion_partitioning_target.sql |   1 +
 tests/unit/test_bigquery_adapter.py           |  46 ++++--
 14 files changed, 412 insertions(+), 121 deletions(-)
 create mode 100644 .changes/unreleased/Features-20220807-164227.yaml
 create mode 100644 dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
 create mode 100644 dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
 create mode 100644 dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
 create mode 100644 dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
 create mode 100644 tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning.sql
 create mode 100644 tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning_target.sql
 create mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning.sql
 create mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning_target.sql

diff --git a/.changes/unreleased/Features-20220807-164227.yaml b/.changes/unreleased/Features-20220807-164227.yaml
new file mode 100644
index 000000000..9352edc27
--- /dev/null
+++ b/.changes/unreleased/Features-20220807-164227.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: Support for ingestion time partition table on BigQuery as incremental materialization
+time: 2022-08-07T16:42:27.232818+02:00
+custom:
+  Author: Kayrnt
+  Issue: "75"
+  PR: "136"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index e9a9dfc04..eee6cdbe6 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -66,11 +66,15 @@ class PartitionConfig(dbtClassMixin):
     data_type: str = "date"
     granularity: str = "day"
     range: Optional[Dict[str, Any]] = None
+    time_ingestion_partitioning: bool = False
+
+    def reject_partition_field_column(self, columns: List[Any]) -> List[str]:
+        return [c for c in columns if not c.name.upper() == self.field.upper()]
 
     def render(self, alias: Optional[str] = None):
-        column: str = self.field
+        column: str = self.field if not self.time_ingestion_partitioning else "_PARTITIONTIME"
         if alias:
-            column = f"{alias}.{self.field}"
+            column = f"{alias}.{column}"
 
         if self.data_type.lower() == "int64" or (
             self.data_type.lower() == "date" and self.granularity.lower() == "day"
@@ -79,6 +83,13 @@ def render(self, alias: Optional[str] = None):
         else:
             return f"{self.data_type}_trunc({column}, {self.granularity})"
 
+    def render_wrapped(self, alias: Optional[str] = None):
+        """Wrap the partitioning column when time involved to ensure it is properly casted to matching time."""
+        if self.data_type in ("date", "timestamp", "datetime"):
+            return f"{self.data_type}({self.render(alias)})"
+        else:
+            return self.render(alias)
+
     @classmethod
     def parse(cls, raw_partition_by) -> Optional["PartitionConfig"]:  # type: ignore [return]
         if raw_partition_by is None:
@@ -236,6 +247,12 @@ def get_columns_in_relation(self, relation: BigQueryRelation) -> List[BigQueryCo
             logger.debug("get_columns_in_relation error: {}".format(e))
             return []
 
+    @available.parse(lambda *a, **k: [])
+    def add_time_ingestion_partition_column(self, columns) -> List[BigQueryColumn]:
+        "Add time ingestion partition column to columns list"
+        columns.append(self.Column("_PARTITIONTIME", "TIMESTAMP", None, "NULLABLE"))
+        return columns
+
     def expand_column_types(self, goal: BigQueryRelation, current: BigQueryRelation) -> None:  # type: ignore[override]
         # This is a no-op on BigQuery
         pass
@@ -434,6 +451,19 @@ def copy_table(self, source, destination, materialization):
 
         return "COPY TABLE with materialization: {}".format(materialization)
 
+    @available.parse(lambda *a, **k: False)
+    def get_columns_in_select_sql(self, select_sql: str) -> List[BigQueryColumn]:
+        try:
+            conn = self.connections.get_thread_connection()
+            client = conn.handle
+            query_job, iterator = self.connections.raw_execute(select_sql)
+            query_table = client.get_table(query_job.destination)
+            return self._get_dbt_columns_from_bq_table(query_table)
+
+        except (ValueError, google.cloud.exceptions.NotFound) as e:
+            logger.debug("get_columns_in_select_sql error: {}".format(e))
+            return []
+
     @classmethod
     def poll_until_job_completes(cls, job, timeout):
         retry_count = timeout
@@ -495,7 +525,8 @@ def _partitions_match(self, table, conf_partition: Optional[PartitionConfig]) ->
         if not is_partitioned and not conf_partition:
             return True
         elif conf_partition and table.time_partitioning is not None:
-            table_field = table.time_partitioning.field.lower()
+            partioning_field = table.time_partitioning.field or "_PARTITIONTIME"
+            table_field = partioning_field.lower()
             table_granularity = table.partitioning_type.lower()
             return (
                 table_field == conf_partition.field.lower()
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index a594e993e..0e474f2bf 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -1,18 +1,3 @@
-{% macro declare_dbt_max_partition(relation, partition_by, complied_code, language='sql') %}
-
-  {#-- TODO: revisit partitioning with python models --#}
-  {%- if '_dbt_max_partition' in complied_code and language == 'sql' -%}
-
-    declare _dbt_max_partition {{ partition_by.data_type }} default (
-      select max({{ partition_by.field }}) from {{ this }}
-      where {{ partition_by.field }} is not null
-    );
-
-  {%- endif -%}
-
-{% endmacro %}
-
-
 {% macro dbt_bigquery_validate_get_incremental_strategy(config) %}
   {#-- Find and validate the incremental strategy #}
   {%- set strategy = config.get("incremental_strategy") or 'merge' -%}
@@ -28,107 +13,40 @@
   {% do return(strategy) %}
 {% endmacro %}
 
+{% macro source_sql_with_partition(partition_by, source_sql) %}
 
-{% macro bq_insert_overwrite(
-    tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
-) %}
-
-  {% if partitions is not none and partitions != [] %} {# static #}
-
-      {% set predicate -%}
-          {{ partition_by.render(alias='DBT_INTERNAL_DEST') }} in (
-              {{ partitions | join (', ') }}
-          )
-      {%- endset %}
-
-      {%- set source_sql -%}
-        (
-          {{sql}}
-        )
-      {%- endset -%}
-
-      {#-- Because we're putting the model SQL _directly_ into the MERGE statement,
-         we need to prepend the MERGE statement with the user-configured sql_header,
-         which may be needed to resolve that model SQL (e.g. referencing a variable or UDF in the header)
-         in the "dynamic" case, we save the model SQL result as a temp table first, wherein the
-         sql_header is included by the create_table_as macro.
-      #}
-      {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate], include_sql_header=true) }}
-
-  {% else %} {# dynamic #}
-
-      {% set predicate -%}
-          {{ partition_by.render(alias='DBT_INTERNAL_DEST') }} in unnest(dbt_partitions_for_replacement)
-      {%- endset %}
-
-      {%- set source_sql -%}
-      (
-        select * from {{ tmp_relation }}
-      )
-      {%- endset -%}
-
-      -- generated script to merge partitions into {{ target_relation }}
-      declare dbt_partitions_for_replacement array<{{ partition_by.data_type }}>;
-
-      {# have we already created the temp table to check for schema changes? #}
-      {% if not tmp_relation_exists %}
-        {{ declare_dbt_max_partition(this, partition_by, sql) }}
-
-        -- 1. create a temp table
-        {{ create_table_as(True, tmp_relation, compiled_code) }}
-      {% else %}
-        -- 1. temp table already exists, we used it to check for schema changes
-      {% endif %}
-
-      -- 2. define partitions to update
-      set (dbt_partitions_for_replacement) = (
-          select as struct
-              array_agg(distinct {{ partition_by.render() }})
-          from {{ tmp_relation }}
-      );
-
-      -- 3. run the merge statement
-      {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate]) }};
-
-      -- 4. clean up the temp table
-      drop table if exists {{ tmp_relation }}
+  {%- if partition_by.time_ingestion_partitioning %}
+    {{ return(wrap_with_time_ingestion_partitioning(build_partition_time_exp(partition_by.field), source_sql, False))  }}
+  {% else %}
+    {{ return(source_sql)  }}
+  {%- endif -%}
 
+{% endmacro %}
+{% macro bq_create_table_as(is_time_ingestion_partitioning, temporary, relation, compiled_code, language='sql') %}
+  {% if is_time_ingestion_partitioning %}
+    {#-- Create the table before inserting data as ingestion time partitioned tables can't be created with the transformed data --#}
+    {% do run_query(create_ingestion_time_partitioned_table_as(temporary, relation, sql)) %}
+    {{ return(bq_insert_into_ingestion_time_partitioned_table(relation, sql)) }}
+  {% else %}
+    {{ return(create_table_as(temporary, relation, sql)) }}
   {% endif %}
-
 {% endmacro %}
 
-
 {% macro bq_generate_incremental_build_sql(
     strategy, tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
 ) %}
   {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#}
   {% if strategy == 'insert_overwrite' %}
 
-    {% set missing_partition_msg -%}
-      The 'insert_overwrite' strategy requires the `partition_by` config.
-    {%- endset %}
-    {% if partition_by is none %}
-      {% do exceptions.raise_compiler_error(missing_partition_msg) %}
-    {% endif %}
-
-    {% set build_sql = bq_insert_overwrite(
+    {% set build_sql = bq_generate_incremental_insert_overwrite_build_sql(
         tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
     ) %}
 
   {% else %} {# strategy == 'merge' #}
-    {%- set source_sql -%}
-      {%- if tmp_relation_exists -%}
-        (
-          select * from {{ tmp_relation }}
-        )
-      {%- else -%} {#-- wrap sql in parens to make it a subquery --#}
-        (
-          {{sql}}
-        )
-      {%- endif -%}
-    {%- endset -%}
-
-    {% set build_sql = get_merge_sql(target_relation, source_sql, unique_key, dest_columns) %}
+
+    {% set build_sql = bq_generate_incremental_merge_build_sql(
+        tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists
+    ) %}
 
   {% endif %}
 
@@ -163,14 +81,14 @@
 
   {% if existing_relation is none %}
       {%- call statement('main', language=language) -%}
-        {{ create_table_as(False, target_relation, compiled_code, language) }}
+        {{ bq_create_table_as(partition_by.time_ingestion_partitioning, False, target_relation, compiled_code, language) }}
       {%- endcall -%}
 
   {% elif existing_relation.is_view %}
       {#-- There's no way to atomically replace a view with a table on BQ --#}
       {{ adapter.drop_relation(existing_relation) }}
       {%- call statement('main', language=language) -%}
-        {{ create_table_as(False, target_relation, compiled_code, language) }}
+        {{ bq_create_table_as(partition_by.time_ingestion_partitioning, False, target_relation, compiled_code, language) }}
       {%- endcall -%}
 
   {% elif full_refresh_mode %}
@@ -180,7 +98,7 @@
           {{ adapter.drop_relation(existing_relation) }}
       {% endif %}
       {%- call statement('main', language=language) -%}
-        {{ create_table_as(False, target_relation, compiled_code, language) }}
+        {{ bq_create_table_as(partition_by.time_ingestion_partitioning, False, target_relation, compiled_code, language) }}
       {%- endcall -%}
 
   {% else %}
@@ -198,7 +116,7 @@
       {#-- Python always needs to create a temp table --#}
       {%- call statement('create_tmp_relation', language=language) -%}
         {{ declare_dbt_max_partition(this, partition_by, compiled_code, language) +
-           create_table_as(True, tmp_relation, compiled_code, language)
+           bq_create_table_as(partition_by.time_ingestion_partitioning, True, tmp_relation, compiled_code, language)
         }}
       {%- endcall -%}
       {% set tmp_relation_exists = true %}
@@ -209,6 +127,9 @@
     {% if not dest_columns %}
       {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %}
     {% endif %}
+    {% if partition_by.time_ingestion_partitioning %}
+      {% set dest_columns = adapter.add_time_ingestion_partition_column(dest_columns) %}
+    {% endif %}
     {% set build_sql = bq_generate_incremental_build_sql(
         strategy, tmp_relation, target_relation, compiled_code, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
     ) %}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
new file mode 100644
index 000000000..ca9d2a2c5
--- /dev/null
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
@@ -0,0 +1,22 @@
+{% macro  build_partition_time_exp(partition_by) %}
+  {% if partition_by.data_type == 'timestamp' %}
+    {% set partition_value = partition_by.field %}
+  {% else %}
+    {% set partition_value = 'timestamp(' + partition_by.field + ')' %}
+  {% endif %}
+  {{ return({'value': partition_value, 'field': partition_by.field}) }}
+{% endmacro %}
+
+{% macro declare_dbt_max_partition(relation, partition_by, complied_code, language='sql') %}
+
+  {#-- TODO: revisit partitioning with python models --#}
+  {%- if '_dbt_max_partition' in complied_code and language == 'sql' -%}
+
+    declare _dbt_max_partition {{ partition_by.data_type }} default (
+      select max({{ partition_by.field }}) from {{ this }}
+      where {{ partition_by.field }} is not null
+    );
+
+  {%- endif -%}
+
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
new file mode 100644
index 000000000..cacdb603e
--- /dev/null
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
@@ -0,0 +1,93 @@
+{% macro bq_generate_incremental_insert_overwrite_build_sql(
+    tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, on_schema_change
+) %}
+    {% if partition_by is none %}
+      {% set missing_partition_msg -%}
+      The 'insert_overwrite' strategy requires the `partition_by` config.
+      {%- endset %}
+      {% do exceptions.raise_compiler_error(missing_partition_msg) %}
+    {% endif %}
+
+    {% set build_sql = bq_insert_overwrite(
+        tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, on_schema_change
+    ) %}
+
+    {{ return(build_sql) }}
+
+{% endmacro %}
+
+{% macro bq_insert_overwrite(
+    tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
+) %}
+
+  {% if partitions is not none and partitions != [] %} {# static #}
+
+      {% set predicate -%}
+          {{ partition_by.render_wrapped(alias='DBT_INTERNAL_DEST') }} in (
+              {{ partitions | join (', ') }}
+          )
+      {%- endset %}
+
+      {%- set source_sql -%}
+        (
+          {%- if partition_by.time_ingestion_partitioning -%}
+          {{ wrap_with_time_ingestion_partitioning(build_partition_time_exp(partition_by), sql, True) }}
+          {%- else -%}
+          {{sql}}
+          {%- endif -%}
+        )
+      {%- endset -%}
+
+      {#-- Because we're putting the model SQL _directly_ into the MERGE statement,
+         we need to prepend the MERGE statement with the user-configured sql_header,
+         which may be needed to resolve that model SQL (e.g. referencing a variable or UDF in the header)
+         in the "dynamic" case, we save the model SQL result as a temp table first, wherein the
+         sql_header is included by the create_table_as macro.
+      #}
+      {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate], include_sql_header=true) }}
+
+  {% else %} {# dynamic #}
+
+      {% set predicate -%}
+          {{ partition_by.render_wrapped(alias='DBT_INTERNAL_DEST') }} in unnest(dbt_partitions_for_replacement)
+      {%- endset %}
+
+      {%- set source_sql -%}
+      (
+        select
+        {% if partition_by.time_ingestion_partitioning -%}
+        _PARTITIONTIME,
+        {%- endif -%}
+        * from {{ tmp_relation }}
+      )
+      {%- endset -%}
+
+      -- generated script to merge partitions into {{ target_relation }}
+      declare dbt_partitions_for_replacement array<{{ partition_by.data_type }}>;
+
+      {# have we already created the temp table to check for schema changes? #}
+      {% if not tmp_relation_exists %}
+        {{ declare_dbt_max_partition(this, partition_by, sql) }}
+
+        -- 1. create a temp table
+        {{ bq_create_table_as(partition_by.time_ingestion_partitioning, True, tmp_relation, compiled_code) }}
+      {% else %}
+        -- 1. temp table already exists, we used it to check for schema changes
+      {% endif %}
+
+      -- 2. define partitions to update
+      set (dbt_partitions_for_replacement) = (
+          select as struct
+              array_agg(distinct {{ partition_by.render_wrapped() }})
+          from {{ tmp_relation }}
+      );
+
+      -- 3. run the merge statement
+      {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate]) }};
+
+      -- 4. clean up the temp table
+      drop table if exists {{ tmp_relation }}
+
+  {% endif %}
+
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
new file mode 100644
index 000000000..8a86d1d8f
--- /dev/null
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
@@ -0,0 +1,28 @@
+{% macro bq_generate_incremental_merge_build_sql(
+    tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists
+) %}
+    {%- set source_sql -%}
+        {%- if tmp_relation_exists -%}
+        (
+        select
+        {% if partition_by.time_ingestion_partitioning -%}
+        _PARTITIONTIME,
+        {%- endif -%}
+        * from {{ tmp_relation }}
+        )
+        {%- else -%} {#-- wrap sql in parens to make it a subquery --#}
+        (
+            {%- if partition_by.time_ingestion_partitioning -%}
+            {{ wrap_with_time_ingestion_partitioning(build_partition_time_exp(partition_by), sql, True) }}
+            {%- else -%}
+            {{sql}}
+            {%- endif -%}
+        )
+        {%- endif -%}
+    {%- endset -%}
+
+    {% set build_sql = get_merge_sql(target_relation, source_sql, unique_key, dest_columns) %}
+
+    {{ return(build_sql) }}
+
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
new file mode 100644
index 000000000..4b2cae0b9
--- /dev/null
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
@@ -0,0 +1,69 @@
+{% macro  wrap_with_time_ingestion_partitioning(partition_time_exp, sql, is_nested) %}
+
+  select {{ partition_time_exp['value'] }} as _partitiontime, * EXCEPT({{ partition_time_exp['field'] }}) from (
+    {{ sql }}
+  ){%- if not is_nested -%};{%- endif -%}
+
+{% endmacro %}
+
+{% macro create_ingestion_time_partitioned_table_as(temporary, relation, sql) -%}
+  {%- set raw_partition_by = config.get('partition_by', none) -%}
+  {%- set raw_cluster_by = config.get('cluster_by', none) -%}
+  {%- set sql_header = config.get('sql_header', none) -%}
+
+  {%- set partition_config = adapter.parse_partition_by(raw_partition_by) -%}
+
+  {%- set columns = get_columns_with_types_in_query(sql) -%}
+  {%- set table_dest_columns_csv = columns_without_partition_fields_csv(partition_config, columns) -%}
+
+  {{ sql_header if sql_header is not none }}
+
+  {% set ingestion_time_partition_config_raw = fromjson(tojson(raw_partition_by))  %}
+  {% do ingestion_time_partition_config_raw.update({'field':'_PARTITIONTIME'}) %}
+
+  {%- set ingestion_time_partition_config = adapter.parse_partition_by(ingestion_time_partition_config_raw) -%}
+
+  create or replace table {{ relation }} ({{table_dest_columns_csv}})
+  {{ partition_by(ingestion_time_partition_config) }}
+  {{ cluster_by(raw_cluster_by) }}
+  {{ bigquery_table_options(config, model, temporary) }}
+
+{%- endmacro -%}
+
+{% macro get_quoted_with_types_csv(columns) %}
+    {% set quoted = [] %}
+    {% for col in columns -%}
+        {%- do quoted.append(adapter.quote(col.name) ~ " " ~ col.data_type) -%}
+    {%- endfor %}
+    {%- set dest_cols_csv = quoted | join(', ') -%}
+    {{ return(dest_cols_csv) }}
+
+{% endmacro %}
+
+{% macro columns_without_partition_fields_csv(partition_config, columns) -%}
+  {%- set columns_no_partition = partition_config.reject_partition_field_column(columns) -%}
+  {% set columns_names = get_quoted_with_types_csv(columns_no_partition) %}
+  {{ return(columns_names) }}
+
+{%- endmacro -%}
+
+{% macro bq_insert_into_ingestion_time_partitioned_table(target_relation, sql) -%}
+  {%- set partition_by = config.get('partition_by', none) -%}
+  {% set dest_columns = adapter.get_columns_in_relation(target_relation) %}
+  {%- set dest_columns_csv = get_quoted_csv(dest_columns | map(attribute="name")) -%}
+
+  insert into {{ target_relation }} (_partitiontime, {{ dest_columns_csv }})
+    {{ wrap_with_time_ingestion_partitioning(build_partition_time_exp(partition_by), sql, False) }}
+
+{%- endmacro -%}
+
+{% macro get_columns_with_types_in_query(select_sql) %}
+  {% set sql %}
+    select * from (
+      {{ select_sql }}
+    ) as __dbt_sbq
+    where false
+    limit 0
+  {% endset %}
+  {{ return(adapter.get_columns_in_select_sql(sql)) }}
+{% endmacro %}
diff --git a/tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning.sql b/tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning.sql
new file mode 100644
index 000000000..ce064b33c
--- /dev/null
+++ b/tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning.sql
@@ -0,0 +1,38 @@
+
+{{
+    config(
+        materialized="incremental",
+        incremental_strategy='insert_overwrite',
+        partition_by={
+            "field": "date_hour",
+            "data_type": "datetime",
+            "granularity": "hour",
+            "time_ingestion_partitioning": true
+        }
+    )
+}}
+
+
+with data as (
+    
+    {% if not is_incremental() %}
+    
+        select 1 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 2 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 3 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 4 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour
+
+    {% else %}
+    
+        -- we want to overwrite the 4 records in the 2020-01-01 01:00:00 partition
+        -- with the 2 records below, but add two more in the 2020-01-00 02:00:00 partition
+        select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour union all
+        select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+    
+    {% endif %}
+
+)
+
+select * from data
diff --git a/tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning_target.sql b/tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning_target.sql
new file mode 100644
index 000000000..2f25229de
--- /dev/null
+++ b/tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning_target.sql
@@ -0,0 +1,24 @@
+
+{{
+    config(
+        materialized="incremental",
+        partition_by={
+            "field": "date_hour",
+            "data_type": "datetime",
+            "granularity": "hour",
+            "time_ingestion_partitioning": true
+        }
+    )
+}}
+
+{% if not is_incremental() %}
+
+    select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+    select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+
+{% else %}
+
+    select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+    select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+
+{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/schema.yml b/tests/integration/incremental_schema_tests/models/schema.yml
index 21aa6095f..fd3136102 100644
--- a/tests/integration/incremental_schema_tests/models/schema.yml
+++ b/tests/integration/incremental_schema_tests/models/schema.yml
@@ -63,6 +63,18 @@ models:
         tags: [column_leveL_tag]
         tests:
           - unique
-  
 
-  
\ No newline at end of file
+  - name: incremental_time_ingestion_partitioning
+    columns:
+      - name: id
+        tags: [column_level_tag]
+        tests:
+          - unique
+
+  - name: incremental_time_ingestion_partitioning_target
+    columns:
+      - name: id
+        tags: [column_level_tag]
+        tests:
+          - unique
+
diff --git a/tests/integration/incremental_schema_tests/test_incremental_schema.py b/tests/integration/incremental_schema_tests/test_incremental_schema.py
index 113a53585..9592c500f 100644
--- a/tests/integration/incremental_schema_tests/test_incremental_schema.py
+++ b/tests/integration/incremental_schema_tests/test_incremental_schema.py
@@ -124,6 +124,22 @@ def run_incremental_fail_on_schema_change(self):
         results_two = self.run_dbt(['run', '--models', select], expect_pass = False)
         self.assertIn('Compilation Error', results_two[1].message)
 
+    def run_incremental_time_ingestion_partitioning(self):
+        select = 'model_a incremental_time_ingestion_partitioning incremental_time_ingestion_partitioning_target'
+        compare_source = 'incremental_time_ingestion_partitioning'
+        compare_target = 'incremental_time_ingestion_partitioning_target'
+        exclude = None
+        expected = [
+            'select_from_a',
+            'select_from_incremental_time_ingestion_partitioning',
+            'select_from_incremental_time_ingestion_partitioning_target',
+            'unique_model_a_id',
+            'unique_incremental_time_ingestion_partitioning_id',
+            'unique_incremental_time_ingestion_partitioning_target_id'
+        ]
+        self.list_tests_and_assert(select, exclude, expected)
+        self.run_tests_and_assert(select, exclude, expected, compare_source, compare_target)
+
     @use_profile('bigquery')
     def test__bigquery__run_incremental_ignore(self):
         self.run_incremental_ignore()
@@ -140,3 +156,7 @@ def test__bigquery__run_incremental_sync_all_columns(self):
     @use_profile('bigquery')
     def test__bigquery__run_incremental_fail_on_schema_change(self):
         self.run_incremental_fail_on_schema_change()
+
+    @use_profile('bigquery')
+    def test__bigquery__run_incremental_time_ingestion_partitioning(self):
+        self.run_incremental_time_ingestion_partitioning()
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning.sql
new file mode 100644
index 000000000..85e653c11
--- /dev/null
+++ b/tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning.sql
@@ -0,0 +1 @@
+select * from {{ ref('incremental_time_ingestion_partitioning') }} where false
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning_target.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning_target.sql
new file mode 100644
index 000000000..e2533dff7
--- /dev/null
+++ b/tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning_target.sql
@@ -0,0 +1 @@
+select * from {{ ref('incremental_time_ingestion_partitioning_target') }} where false
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 188c19b7c..36d84fc44 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -686,7 +686,8 @@ def test_parse_partition_by(self):
             }).to_dict(omit_none=True), {
                 "field": "ts",
                 "data_type": "date",
-                "granularity": "day"
+                "granularity": "day",
+                "time_ingestion_partitioning": False
             }
         )
 
@@ -697,7 +698,8 @@ def test_parse_partition_by(self):
             }).to_dict(omit_none=True), {
                 "field": "ts",
                 "data_type": "date",
-                "granularity": "day"
+                "granularity": "day",
+                "time_ingestion_partitioning": False
             }
         )
 
@@ -710,7 +712,8 @@ def test_parse_partition_by(self):
             }).to_dict(omit_none=True), {
                 "field": "ts",
                 "data_type": "date",
-                "granularity": "MONTH"
+                "granularity": "MONTH",
+                "time_ingestion_partitioning": False
             }
         )
         
@@ -723,7 +726,8 @@ def test_parse_partition_by(self):
             }).to_dict(omit_none=True), {
                 "field": "ts",
                 "data_type": "date",
-                "granularity": "YEAR"
+                "granularity": "YEAR",
+                "time_ingestion_partitioning": False
             }
         )
 
@@ -736,7 +740,8 @@ def test_parse_partition_by(self):
             }).to_dict(omit_none=True), {
                 "field": "ts",
                 "data_type": "timestamp",
-                "granularity": "HOUR"
+                "granularity": "HOUR",
+                "time_ingestion_partitioning": False
             }
         )
 
@@ -750,7 +755,8 @@ def test_parse_partition_by(self):
                 ), {
                 "field": "ts",
                 "data_type": "timestamp",
-                "granularity": "MONTH"
+                "granularity": "MONTH",
+                "time_ingestion_partitioning": False
             }
         )
 
@@ -763,7 +769,8 @@ def test_parse_partition_by(self):
             }).to_dict(omit_none=True), {
                 "field": "ts",
                 "data_type": "timestamp",
-                "granularity": "YEAR"
+                "granularity": "YEAR",
+                "time_ingestion_partitioning": False
             }
         )
 
@@ -776,7 +783,8 @@ def test_parse_partition_by(self):
             }).to_dict(omit_none=True), {
                 "field": "ts",
                 "data_type": "datetime",
-                "granularity": "HOUR"
+                "granularity": "HOUR",
+                "time_ingestion_partitioning": False
             }
         )
 
@@ -789,7 +797,8 @@ def test_parse_partition_by(self):
             }).to_dict(omit_none=True), {
                 "field": "ts",
                 "data_type": "datetime",
-                "granularity": "MONTH"
+                "granularity": "MONTH",
+                "time_ingestion_partitioning": False
             }
         )
 
@@ -802,7 +811,21 @@ def test_parse_partition_by(self):
             }).to_dict(omit_none=True), {
                 "field": "ts",
                 "data_type": "datetime",
-                "granularity": "YEAR"
+                "granularity": "YEAR",
+                "time_ingestion_partitioning": False
+            }
+        )
+
+        self.assertEqual(
+            adapter.parse_partition_by({
+                "field": "ts",
+                "time_ingestion_partitioning": True
+
+            }).to_dict(omit_none=True), {
+                "field": "ts",
+                "data_type": "date",
+                "granularity": "day",
+                "time_ingestion_partitioning": True
             }
         )
 
@@ -829,7 +852,8 @@ def test_parse_partition_by(self):
                     "start": 1,
                     "end": 100,
                     "interval": 20
-                }
+                },
+                "time_ingestion_partitioning": False
             }
         )
 

From a3d70544ab0cb736676c17932b8f52cd215fe1d7 Mon Sep 17 00:00:00 2001
From: Christophe Oudar
 <90898980+github-christophe-oudar@users.noreply.github.com>
Date: Tue, 25 Oct 2022 22:38:35 +0200
Subject: [PATCH 501/860] Optimize insert_overwrite incremental strategy with
 WRITE_TRUNCATE / Partition copy (#167)

* Support copy partitions option for insert overwrite mode

* macro renames + missing declare_dbt_max_partition

Co-authored-by: Christophe Oudar <kayrnt@gmail.com>
---
 .../unreleased/Features-20221020-223914.yaml  |  8 ++
 dbt/adapters/bigquery/impl.py                 | 12 ++-
 .../macros/materializations/incremental.sql   | 22 ++++--
 .../incremental_strategy/common.sql           |  6 +-
 .../incremental_strategy/insert_overwrite.sql | 76 ++++++++++++++++---
 .../incremental_strategy/merge.sql            |  2 +-
 .../time_ingestion_tables.sql                 | 14 ++--
 ...tal_overwrite_day_with_copy_partitions.sql | 41 ++++++++++
 .../test_incremental_strategies.py            |  6 +-
 tests/unit/test_bigquery_adapter.py           | 47 +++++++-----
 10 files changed, 183 insertions(+), 51 deletions(-)
 create mode 100644 .changes/unreleased/Features-20221020-223914.yaml
 create mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day_with_copy_partitions.sql

diff --git a/.changes/unreleased/Features-20221020-223914.yaml b/.changes/unreleased/Features-20221020-223914.yaml
new file mode 100644
index 000000000..58b8e60a3
--- /dev/null
+++ b/.changes/unreleased/Features-20221020-223914.yaml
@@ -0,0 +1,8 @@
+kind: Features
+body: Optimize insert_overwrite incremental strategy with WRITE_TRUNCATE / Partition
+  copy
+time: 2022-10-20T22:39:14.091878+02:00
+custom:
+  Author: Kayrnt
+  Issue: "77"
+  PR: "167"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index eee6cdbe6..8597db6f1 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -67,6 +67,7 @@ class PartitionConfig(dbtClassMixin):
     granularity: str = "day"
     range: Optional[Dict[str, Any]] = None
     time_ingestion_partitioning: bool = False
+    copy_partitions: bool = False
 
     def reject_partition_field_column(self, columns: List[Any]) -> List[str]:
         return [c for c in columns if not c.name.upper() == self.field.upper()]
@@ -525,11 +526,16 @@ def _partitions_match(self, table, conf_partition: Optional[PartitionConfig]) ->
         if not is_partitioned and not conf_partition:
             return True
         elif conf_partition and table.time_partitioning is not None:
-            partioning_field = table.time_partitioning.field or "_PARTITIONTIME"
-            table_field = partioning_field.lower()
+            partitioning_field = table.time_partitioning.field or "_PARTITIONTIME"
+            table_field = partitioning_field.lower()
             table_granularity = table.partitioning_type.lower()
+            conf_table_field = (
+                conf_partition.field
+                if not conf_partition.time_ingestion_partitioning
+                else "_PARTITIONTIME"
+            )
             return (
-                table_field == conf_partition.field.lower()
+                table_field == conf_table_field.lower()
                 and table_granularity == conf_partition.granularity.lower()
             )
         elif conf_partition and table.range_partitioning is not None:
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index 0e474f2bf..9ed100405 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -16,7 +16,7 @@
 {% macro source_sql_with_partition(partition_by, source_sql) %}
 
   {%- if partition_by.time_ingestion_partitioning %}
-    {{ return(wrap_with_time_ingestion_partitioning(build_partition_time_exp(partition_by.field), source_sql, False))  }}
+    {{ return(wrap_with_time_ingestion_partitioning_sql(build_partition_time_exp(partition_by.field), source_sql, False))  }}
   {% else %}
     {{ return(source_sql)  }}
   {%- endif -%}
@@ -25,21 +25,21 @@
 {% macro bq_create_table_as(is_time_ingestion_partitioning, temporary, relation, compiled_code, language='sql') %}
   {% if is_time_ingestion_partitioning %}
     {#-- Create the table before inserting data as ingestion time partitioned tables can't be created with the transformed data --#}
-    {% do run_query(create_ingestion_time_partitioned_table_as(temporary, relation, sql)) %}
-    {{ return(bq_insert_into_ingestion_time_partitioned_table(relation, sql)) }}
+    {% do run_query(create_ingestion_time_partitioned_table_as_sql(temporary, relation, sql)) %}
+    {{ return(bq_insert_into_ingestion_time_partitioned_table_sql(relation, sql)) }}
   {% else %}
     {{ return(create_table_as(temporary, relation, sql)) }}
   {% endif %}
 {% endmacro %}
 
 {% macro bq_generate_incremental_build_sql(
-    strategy, tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
+    strategy, tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions
 ) %}
   {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#}
   {% if strategy == 'insert_overwrite' %}
 
     {% set build_sql = bq_generate_incremental_insert_overwrite_build_sql(
-        tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
+        tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions
     ) %}
 
   {% else %} {# strategy == 'merge' #}
@@ -74,12 +74,18 @@
 
   {% set on_schema_change = incremental_validate_on_schema_change(config.get('on_schema_change'), default='ignore') %}
 
-   -- grab current tables grants config for comparision later on
+   -- grab current tables grants config for comparison later on
   {% set grant_config = config.get('grants') %}
 
   {{ run_hooks(pre_hooks) }}
 
-  {% if existing_relation is none %}
+  {% if partition_by.copy_partitions is true and strategy != 'insert_overwrite' %} {#-- We can't copy partitions with merge strategy --#}
+        {% set wrong_strategy_msg -%}
+        The 'copy_partitions' option requires the 'incremental_strategy' option to be set to 'insert_overwrite'.
+        {%- endset %}
+        {% do exceptions.raise_compiler_error(wrong_strategy_msg) %}
+
+  {% elif existing_relation is none %}
       {%- call statement('main', language=language) -%}
         {{ bq_create_table_as(partition_by.time_ingestion_partitioning, False, target_relation, compiled_code, language) }}
       {%- endcall -%}
@@ -131,7 +137,7 @@
       {% set dest_columns = adapter.add_time_ingestion_partition_column(dest_columns) %}
     {% endif %}
     {% set build_sql = bq_generate_incremental_build_sql(
-        strategy, tmp_relation, target_relation, compiled_code, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
+        strategy, tmp_relation, target_relation, compiled_code, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, partition_by.copy_partitions
     ) %}
 
     {%- call statement('main') -%}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
index ca9d2a2c5..237dfbbf0 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
@@ -1,4 +1,4 @@
-{% macro  build_partition_time_exp(partition_by) %}
+{% macro build_partition_time_exp(partition_by) %}
   {% if partition_by.data_type == 'timestamp' %}
     {% set partition_value = partition_by.field %}
   {% else %}
@@ -7,10 +7,10 @@
   {{ return({'value': partition_value, 'field': partition_by.field}) }}
 {% endmacro %}
 
-{% macro declare_dbt_max_partition(relation, partition_by, complied_code, language='sql') %}
+{% macro declare_dbt_max_partition(relation, partition_by, compiled_code, language='sql') %}
 
   {#-- TODO: revisit partitioning with python models --#}
-  {%- if '_dbt_max_partition' in complied_code and language == 'sql' -%}
+  {%- if '_dbt_max_partition' in compiled_code and language == 'sql' -%}
 
     declare _dbt_max_partition {{ partition_by.data_type }} default (
       select max({{ partition_by.field }}) from {{ this }}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
index cacdb603e..5cd5a5354 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
@@ -1,5 +1,5 @@
 {% macro bq_generate_incremental_insert_overwrite_build_sql(
-    tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, on_schema_change
+    tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, on_schema_change, copy_partitions
 ) %}
     {% if partition_by is none %}
       {% set missing_partition_msg -%}
@@ -8,19 +8,46 @@
       {% do exceptions.raise_compiler_error(missing_partition_msg) %}
     {% endif %}
 
-    {% set build_sql = bq_insert_overwrite(
-        tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, on_schema_change
+    {% set build_sql = bq_insert_overwrite_sql(
+        tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, on_schema_change, copy_partitions
     ) %}
 
     {{ return(build_sql) }}
 
 {% endmacro %}
 
-{% macro bq_insert_overwrite(
-    tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists
-) %}
+{% macro bq_copy_partitions(tmp_relation, target_relation, partitions, partition_by) %}
+
+  {% for partition in partitions %}
+    {% if partition_by.granularity == 'hour' %}
+      {% set partition = partition.strftime("%Y%m%d%H") %}
+    {% elif partition_by.granularity == 'day' %}
+      {% set partition = partition.strftime("%Y%m%d") %}
+    {% elif partition_by.granularity == 'month' %}
+      {% set partition = partition.strftime("%Y%m") %}
+    {% elif partition_by.granularity == 'year' %}
+      {% set partition = partition.strftime("%Y") %}
+    {% endif %}
+    {% set tmp_relation_partitioned = api.Relation.create(database=tmp_relation.database, schema=tmp_relation.schema, identifier=tmp_relation.table ~ '$' ~ partition, type=tmp_relation.type) %}
+    {% set target_relation_partitioned = api.Relation.create(database=target_relation.database, schema=target_relation.schema, identifier=target_relation.table ~ '$' ~ partition, type=target_relation.type) %}
+    {% do adapter.copy_table(tmp_relation_partitioned, target_relation_partitioned, "table") %}
+  {% endfor %}
 
+{% endmacro %}
+
+{% macro bq_insert_overwrite_sql(
+    tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions
+) %}
   {% if partitions is not none and partitions != [] %} {# static #}
+      {{ bq_static_insert_overwrite_sql(tmp_relation, target_relation, sql, partition_by, partitions, dest_columns, copy_partitions) }}
+  {% else %} {# dynamic #}
+      {{ bq_dynamic_insert_overwrite_sql(tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists, copy_partitions) }}
+  {% endif %}
+{% endmacro %}
+
+{% macro bq_static_insert_overwrite_sql(
+    tmp_relation, target_relation, sql, partition_by, partitions, dest_columns, copy_partitions
+) %}
 
       {% set predicate -%}
           {{ partition_by.render_wrapped(alias='DBT_INTERNAL_DEST') }} in (
@@ -31,13 +58,17 @@
       {%- set source_sql -%}
         (
           {%- if partition_by.time_ingestion_partitioning -%}
-          {{ wrap_with_time_ingestion_partitioning(build_partition_time_exp(partition_by), sql, True) }}
+          {{ wrap_with_time_ingestion_partitioning_sql(build_partition_time_exp(partition_by), sql, True) }}
           {%- else -%}
           {{sql}}
           {%- endif -%}
         )
       {%- endset -%}
 
+      {% if copy_partitions %}
+          {% do bq_copy_partitions(tmp_relation, target_relation, partitions, partition_by) %}
+      {% else %}
+
       {#-- Because we're putting the model SQL _directly_ into the MERGE statement,
          we need to prepend the MERGE statement with the user-configured sql_header,
          which may be needed to resolve that model SQL (e.g. referencing a variable or UDF in the header)
@@ -46,8 +77,33 @@
       #}
       {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate], include_sql_header=true) }}
 
-  {% else %} {# dynamic #}
+  {% endif %}
+{% endmacro %}
+
+{% macro bq_dynamic_copy_partitions_insert_overwrite_sql(
+  tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists, copy_partitions
+  ) %}
+  {# We run temp table creation in a separated script to move to partitions copy #}
+  {%- call statement('create_tmp_relation_for_copy', language='sql') -%}
+    {{ declare_dbt_max_partition(this, partition_by, sql, 'sql') +
+     bq_create_table_as(partition_by.time_ingestion_partitioning, True, tmp_relation, sql, 'sql')
+  }}
+  {%- endcall %}
+  {%- set partitions_sql -%}
+    select distinct {{ partition_by.render_wrapped() }}
+    from {{ tmp_relation }}
+  {%- endset -%}
+  {%- set partitions = run_query(partitions_sql).columns[0].values() -%}
+  {# We copy the partitions #}
+  {%- do bq_copy_partitions(tmp_relation, target_relation, partitions, partition_by) -%}
+  -- Clean up the temp table
+  drop table if exists {{ tmp_relation }}
+{% endmacro %}
 
+{% macro bq_dynamic_insert_overwrite_sql(tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists, copy_partitions) %}
+  {%- if copy_partitions is true %}
+     {{ bq_dynamic_copy_partitions_insert_overwrite_sql(tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists, copy_partitions) }}
+  {% else -%}
       {% set predicate -%}
           {{ partition_by.render_wrapped(alias='DBT_INTERNAL_DEST') }} in unnest(dbt_partitions_for_replacement)
       {%- endset %}
@@ -69,8 +125,8 @@
       {% if not tmp_relation_exists %}
         {{ declare_dbt_max_partition(this, partition_by, sql) }}
 
-        -- 1. create a temp table
-        {{ bq_create_table_as(partition_by.time_ingestion_partitioning, True, tmp_relation, compiled_code) }}
+       -- 1. create a temp table with model data
+        {{ bq_create_table_as(partition_by.time_ingestion_partitioning, True, tmp_relation, sql, 'sql') }}
       {% else %}
         -- 1. temp table already exists, we used it to check for schema changes
       {% endif %}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
index 8a86d1d8f..e6ce91c88 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
@@ -13,7 +13,7 @@
         {%- else -%} {#-- wrap sql in parens to make it a subquery --#}
         (
             {%- if partition_by.time_ingestion_partitioning -%}
-            {{ wrap_with_time_ingestion_partitioning(build_partition_time_exp(partition_by), sql, True) }}
+            {{ wrap_with_time_ingestion_partitioning_sql(build_partition_time_exp(partition_by), sql, True) }}
             {%- else -%}
             {{sql}}
             {%- endif -%}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
index 4b2cae0b9..79d6a74eb 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
@@ -1,4 +1,4 @@
-{% macro  wrap_with_time_ingestion_partitioning(partition_time_exp, sql, is_nested) %}
+{% macro wrap_with_time_ingestion_partitioning_sql(partition_time_exp, sql, is_nested) %}
 
   select {{ partition_time_exp['value'] }} as _partitiontime, * EXCEPT({{ partition_time_exp['field'] }}) from (
     {{ sql }}
@@ -6,19 +6,19 @@
 
 {% endmacro %}
 
-{% macro create_ingestion_time_partitioned_table_as(temporary, relation, sql) -%}
+{% macro create_ingestion_time_partitioned_table_as_sql(temporary, relation, sql) -%}
   {%- set raw_partition_by = config.get('partition_by', none) -%}
   {%- set raw_cluster_by = config.get('cluster_by', none) -%}
   {%- set sql_header = config.get('sql_header', none) -%}
 
   {%- set partition_config = adapter.parse_partition_by(raw_partition_by) -%}
 
-  {%- set columns = get_columns_with_types_in_query(sql) -%}
+  {%- set columns = get_columns_with_types_in_query_sql(sql) -%}
   {%- set table_dest_columns_csv = columns_without_partition_fields_csv(partition_config, columns) -%}
 
   {{ sql_header if sql_header is not none }}
 
-  {% set ingestion_time_partition_config_raw = fromjson(tojson(raw_partition_by))  %}
+  {% set ingestion_time_partition_config_raw = fromjson(tojson(raw_partition_by)) %}
   {% do ingestion_time_partition_config_raw.update({'field':'_PARTITIONTIME'}) %}
 
   {%- set ingestion_time_partition_config = adapter.parse_partition_by(ingestion_time_partition_config_raw) -%}
@@ -47,17 +47,17 @@
 
 {%- endmacro -%}
 
-{% macro bq_insert_into_ingestion_time_partitioned_table(target_relation, sql) -%}
+{% macro bq_insert_into_ingestion_time_partitioned_table_sql(target_relation, sql) -%}
   {%- set partition_by = config.get('partition_by', none) -%}
   {% set dest_columns = adapter.get_columns_in_relation(target_relation) %}
   {%- set dest_columns_csv = get_quoted_csv(dest_columns | map(attribute="name")) -%}
 
   insert into {{ target_relation }} (_partitiontime, {{ dest_columns_csv }})
-    {{ wrap_with_time_ingestion_partitioning(build_partition_time_exp(partition_by), sql, False) }}
+    {{ wrap_with_time_ingestion_partitioning_sql(build_partition_time_exp(partition_by), sql, False) }}
 
 {%- endmacro -%}
 
-{% macro get_columns_with_types_in_query(select_sql) %}
+{% macro get_columns_with_types_in_query_sql(select_sql) %}
   {% set sql %}
     select * from (
       {{ select_sql }}
diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day_with_copy_partitions.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day_with_copy_partitions.sql
new file mode 100644
index 000000000..77e7c0f52
--- /dev/null
+++ b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day_with_copy_partitions.sql
@@ -0,0 +1,41 @@
+{{
+    config(
+        materialized="incremental",
+        incremental_strategy='insert_overwrite',
+        cluster_by="id",
+        partition_by={
+            "field": "date_time",
+            "data_type": "datetime",
+            "copy_partitions": true
+        }
+    )
+}}
+
+
+with data as (
+
+    {% if not is_incremental() %}
+
+        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 4 as id, cast('2020-01-01' as datetime) as date_time
+
+    {% else %}
+
+        -- we want to overwrite the 4 records in the 2020-01-01 partition
+        -- with the 2 records below, but add two more in the 2020-01-02 partition
+        select 10 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 20 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 30 as id, cast('2020-01-02' as datetime) as date_time union all
+        select 40 as id, cast('2020-01-02' as datetime) as date_time
+
+    {% endif %}
+
+)
+
+select * from data
+
+{% if is_incremental() %}
+where date_time >= _dbt_max_partition
+{% endif %}
diff --git a/tests/integration/bigquery_test/test_incremental_strategies.py b/tests/integration/bigquery_test/test_incremental_strategies.py
index 32c151b20..920460eaa 100644
--- a/tests/integration/bigquery_test/test_incremental_strategies.py
+++ b/tests/integration/bigquery_test/test_incremental_strategies.py
@@ -13,10 +13,10 @@ def models(self):
     @use_profile('bigquery')
     def test__bigquery_assert_incrementals(self):
         results = self.run_dbt()
-        self.assertEqual(len(results), 7)
+        self.assertEqual(len(results), 8)
 
         results = self.run_dbt()
-        self.assertEqual(len(results), 7)
+        self.assertEqual(len(results), 8)
 
         results = self.run_dbt(['seed'])
 
@@ -27,3 +27,5 @@ def test__bigquery_assert_incrementals(self):
         self.assertTablesEqual('incremental_overwrite_partitions', 'incremental_overwrite_date_expected')
         self.assertTablesEqual('incremental_overwrite_day', 'incremental_overwrite_day_expected')
         self.assertTablesEqual('incremental_overwrite_range', 'incremental_overwrite_range_expected')
+        self.assertTablesEqual('incremental_overwrite_day_with_copy_partitions', 'incremental_overwrite_day_expected')
+        
\ No newline at end of file
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 36d84fc44..83a2eb3a6 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -522,7 +522,7 @@ def setUp(self):
         'dbt.adapters.bigquery.connections._is_retryable', return_value=True)
     def test_retry_and_handle(self, is_retryable):
         self.connections.DEFAULT_MAXIMUM_DELAY = 2.0
-    
+
         @contextmanager
         def dummy_handler(msg):
             yield
@@ -687,7 +687,8 @@ def test_parse_partition_by(self):
                 "field": "ts",
                 "data_type": "date",
                 "granularity": "day",
-                "time_ingestion_partitioning": False
+                "time_ingestion_partitioning": False,
+                "copy_partitions": False
             }
         )
 
@@ -699,7 +700,8 @@ def test_parse_partition_by(self):
                 "field": "ts",
                 "data_type": "date",
                 "granularity": "day",
-                "time_ingestion_partitioning": False
+                "time_ingestion_partitioning": False,
+                "copy_partitions": False
             }
         )
 
@@ -713,10 +715,11 @@ def test_parse_partition_by(self):
                 "field": "ts",
                 "data_type": "date",
                 "granularity": "MONTH",
-                "time_ingestion_partitioning": False
+                "time_ingestion_partitioning": False,
+                "copy_partitions": False
             }
         )
-        
+
         self.assertEqual(
             adapter.parse_partition_by({
                 "field": "ts",
@@ -727,7 +730,8 @@ def test_parse_partition_by(self):
                 "field": "ts",
                 "data_type": "date",
                 "granularity": "YEAR",
-                "time_ingestion_partitioning": False
+                "time_ingestion_partitioning": False,
+                "copy_partitions": False
             }
         )
 
@@ -741,7 +745,8 @@ def test_parse_partition_by(self):
                 "field": "ts",
                 "data_type": "timestamp",
                 "granularity": "HOUR",
-                "time_ingestion_partitioning": False
+                "time_ingestion_partitioning": False,
+                "copy_partitions": False
             }
         )
 
@@ -756,7 +761,8 @@ def test_parse_partition_by(self):
                 "field": "ts",
                 "data_type": "timestamp",
                 "granularity": "MONTH",
-                "time_ingestion_partitioning": False
+                "time_ingestion_partitioning": False,
+                "copy_partitions": False
             }
         )
 
@@ -770,7 +776,8 @@ def test_parse_partition_by(self):
                 "field": "ts",
                 "data_type": "timestamp",
                 "granularity": "YEAR",
-                "time_ingestion_partitioning": False
+                "time_ingestion_partitioning": False,
+                "copy_partitions": False
             }
         )
 
@@ -784,7 +791,8 @@ def test_parse_partition_by(self):
                 "field": "ts",
                 "data_type": "datetime",
                 "granularity": "HOUR",
-                "time_ingestion_partitioning": False
+                "time_ingestion_partitioning": False,
+                "copy_partitions": False
             }
         )
 
@@ -798,7 +806,8 @@ def test_parse_partition_by(self):
                 "field": "ts",
                 "data_type": "datetime",
                 "granularity": "MONTH",
-                "time_ingestion_partitioning": False
+                "time_ingestion_partitioning": False,
+                "copy_partitions": False
             }
         )
 
@@ -812,20 +821,23 @@ def test_parse_partition_by(self):
                 "field": "ts",
                 "data_type": "datetime",
                 "granularity": "YEAR",
-                "time_ingestion_partitioning": False
+                "time_ingestion_partitioning": False,
+                "copy_partitions": False
             }
         )
 
         self.assertEqual(
             adapter.parse_partition_by({
                 "field": "ts",
-                "time_ingestion_partitioning": True
+                "time_ingestion_partitioning": True,
+                "copy_partitions": True
 
             }).to_dict(omit_none=True), {
                 "field": "ts",
                 "data_type": "date",
                 "granularity": "day",
-                "time_ingestion_partitioning": True
+                "time_ingestion_partitioning": True,
+                "copy_partitions": True
             }
         )
 
@@ -853,7 +865,8 @@ def test_parse_partition_by(self):
                     "end": 100,
                     "interval": 20
                 },
-                "time_ingestion_partitioning": False
+                "time_ingestion_partitioning": False,
+                "copy_partitions": False
             }
         )
 
@@ -898,7 +911,7 @@ def test_table_kms_key_name(self):
         actual = adapter.get_table_options(mock_config, node={}, temporary=False)
         self.assertEqual(expected, actual)
 
-        
+
     def test_view_kms_key_name(self):
         adapter = self.get_adapter('oauth')
         mock_config = create_autospec(
@@ -1034,4 +1047,4 @@ def test_sanitize_label_length(label_length):
         )
     with pytest.raises(dbt.exceptions.RuntimeException) as error_info:
         _sanitize_label(random_string)
-    assert error_info.value.args[0] == test_error_msg
+    assert error_info.value.args[0] == test_error_msg
\ No newline at end of file

From 11478b8d859b228f1f530eeec71244dc7fc88226 Mon Sep 17 00:00:00 2001
From: Christophe Oudar
 <90898980+github-christophe-oudar@users.noreply.github.com>
Date: Thu, 27 Oct 2022 21:56:03 +0200
Subject: [PATCH 502/860] fix args for console link logging (#363)

Co-authored-by: Christophe Oudar <kayrnt@gmail.com>
---
 .changes/unreleased/Fixes-20221026-192327.yaml | 7 +++++++
 dbt/adapters/bigquery/connections.py           | 2 +-
 2 files changed, 8 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20221026-192327.yaml

diff --git a/.changes/unreleased/Fixes-20221026-192327.yaml b/.changes/unreleased/Fixes-20221026-192327.yaml
new file mode 100644
index 000000000..515095c98
--- /dev/null
+++ b/.changes/unreleased/Fixes-20221026-192327.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: fix args for console link logging
+time: 2022-10-26T19:23:27.916326+02:00
+custom:
+  Author: Kayrnt
+  Issue: "362"
+  PR: "363"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 336db3fd1..e4643f660 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -502,7 +502,7 @@ def execute(
             message = f"{code}"
 
         if location is not None and job_id is not None and project_id is not None:
-            logger.debug(self._bq_job_link(job_id, project_id, location))
+            logger.debug(self._bq_job_link(location, project_id, job_id))
 
         response = BigQueryAdapterResponse(  # type: ignore[call-arg]
             _message=message,

From 3e6e1bfefec3dc891d043c03d53ac3103dc96f33 Mon Sep 17 00:00:00 2001
From: ericapetersson <erica@precisdigital.com>
Date: Tue, 1 Nov 2022 17:28:49 +0100
Subject: [PATCH 503/860] Update python_submissions.py (#356)

* Update python_submissions.py

* Create Fixes-20221019-115618.yaml
---
 .changes/unreleased/Fixes-20221019-115618.yaml | 7 +++++++
 dbt/adapters/bigquery/python_submissions.py    | 6 +++---
 2 files changed, 10 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20221019-115618.yaml

diff --git a/.changes/unreleased/Fixes-20221019-115618.yaml b/.changes/unreleased/Fixes-20221019-115618.yaml
new file mode 100644
index 000000000..4aef9955a
--- /dev/null
+++ b/.changes/unreleased/Fixes-20221019-115618.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: use execution_project in python models
+time: 2022-10-19T11:56:18.842401+02:00
+custom:
+  Author: ericapetersson
+  Issue: "355"
+  PR: "356"
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index d0cf9c275..43b8201d1 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -34,7 +34,7 @@ def __init__(self, parsed_model: Dict, credential: BigQueryCredentials) -> None:
         self.credential = credential
         self.GoogleCredentials = BigQueryConnectionManager.get_credentials(credential)
         self.storage_client = storage.Client(
-            project=self.credential.database, credentials=self.GoogleCredentials
+            project=self.credential.execution_project, credentials=self.GoogleCredentials
         )
         self.gcs_location = "gs://{}/{}".format(self.credential.gcs_bucket, self.model_file_name)
 
@@ -100,7 +100,7 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         }
         operation = self.job_client.submit_job_as_operation(  # type: ignore
             request={
-                "project_id": self.credential.database,
+                "project_id": self.credential.execution_project,
                 "region": self.credential.dataproc_region,
                 "job": job,
             }
@@ -136,7 +136,7 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         batch.runtime_config.properties = {
             "spark.executor.instances": "2",
         }
-        parent = f"projects/{self.credential.database}/locations/{self.credential.dataproc_region}"
+        parent = f"projects/{self.credential.execution_project}/locations/{self.credential.dataproc_region}"
         request = dataproc_v1.CreateBatchRequest(
             parent=parent,
             batch=batch,

From 51de4306d272c5c9f9739ec4bcea8108bc2c8ef3 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 3 Nov 2022 11:25:13 -0500
Subject: [PATCH 504/860] [CT-1469] init pr for short term python version issue
 fix (#375)

---
 .github/workflows/integration.yml |  2 +-
 .github/workflows/main.yml        | 12 +++++++-----
 2 files changed, 8 insertions(+), 6 deletions(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 12265849a..ce403235d 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -144,7 +144,7 @@ jobs:
           ref: ${{ github.event.pull_request.head.sha }}
 
       - name: Set up Python ${{ matrix.python-version }}
-        uses: actions/setup-python@v2
+        uses: actions/setup-python@v4.3.0
         with:
           python-version: ${{ matrix.python-version }}
 
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 9e6ef8688..bcd9d2677 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -48,7 +48,9 @@ jobs:
           persist-credentials: false
 
       - name: Set up Python
-        uses: actions/setup-python@v2
+        uses: actions/setup-python@v4.3.0
+        with:
+          python-version: '3.8'
 
       - name: Install python dependencies
         run: |
@@ -82,7 +84,7 @@ jobs:
           persist-credentials: false
 
       - name: Set up Python ${{ matrix.python-version }}
-        uses: actions/setup-python@v2
+        uses: actions/setup-python@v4.3.0
         with:
           python-version: ${{ matrix.python-version }}
 
@@ -124,9 +126,9 @@ jobs:
           persist-credentials: false
 
       - name: Set up Python
-        uses: actions/setup-python@v2
+        uses: actions/setup-python@v4.3.0
         with:
-          python-version: 3.8
+          python-version: '3.8'
 
       - name: Install python dependencies
         run: |
@@ -177,7 +179,7 @@ jobs:
 
     steps:
       - name: Set up Python ${{ matrix.python-version }}
-        uses: actions/setup-python@v2
+        uses: actions/setup-python@v4.3.0
         with:
           python-version: ${{ matrix.python-version }}
 

From 4bb481d1f1f817e87e0fd951260eebd155c65e9f Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Mon, 14 Nov 2022 14:45:10 -0600
Subject: [PATCH 505/860] [CT-1262] Convert dbt_debug integration test to
 functional test (#367)

* init push for 049 conversion

* change pointer back to core
---
 tests/functional/adapter/test_dbt_debug.py    | 15 ++++++++
 .../dbt_debug_test/models/model.sql           |  1 -
 .../integration/dbt_debug_test/test_debug.py  | 37 -------------------
 3 files changed, 15 insertions(+), 38 deletions(-)
 create mode 100644 tests/functional/adapter/test_dbt_debug.py
 delete mode 100644 tests/integration/dbt_debug_test/models/model.sql
 delete mode 100644 tests/integration/dbt_debug_test/test_debug.py

diff --git a/tests/functional/adapter/test_dbt_debug.py b/tests/functional/adapter/test_dbt_debug.py
new file mode 100644
index 000000000..d77977589
--- /dev/null
+++ b/tests/functional/adapter/test_dbt_debug.py
@@ -0,0 +1,15 @@
+import pytest
+from dbt.tests.util import run_dbt
+from dbt.tests.adapter.dbt_debug.test_dbt_debug import BaseDebug, BaseDebugProfileVariable
+
+
+class TestDebugBigQuery(BaseDebug):
+    def test_ok_bigquery(self, project):
+        run_dbt(["debug"])
+        assert "ERROR" not in self.capsys.readouterr().out
+
+
+class TestDebugProfileVariableBigQuery(BaseDebugProfileVariable):
+    def test_ok_bigquery(self, project):
+        run_dbt(["debug"])
+        assert "ERROR" not in self.capsys.readouterr().out
diff --git a/tests/integration/dbt_debug_test/models/model.sql b/tests/integration/dbt_debug_test/models/model.sql
deleted file mode 100644
index 2c2d9c8de..000000000
--- a/tests/integration/dbt_debug_test/models/model.sql
+++ /dev/null
@@ -1 +0,0 @@
-seled 1 as id
diff --git a/tests/integration/dbt_debug_test/test_debug.py b/tests/integration/dbt_debug_test/test_debug.py
deleted file mode 100644
index f057c72a7..000000000
--- a/tests/integration/dbt_debug_test/test_debug.py
+++ /dev/null
@@ -1,37 +0,0 @@
-from tests.integration.base import DBTIntegrationTest,  use_profile
-import os
-import re
-import yaml
-
-import pytest
-
-
-class TestDebug(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return 'dbt_debug'
-
-    @staticmethod
-    def dir(value):
-        return os.path.normpath(value)
-
-    @property
-    def models(self):
-        return self.dir('models')
-
-    @pytest.fixture(autouse=True)
-    def capsys(self, capsys):
-        self.capsys = capsys
-
-    @use_profile('bigquery')
-    def test_bigquery_ok(self):
-        self.run_dbt(['debug'])
-        self.assertNotIn('ERROR', self.capsys.readouterr().out)
-
-class TestDebugProfileVariable(TestDebug):
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'profile': '{{ "te" ~ "st" }}'
-        }

From 631281df3fa5f82204297d1ff84a7e92b712c519 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Tue, 15 Nov 2022 11:24:57 -0600
Subject: [PATCH 506/860] fix flake8 change (#386)

---
 .pre-commit-config.yaml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index e366015f2..c75c98b70 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -31,7 +31,7 @@ repos:
     - "--target-version=py38"
     - "--check"
     - "--diff"
-- repo: https://gitlab.com/pycqa/flake8
+- repo: https://github.com/pycqa/flake8
   rev: 4.0.1
   hooks:
   - id: flake8

From 2d4c92b94ad5ceb5b20688b51eb2033848d97592 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Tue, 15 Nov 2022 12:33:37 -0600
Subject: [PATCH 507/860] init pr fix to defer_state integration test hanging
 artifacts (#374)

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 tests/integration/defer_state_test/test_defer_state.py | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/tests/integration/defer_state_test/test_defer_state.py b/tests/integration/defer_state_test/test_defer_state.py
index d8296babf..ab319d0b1 100644
--- a/tests/integration/defer_state_test/test_defer_state.py
+++ b/tests/integration/defer_state_test/test_defer_state.py
@@ -21,6 +21,12 @@ def setUp(self):
         super().setUp()
         self._created_schemas.add(self.other_schema)
 
+    def tearDown(self):
+        with self.adapter.connection_named('__test'):
+            self._drop_schema_named(self.default_database, self.other_schema)
+
+        super().tearDown()
+
     @property
     def project_config(self):
         return {

From 9d2eb5f49696ff1288f84d1f46ed706b2e67eb00 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Mon, 28 Nov 2022 15:36:00 -0600
Subject: [PATCH 508/860] [CT-1412] proposed fix to override_database
 functional test hanging artifacts (#379)

* init pr fix to override_database functional test hanging artifacts

* spacing fix

* creating fixture to delete alt database relation

* setting alt_database env variable to a golabal variable

* reformat import, rename fixture
---
 .../test_override_database.py                 | 60 +++++++++++--------
 1 file changed, 34 insertions(+), 26 deletions(-)

diff --git a/tests/functional/test_override_database/test_override_database.py b/tests/functional/test_override_database/test_override_database.py
index 8ce179056..a328809bc 100644
--- a/tests/functional/test_override_database/test_override_database.py
+++ b/tests/functional/test_override_database/test_override_database.py
@@ -1,13 +1,16 @@
 import pytest
-from dbt.tests.util import run_dbt, check_relations_equal, check_relations_equal_with_relations
+import os
+from dbt.tests.util import (
+    run_dbt,
+    check_relations_equal_with_relations
+)
 from tests.functional.test_override_database.fixtures import (
     models,
     seeds,
     project_files
 )
-import os
-
 
+ALT_DATABASE = os.getenv("BIGQUERY_TEST_ALT_DATABASE")
 
 
 class BaseOverrideDatabase:
@@ -21,7 +24,7 @@ def project_config_update(self):
             "config-version": 2,
             "seed-paths": ["seeds"],
             "vars": {
-                "alternate_db": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+                "alternate_db": ALT_DATABASE,
             },
             "quoting": {
                 "database": True,
@@ -31,6 +34,12 @@ def project_config_update(self):
             }
         }
 
+    @pytest.fixture(scope="function")
+    def clean_up(self, project):
+        yield
+        relation = project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema)
+        project.adapter.drop_schema(relation)
+
 
 class TestModelOverrideBigQuery(BaseOverrideDatabase):
     def run_database_override(self, project):
@@ -38,19 +47,17 @@ def run_database_override(self, project):
         assert len(run_dbt(["run"])) == 4
         check_relations_equal_with_relations(project.adapter, [
             project.adapter.Relation.create(schema=project.test_schema, identifier="seed"),
-            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_2"),
+            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_2"),
             project.adapter.Relation.create(schema=project.test_schema, identifier="view_1"),
             project.adapter.Relation.create(schema=project.test_schema, identifier="view_3"),
-            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_4")
+            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_4")
         ])
 
-
-    def test_bigquery_database_override(self, project):
+    def test_bigquery_database_override(self, project, clean_up):
         self.run_database_override(project)
 
 
 class BaseTestProjectModelOverrideBigQuery(BaseOverrideDatabase):
-
     def run_database_override(self, project):
         run_dbt(["seed"])
         assert len(run_dbt(["run"])) == 4
@@ -59,10 +66,10 @@ def run_database_override(self, project):
     def assertExpectedRelations(self, project):
         check_relations_equal_with_relations(project.adapter, [
             project.adapter.Relation.create(schema=project.test_schema, identifier="seed"),
-            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_2"),
-            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_1"),
+            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_2"),
+            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_1"),
             project.adapter.Relation.create(schema=project.test_schema, identifier="view_3"),
-            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_4")
+            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_4")
         ])
 
 
@@ -72,10 +79,10 @@ def project_config_update(self):
         return {
             "config-version": 2,
             "vars": {
-                "alternate_db": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+                "alternate_db": ALT_DATABASE,
             },
             "models": {
-                "database": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+                "database": ALT_DATABASE,
                 "test": {
                     "subfolder": {
                         "database": "{{ target.database }}"
@@ -84,7 +91,7 @@ def project_config_update(self):
             },
             "seed-paths": ["seeds"],
             "vars": {
-                "alternate_db": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+                "alternate_db": ALT_DATABASE,
             },
             "quoting": {
                 "database": True,
@@ -94,7 +101,7 @@ def project_config_update(self):
             }
         }
 
-    def test_bigquery_database_override(self, project):
+    def test_bigquery_database_override(self, project, clean_up):
         self.run_database_override(project)
 
 
@@ -104,10 +111,10 @@ def project_config_update(self):
         return {
             "config-version": 2,
             "vars": {
-                "alternate_db": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+                "alternate_db": ALT_DATABASE,
             },
             "models": {
-                "project": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+                "project": ALT_DATABASE,
                 "test": {
                     "subfolder": {
                         "project": "{{ target.database }}"
@@ -116,7 +123,7 @@ def project_config_update(self):
             },
             "seed-paths": ["seeds"],
             "vars": {
-                "alternate_db": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+                "alternate_db": ALT_DATABASE,
             },
             "quoting": {
                 "database": True,
@@ -126,7 +133,7 @@ def project_config_update(self):
             }
         }
 
-    def test_bigquery_project_override(self, project):
+    def test_bigquery_project_override(self, project, clean_up):
         self.run_database_override(project)
 
 
@@ -137,22 +144,23 @@ def project_config_update(self):
             "config-version": 2,
             "seed-paths": ["seeds"],
             "vars": {
-                "alternate_db": os.getenv("BIGQUERY_TEST_ALT_DATABASE"),
+                "alternate_db": ALT_DATABASE,
             },
             "seeds": {
-                "database": os.getenv("BIGQUERY_TEST_ALT_DATABASE")
+                "database": ALT_DATABASE
             }
         }
+
     def run_database_override(self, project):
         run_dbt(["seed"])
         assert len(run_dbt(["run"])) == 4
         check_relations_equal_with_relations(project.adapter, [
-            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="seed"),
-            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_2"),
+            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="seed"),
+            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_2"),
             project.adapter.Relation.create(schema=project.test_schema, identifier="view_1"),
             project.adapter.Relation.create(schema=project.test_schema, identifier="view_3"),
-            project.adapter.Relation.create(database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema, identifier="view_4")
+            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_4")
         ])
 
-    def test_bigquery_database_override(self, project):
+    def test_bigquery_database_override(self, project, clean_up):
         self.run_database_override(project)

From d4993cc848214f997c4a0cf1716656fc7a70bd02 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Fri, 2 Dec 2022 00:26:16 -0600
Subject: [PATCH 509/860] [CT-1412] test_aliases functional test hanging
 artifacts (#382)

* [CT-1412] test_aliases functional test hanging artifacts

* reformatting of test after some spike investigation, adding type ignore for mypy

* reformatting test again

* change core pointer

* change core pointer for testing
---
 dbt/adapters/bigquery/column.py          |  2 +-
 dbt/adapters/bigquery/impl.py            |  2 +-
 tests/functional/adapter/test_aliases.py | 62 +++++++++++++++++++++---
 3 files changed, 58 insertions(+), 8 deletions(-)

diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py
index 0f9269283..158f6650c 100644
--- a/dbt/adapters/bigquery/column.py
+++ b/dbt/adapters/bigquery/column.py
@@ -125,4 +125,4 @@ def column_to_bq_schema(self) -> SchemaField:
             fields = [field.column_to_bq_schema() for field in self.fields]  # type: ignore[attr-defined]
             kwargs = {"fields": fields}
 
-        return SchemaField(self.name, self.dtype, self.mode, **kwargs)
+        return SchemaField(self.name, self.dtype, self.mode, **kwargs)  # type: ignore[arg-type]
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 8597db6f1..b59aff6c3 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -398,7 +398,7 @@ def _agate_to_schema(
         for idx, col_name in enumerate(agate_table.column_names):
             inferred_type = self.convert_agate_type(agate_table, idx)
             type_ = column_override.get(col_name, inferred_type)
-            bq_schema.append(SchemaField(col_name, type_))
+            bq_schema.append(SchemaField(col_name, type_))  # type: ignore[arg-type]
         return bq_schema
 
     def _materialize_as_view(self, model: Dict[str, Any]) -> str:
diff --git a/tests/functional/adapter/test_aliases.py b/tests/functional/adapter/test_aliases.py
index 2b4cdd381..d64a944fa 100644
--- a/tests/functional/adapter/test_aliases.py
+++ b/tests/functional/adapter/test_aliases.py
@@ -1,17 +1,18 @@
 import pytest
 import os
-from dbt.tests.adapter.aliases.test_aliases import BaseAliases, BaseSameAliasDifferentDatabases
+from dbt.tests.util import run_dbt
+from dbt.tests.adapter.aliases.test_aliases import (
+    BaseAliases,
+    BaseSameAliasDifferentDatabases
+)
 
 MACROS__BIGQUERY_CAST_SQL = """
-
 {% macro bigquery__string_literal(s) %}
     cast('{{ s }}' as string)
 {% endmacro %}
-
 """
 
 MACROS__EXPECT_VALUE_SQL = """
-
 -- cross-db compatible test, similar to accepted_values
 
 {% test expect_value(model, field, value) %}
@@ -21,13 +22,38 @@
 where {{ field }} != '{{ value }}'
 
 {% endtest %}
+"""
 
+MODELS_DUPE_CUSTOM_DATABASE_A = """
+select {{ string_literal(this.name) }} as tablename
+"""
+
+MODELS_DUPE_CUSTOM_DATABASE_B = """
+select {{ string_literal(this.name) }} as tablename
+"""
+
+MODELS_SCHEMA_YML = """
+version: 2
+models:
+- name: model_a
+  tests:
+  - expect_value:
+      field: tablename
+      value: duped_alias
+- name: model_b
+  tests:
+  - expect_value:
+      field: tablename
+      value: duped_alias
 """
 
 class TestAliasesBigQuery(BaseAliases):
     @pytest.fixture(scope="class")
     def macros(self):
-        return {"bigquery_cast.sql": MACROS__BIGQUERY_CAST_SQL, "expect_value.sql": MACROS__EXPECT_VALUE_SQL}
+        return {
+            "bigquery_cast.sql": MACROS__BIGQUERY_CAST_SQL,
+            "expect_value.sql": MACROS__EXPECT_VALUE_SQL
+        }
 
 
 class TestSameTestSameAliasDifferentDatabasesBigQuery(BaseSameAliasDifferentDatabases):
@@ -43,6 +69,30 @@ def project_config_update(self):
                 },
             },
         }
+
     @pytest.fixture(scope="class")
     def macros(self):
-        return {"bigquery_cast.sql": MACROS__BIGQUERY_CAST_SQL, "expect_value.sql": MACROS__EXPECT_VALUE_SQL}
+        return {
+            "bigquery_cast.sql": MACROS__BIGQUERY_CAST_SQL,
+            "expect_value.sql": MACROS__EXPECT_VALUE_SQL
+        }
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "schema.yml": MODELS_SCHEMA_YML,
+            "model_a.sql": MODELS_DUPE_CUSTOM_DATABASE_A,
+            "model_b.sql": MODELS_DUPE_CUSTOM_DATABASE_B
+        }
+
+    @pytest.fixture(autouse=True)
+    def clean_up(self, project):
+        yield
+        with project.adapter.connection_named('__test'):
+            relation = project.adapter.Relation.create(
+                database=os.getenv(
+                "BIGQUERY_TEST_ALT_DATABASE"
+                ),
+                 schema=project.test_schema
+            )
+            project.adapter.drop_schema(relation)

From 17916264a5cc95852c44ebbb11b673661a780c67 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 7 Dec 2022 15:29:12 -0600
Subject: [PATCH 510/860] [CT-1405] Bigquery minor changes for event logging
 (#410)

* adding StringIO to run_dbt_and_capture to take into account newly added event logging stuff for core

* add changelog
---
 .changes/unreleased/Under the Hood-20221207-103505.yaml | 7 +++++++
 tests/integration/base.py                               | 7 ++++---
 2 files changed, 11 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20221207-103505.yaml

diff --git a/.changes/unreleased/Under the Hood-20221207-103505.yaml b/.changes/unreleased/Under the Hood-20221207-103505.yaml
new file mode 100644
index 000000000..ea1e3a62f
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20221207-103505.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: event logging changes to be in parity with dbt-core post pr#6291
+time: 2022-12-07T10:35:05.567885-06:00
+custom:
+  Author: McKnight-42
+  Issue: "6139"
+  PR: "410"
diff --git a/tests/integration/base.py b/tests/integration/base.py
index ec6769e8e..f8062f802 100644
--- a/tests/integration/base.py
+++ b/tests/integration/base.py
@@ -1,6 +1,5 @@
 import json
 import os
-import io
 import random
 import shutil
 import sys
@@ -8,6 +7,7 @@
 import traceback
 import unittest
 import warnings
+from io import StringIO
 from contextlib import contextmanager
 from datetime import datetime
 from functools import wraps
@@ -231,7 +231,7 @@ def _generate_test_root_dir(self):
         return normalize(tempfile.mkdtemp(prefix='dbt-int-test-'))
 
     def setUp(self):
-        # Logbook warnings are ignored so we don't have to fork logbook to support python 3.10. 
+        # Logbook warnings are ignored so we don't have to fork logbook to support python 3.10.
         # This _only_ works for tests in `test/integration`.
         warnings.filterwarnings(
             "ignore",
@@ -417,7 +417,8 @@ def run_dbt(self, args=None, expect_pass=True, profiles_dir=True):
 
     def run_dbt_and_capture(self, *args, **kwargs):
         try:
-            stringbuf = capture_stdout_logs()
+            stringbuf = StringIO()
+            capture_stdout_logs(stringbuf)
             res = self.run_dbt(*args, **kwargs)
             stdout = stringbuf.getvalue()
 

From 24bed99bfb48d76d36785362b86e1d9dba9c1ae9 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 7 Dec 2022 15:31:26 -0600
Subject: [PATCH 511/860] updating passenv in tox (#411)

* fixing env write out in tox due to 4.0.0 release

* finish tox alignment and add changelog
---
 .../unreleased/Under the Hood-20221207-151813.yaml     |  8 ++++++++
 tox.ini                                                | 10 ++++++++--
 2 files changed, 16 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20221207-151813.yaml

diff --git a/.changes/unreleased/Under the Hood-20221207-151813.yaml b/.changes/unreleased/Under the Hood-20221207-151813.yaml
new file mode 100644
index 000000000..32084a6a7
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20221207-151813.yaml	
@@ -0,0 +1,8 @@
+kind: Under the Hood
+body: fix issue with tox 4.0.0 release which changes passenv syntax for space-separated
+  variables
+time: 2022-12-07T15:18:13.996118-06:00
+custom:
+  Author: McKnight-42
+  Issue: "411"
+  PR: "411"
diff --git a/tox.ini b/tox.ini
index 38797c0cf..037b36f4f 100644
--- a/tox.ini
+++ b/tox.ini
@@ -5,7 +5,9 @@ envlist = py37,py38,py39,py310
 [testenv:{unit,py37,py38,py39,py310,py}]
 description = unit testing
 skip_install = true
-passenv = DBT_* PYTEST_ADDOPTS
+passenv =
+    DBT_*
+    PYTEST_ADDOPTS
 commands = {envpython} -m pytest {posargs} tests/unit
 deps =
   -rdev-requirements.txt
@@ -14,7 +16,11 @@ deps =
 [testenv:{integration,py37,py38,py39,py310,py}-{bigquery}]
 description = adapter plugin integration testing
 skip_install = true
-passenv = DBT_* BIGQUERY_TEST_* PYTEST_ADDOPTS DATAPROC_* GCS_BUCKET
+passenv =
+    DBT_*
+    BIGQUERY_TEST_*
+    PYTEST_ADDOPTS DATAPROC_*
+    GCS_BUCKET
 commands =
   bigquery: {envpython} -m pytest {posargs} -m profile_bigquery tests/integration
   bigquery: {envpython} -m pytest {posargs} -vv tests/functional --profile service_account

From 32085f91ea973745a5cc631fe202f49b3e47204d Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 8 Dec 2022 11:55:08 -0600
Subject: [PATCH 512/860] format fix for tox changes (#417)

* fixing env write out in tox due to 4.0.0 release

* finish tox alignment and add changelog

* minor fix for format

* add changelog

* remove uneeded changie entry
---
 tox.ini | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/tox.ini b/tox.ini
index 037b36f4f..de228d534 100644
--- a/tox.ini
+++ b/tox.ini
@@ -19,7 +19,8 @@ skip_install = true
 passenv =
     DBT_*
     BIGQUERY_TEST_*
-    PYTEST_ADDOPTS DATAPROC_*
+    PYTEST_ADDOPTS
+    DATAPROC_*
     GCS_BUCKET
 commands =
   bigquery: {envpython} -m pytest {posargs} -m profile_bigquery tests/integration

From 2d044e41b1d7f52e702ea5e1ee15433a7f1d2001 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Tue, 13 Dec 2022 15:27:39 -0600
Subject: [PATCH 513/860] add release workflow (#430)

---
 .github/workflows/release.yml | 211 ++++++++++++++++++++++++++++++++++
 1 file changed, 211 insertions(+)
 create mode 100644 .github/workflows/release.yml

diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml
new file mode 100644
index 000000000..fa3d6b397
--- /dev/null
+++ b/.github/workflows/release.yml
@@ -0,0 +1,211 @@
+# **what?**
+# Take the given commit, run unit tests specifically on that sha, build and
+# package it, and then release to GitHub with that specific build (PyPi to follow later)
+
+# **why?**
+# Ensure an automated and tested release process
+
+# **when?**
+# This will only run manually with a given sha and version
+
+name: Build, Test, and Package
+
+on:
+  workflow_call:
+    inputs:
+      sha:
+        description: "The last commit sha in the release"
+        type: string
+        required: true
+      changelog_path:
+        description: "Path to changes log"
+        type: string
+        default: "./CHANGELOG.md"
+        required: false
+      version_number:
+        description: "The release version number (i.e. 1.0.0b1)"
+        type: string
+        required: true
+      test_run:
+        description: "Test run (Publish release as draft to GitHub)"
+        type: boolean
+        default: false
+        required: false
+
+permissions:
+  contents: write # this is the permission that allows creating a new release
+
+env:
+  PYTHON_TARGET_VERSION: 3.8
+  ARTIFACT_RETENTION_DAYS: 2
+
+defaults:
+  run:
+    shell: bash
+
+jobs:
+  log-inputs:
+    name: Log Inputs
+    runs-on: ubuntu-latest
+    steps:
+      - name: "[DEBUG] Print Variables"
+        run: |
+          echo The last commit sha in the release:  ${{ inputs.sha }}
+          echo The release version number:          ${{ inputs.version_number }}
+          echo The path to the changelog markdpown: ${{ inputs.changelog_path }}
+          echo This is a test run:                  ${{ inputs.test_run }}
+          echo Python target version:               ${{ env.PYTHON_TARGET_VERSION }}
+          echo Artifact retention days:             ${{ env.ARTIFACT_RETENTION_DAYS }}
+
+  unit:
+    name: Unit Test
+    runs-on: ubuntu-latest
+
+    env:
+      TOXENV: "unit"
+
+    steps:
+      - name: "Checkout Commit - ${{ inputs.sha }}"
+        uses: actions/checkout@v3
+        with:
+          persist-credentials: false
+          ref: ${{ github.event.inputs.sha }}
+
+      - name: "Set up Python - ${{ env.PYTHON_TARGET_VERSION }}"
+        uses: actions/setup-python@v4
+        with:
+          python-version: ${{ env.PYTHON_TARGET_VERSION }}
+
+      - name: "Install Python Dependencies"
+        run: |
+          python -m pip install --user --upgrade pip
+          python -m pip install tox
+          python -m pip --version
+          python -m tox --version
+
+      - name: "Run Tox"
+        run: tox
+
+  build:
+    name: Build Packages
+
+    runs-on: ubuntu-latest
+
+    steps:
+      - name: "Checkout Commit - ${{ inputs.sha }}"
+        uses: actions/checkout@v3
+        with:
+          persist-credentials: false
+          ref: ${{ inputs.sha }}
+
+      - name: "Set up Python - ${{ env.PYTHON_TARGET_VERSION }}"
+        uses: actions/setup-python@v4
+        with:
+          python-version: ${{ env.PYTHON_TARGET_VERSION }}
+
+      - name: "Install Python Dependencies"
+        run: |
+          python -m pip install --user --upgrade pip
+          python -m pip install --upgrade setuptools wheel twine check-wheel-contents
+          python -m pip --version
+
+      - name: "Build Distributions"
+        run: ./scripts/build-dist.sh
+
+      - name: "[DEBUG] Show Distributions"
+        run: ls -lh dist/
+
+      - name: "Check Distribution Descriptions"
+        run: |
+          twine check dist/*
+
+      - name: "[DEBUG] Check Wheel Contents"
+        run: |
+          check-wheel-contents dist/*.whl --ignore W007,W008
+
+      - name: "Upload Build Artifact - ${{ inputs.version_number }}"
+        uses: actions/upload-artifact@v3
+        with:
+          name: ${{ inputs.version_number }}
+          path: |
+            dist/
+            !dist/dbt-${{ inputs.version_number }}.tar.gz
+          retention-days: ${{ env.ARTIFACT_RETENTION_DAYS }}
+
+  test-build:
+    name: Verify Packages
+
+    needs: [unit, build]
+
+    runs-on: ubuntu-latest
+
+    steps:
+      - name: "Set up Python - ${{ env.PYTHON_TARGET_VERSION }}"
+        uses: actions/setup-python@v4
+        with:
+          python-version: ${{ env.PYTHON_TARGET_VERSION }}
+
+      - name: "Install Python Dependencies"
+        run: |
+          python -m pip install --user --upgrade pip
+          python -m pip install --upgrade wheel
+          python -m pip --version
+
+      - name: "Download Build Artifact - ${{ inputs.version_number }}"
+        uses: actions/download-artifact@v3
+        with:
+          name: ${{ inputs.version_number }}
+          path: dist/
+
+      - name: "[DEBUG] Show Distributions"
+        run: ls -lh dist/
+
+      - name: "Install Wheel Distributions"
+        run: |
+          find ./dist/*.whl -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/
+
+      - name: "[DEBUG] Check Wheel Distributions"
+        run: |
+          dbt --version
+
+      - name: "Install Source Distributions"
+        run: |
+          find ./dist/*.gz -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/
+
+      - name: "[DEBUG] Check Source Distributions"
+        run: |
+          dbt --version
+
+  github-release:
+    name: GitHub Release
+    if: ${{ !failure() && !cancelled() }}
+    needs: build-test-package
+
+    # pin to commit since this is workflow is WIP but this commit has been tested as working
+    uses: dbt-labs/dbt-release/.github/workflows/github-release.yml@7b6e01d
+
+    with:
+      sha: ${{ inputs.sha }}
+      version_number: ${{ inputs.version_number }}
+      changelog_path: ${{ inputs.changelog_path }}
+      test_run: ${{ inputs.test_run }}
+
+# Skipping this for now until we've proven build work in the repos
+  # pypi-release:
+  #   name: Pypi release
+
+  #   runs-on: ubuntu-latest
+
+  #   needs: github-release
+
+  #   environment: PypiProd
+  #   steps:
+  #     - uses: actions/download-artifact@v2
+  #       with:
+  #         name: dist
+  #         path: 'dist'
+
+  #     - name: Publish distribution to PyPI
+  #       uses: pypa/gh-action-pypi-publish@v1.4.2
+  #       with:
+  #         password: ${{ secrets.PYPI_API_TOKEN }}

From 597459cfcbef9abc6dea4deebc5eabd586125d14 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Wed, 14 Dec 2022 10:15:23 -0600
Subject: [PATCH 514/860] fix footer, update contributors (#433)

---
 .changes/0.0.0.md | 1 +
 .changie.yaml     | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)

diff --git a/.changes/0.0.0.md b/.changes/0.0.0.md
index 9c545f562..f27e1ab71 100644
--- a/.changes/0.0.0.md
+++ b/.changes/0.0.0.md
@@ -1,5 +1,6 @@
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
+- [1.3](https://github.com/dbt-labs/dbt-bigquery/blob/1.3.latest/CHANGELOG.md)
 - [1.2](https://github.com/dbt-labs/dbt-bigquery/blob/1.2.latest/CHANGELOG.md)
 - [1.1](https://github.com/dbt-labs/dbt-bigquery/blob/1.1.latest/CHANGELOG.md)
 - [1.0](https://github.com/dbt-labs/dbt-bigquery/blob/1.0.latest/CHANGELOG.md)
diff --git a/.changie.yaml b/.changie.yaml
index 88670f09c..fc7a42cd5 100644
--- a/.changie.yaml
+++ b/.changie.yaml
@@ -32,7 +32,7 @@ custom:
 footerFormat: |
   {{- $contributorDict := dict }}
   {{- /* any names added to this list should be all lowercase for later matching purposes */}}
-  {{- $core_team := list "emmyoop" "nathaniel-may" "gshank" "leahwicz" "chenyulinx" "stu-k" "iknox-fa" "versusfacit" "mcknight-42" "jtcohen6" "dependabot[bot]" "snyk-bot" }}
+  {{- $core_team := list "michelleark" "peterallenwebb" "emmyoop" "nathaniel-may" "gshank" "leahwicz" "chenyulinx" "stu-k" "iknox-fa" "versusfacit" "mcknight-42" "jtcohen6" "aranke" "mikealfare" "dependabot[bot]" "snyk-bot" "colin-rogers-dbt" }}
   {{- range $change := .Changes }}
     {{- $authorList := splitList " " $change.Custom.Author }}
     {{- /* loop through all authors for a PR */}}

From 33eabc69c7a72745012b27ccc34638f0e14e8ed2 Mon Sep 17 00:00:00 2001
From: dave-connors-3 <73915542+dave-connors-3@users.noreply.github.com>
Date: Thu, 15 Dec 2022 10:34:18 -0600
Subject: [PATCH 515/860] Feature/incremental predicates support (#284)

* add support for custom predicates on merge incremental strategy

* incremental predicate tests

* changie

* pin dbt core branch

* ye olde typo

* functional test for BQ merge with incremental predicates

* remove old integration test

* handle predicates keyword, add test

* unpin feature branch

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .../unreleased/Features-20220823-085601.yaml  |  7 +++++
 .../macros/materializations/incremental.sql   |  7 ++---
 .../incremental_strategy/merge.sql            |  4 +--
 .../adapter/test_incremental_predicates.py    | 27 +++++++++++++++++++
 .../simple_copy_test/test_simple_copy.py      |  1 +
 5 files changed, 41 insertions(+), 5 deletions(-)
 create mode 100644 .changes/unreleased/Features-20220823-085601.yaml
 create mode 100644 tests/functional/adapter/test_incremental_predicates.py

diff --git a/.changes/unreleased/Features-20220823-085601.yaml b/.changes/unreleased/Features-20220823-085601.yaml
new file mode 100644
index 000000000..d8bcd51bc
--- /dev/null
+++ b/.changes/unreleased/Features-20220823-085601.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: incremental predicates
+time: 2022-08-23T08:56:01.043831-05:00
+custom:
+  Author: dave-connors-3
+  Issue: "283"
+  PR: "284"
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index 9ed100405..88fc91eae 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -33,7 +33,7 @@
 {% endmacro %}
 
 {% macro bq_generate_incremental_build_sql(
-    strategy, tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions
+    strategy, tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions, incremental_predicates
 ) %}
   {#-- if partitioned, use BQ scripting to get the range of partition values to be updated --#}
   {% if strategy == 'insert_overwrite' %}
@@ -45,7 +45,7 @@
   {% else %} {# strategy == 'merge' #}
 
     {% set build_sql = bq_generate_incremental_merge_build_sql(
-        tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists
+        tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists, incremental_predicates
     ) %}
 
   {% endif %}
@@ -73,6 +73,7 @@
   {%- set cluster_by = config.get('cluster_by', none) -%}
 
   {% set on_schema_change = incremental_validate_on_schema_change(config.get('on_schema_change'), default='ignore') %}
+  {% set incremental_predicates = config.get('predicates', default=none) or config.get('incremental_predicates', default=none) %}
 
    -- grab current tables grants config for comparison later on
   {% set grant_config = config.get('grants') %}
@@ -137,7 +138,7 @@
       {% set dest_columns = adapter.add_time_ingestion_partition_column(dest_columns) %}
     {% endif %}
     {% set build_sql = bq_generate_incremental_build_sql(
-        strategy, tmp_relation, target_relation, compiled_code, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, partition_by.copy_partitions
+        strategy, tmp_relation, target_relation, compiled_code, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, partition_by.copy_partitions, incremental_predicates
     ) %}
 
     {%- call statement('main') -%}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
index e6ce91c88..5efb079b6 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
@@ -1,5 +1,5 @@
 {% macro bq_generate_incremental_merge_build_sql(
-    tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists
+    tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists, incremental_predicates
 ) %}
     {%- set source_sql -%}
         {%- if tmp_relation_exists -%}
@@ -21,7 +21,7 @@
         {%- endif -%}
     {%- endset -%}
 
-    {% set build_sql = get_merge_sql(target_relation, source_sql, unique_key, dest_columns) %}
+    {% set build_sql = get_merge_sql(target_relation, source_sql, unique_key, dest_columns, incremental_predicates) %}
 
     {{ return(build_sql) }}
 
diff --git a/tests/functional/adapter/test_incremental_predicates.py b/tests/functional/adapter/test_incremental_predicates.py
new file mode 100644
index 000000000..cebd17bab
--- /dev/null
+++ b/tests/functional/adapter/test_incremental_predicates.py
@@ -0,0 +1,27 @@
+import pytest
+from dbt.tests.adapter.incremental.test_incremental_predicates import BaseIncrementalPredicates
+
+
+class TestIncrementalPredicatesMergeBigQuery(BaseIncrementalPredicates):
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            "models": { 
+                "+incremental_predicates": [
+                    "dbt_internal_dest.id != 2"
+                    ],
+                "+incremental_strategy": "merge"
+            }
+        }
+
+class TestPredicatesMergeBigQuery(BaseIncrementalPredicates):
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            "models": { 
+                "+predicates": [
+                    "dbt_internal_dest.id != 2"
+                    ],
+                "+incremental_strategy": "merge"
+            }
+        }
\ No newline at end of file
diff --git a/tests/integration/simple_copy_test/test_simple_copy.py b/tests/integration/simple_copy_test/test_simple_copy.py
index e388dede7..45a9838e7 100644
--- a/tests/integration/simple_copy_test/test_simple_copy.py
+++ b/tests/integration/simple_copy_test/test_simple_copy.py
@@ -94,3 +94,4 @@ def test__bigquery__incremental_merge_columns(self):
         })
         self.seed_and_run()
         self.assertTablesEqual("incremental_update_cols", "expected_result")
+

From b14b39193eceda6680052525a95cbc8340760200 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Thu, 15 Dec 2022 11:57:25 -0600
Subject: [PATCH 516/860] swap trigger, update commit sha on release workflow
 (#435)

* swap trigger, update commit sha

* update needs
---
 .github/workflows/release.yml | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml
index fa3d6b397..2eea6e8cc 100644
--- a/.github/workflows/release.yml
+++ b/.github/workflows/release.yml
@@ -11,7 +11,7 @@
 name: Build, Test, and Package
 
 on:
-  workflow_call:
+  workflow_dispatch:
     inputs:
       sha:
         description: "The last commit sha in the release"
@@ -179,10 +179,10 @@ jobs:
   github-release:
     name: GitHub Release
     if: ${{ !failure() && !cancelled() }}
-    needs: build-test-package
+    needs: test-build
 
     # pin to commit since this is workflow is WIP but this commit has been tested as working
-    uses: dbt-labs/dbt-release/.github/workflows/github-release.yml@7b6e01d
+    uses: dbt-labs/dbt-release/.github/workflows/github-release.yml@7b6e01d73d2c8454e06302cc66ef4c2dbd4dbe4e
 
     with:
       sha: ${{ inputs.sha }}

From 82e18a559ee13701cc9be9519f13b98e5c464d4c Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Thu, 15 Dec 2022 12:31:58 -0600
Subject: [PATCH 517/860] Bumping version to 1.4.0b1 and generate changelog
 (#434)

* Bumping version to 1.4.0b1 and generate CHANGELOG

* Update 1.4.0-b1.md

* Update CHANGELOG.md

Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Emily Rockman <emily.rockman@dbtlabs.com>
---
 .bumpversion.cfg                               |  2 +-
 .changes/1.4.0-b1.md                           | 18 ++++++++++++++++++
 .../Dependencies-20221010-181501.yaml          |  0
 .../Features-20220807-164227.yaml              |  0
 .../Features-20220926-105700.yaml              |  0
 .../Features-20221020-223914.yaml              |  0
 .../Fixes-20221019-115618.yaml                 |  0
 .../Fixes-20221026-192327.yaml                 |  0
 .../Under the Hood-20221207-103505.yaml        |  0
 .../Under the Hood-20221207-151813.yaml        |  0
 CHANGELOG.md                                   | 17 +++++++++++++++++
 dbt/adapters/bigquery/__version__.py           |  2 +-
 setup.py                                       |  2 +-
 13 files changed, 38 insertions(+), 3 deletions(-)
 create mode 100644 .changes/1.4.0-b1.md
 rename .changes/{unreleased => 1.4.0}/Dependencies-20221010-181501.yaml (100%)
 rename .changes/{unreleased => 1.4.0}/Features-20220807-164227.yaml (100%)
 rename .changes/{unreleased => 1.4.0}/Features-20220926-105700.yaml (100%)
 rename .changes/{unreleased => 1.4.0}/Features-20221020-223914.yaml (100%)
 rename .changes/{unreleased => 1.4.0}/Fixes-20221019-115618.yaml (100%)
 rename .changes/{unreleased => 1.4.0}/Fixes-20221026-192327.yaml (100%)
 rename .changes/{unreleased => 1.4.0}/Under the Hood-20221207-103505.yaml (100%)
 rename .changes/{unreleased => 1.4.0}/Under the Hood-20221207-151813.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 047a5cd7d..487f15b2a 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.4.0a1
+current_version = 1.4.0b1
 parse = (?P<major>\d+)
 	\.(?P<minor>\d+)
 	\.(?P<patch>\d+)
diff --git a/.changes/1.4.0-b1.md b/.changes/1.4.0-b1.md
new file mode 100644
index 000000000..22f898ceb
--- /dev/null
+++ b/.changes/1.4.0-b1.md
@@ -0,0 +1,18 @@
+## dbt-bigquery 1.4.0-b1 - December 15, 2022
+### Features
+- Support for ingestion time partition table on BigQuery as incremental materialization ([#75](https://github.com/dbt-labs/dbt-bigquery/issues/75), [#136](https://github.com/dbt-labs/dbt-bigquery/pull/136))
+- Migrate dbt-utils current_timestamp macros into core + adapters ([#324](https://github.com/dbt-labs/dbt-bigquery/issues/324), [#323](https://github.com/dbt-labs/dbt-bigquery/pull/323))
+- Optimize insert_overwrite incremental strategy with WRITE_TRUNCATE / Partition copy ([#77](https://github.com/dbt-labs/dbt-bigquery/issues/77), [#167](https://github.com/dbt-labs/dbt-bigquery/pull/167))
+### Fixes
+- use execution_project in python models ([#355](https://github.com/dbt-labs/dbt-bigquery/issues/355), [#356](https://github.com/dbt-labs/dbt-bigquery/pull/356))
+- fix args for console link logging ([#362](https://github.com/dbt-labs/dbt-bigquery/issues/362), [#363](https://github.com/dbt-labs/dbt-bigquery/pull/363))
+### Under the Hood
+- event logging changes to be in parity with dbt-core post pr#6291 ([#6139](https://github.com/dbt-labs/dbt-bigquery/issues/6139), [#410](https://github.com/dbt-labs/dbt-bigquery/pull/410))
+- fix issue with tox 4.0.0 release which changes passenv syntax for space-separated variables ([#411](https://github.com/dbt-labs/dbt-bigquery/issues/411), [#411](https://github.com/dbt-labs/dbt-bigquery/pull/411))
+### Dependencies
+- Bump google-cloud-bigquery dependency to < 3.3.3 ([#269](https://github.com/dbt-labs/dbt-bigquery/issues/269), [#332](https://github.com/dbt-labs/dbt-bigquery/pull/332))
+
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#136](https://github.com/dbt-labs/dbt-bigquery/pull/136), [#167](https://github.com/dbt-labs/dbt-bigquery/pull/167), [#363](https://github.com/dbt-labs/dbt-bigquery/pull/363))
+- [@eplus-aolalere](https://github.com/eplus-aolalere) ([#332](https://github.com/dbt-labs/dbt-bigquery/pull/332))
+- [@ericapetersson](https://github.com/ericapetersson) ([#356](https://github.com/dbt-labs/dbt-bigquery/pull/356))
diff --git a/.changes/unreleased/Dependencies-20221010-181501.yaml b/.changes/1.4.0/Dependencies-20221010-181501.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20221010-181501.yaml
rename to .changes/1.4.0/Dependencies-20221010-181501.yaml
diff --git a/.changes/unreleased/Features-20220807-164227.yaml b/.changes/1.4.0/Features-20220807-164227.yaml
similarity index 100%
rename from .changes/unreleased/Features-20220807-164227.yaml
rename to .changes/1.4.0/Features-20220807-164227.yaml
diff --git a/.changes/unreleased/Features-20220926-105700.yaml b/.changes/1.4.0/Features-20220926-105700.yaml
similarity index 100%
rename from .changes/unreleased/Features-20220926-105700.yaml
rename to .changes/1.4.0/Features-20220926-105700.yaml
diff --git a/.changes/unreleased/Features-20221020-223914.yaml b/.changes/1.4.0/Features-20221020-223914.yaml
similarity index 100%
rename from .changes/unreleased/Features-20221020-223914.yaml
rename to .changes/1.4.0/Features-20221020-223914.yaml
diff --git a/.changes/unreleased/Fixes-20221019-115618.yaml b/.changes/1.4.0/Fixes-20221019-115618.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20221019-115618.yaml
rename to .changes/1.4.0/Fixes-20221019-115618.yaml
diff --git a/.changes/unreleased/Fixes-20221026-192327.yaml b/.changes/1.4.0/Fixes-20221026-192327.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20221026-192327.yaml
rename to .changes/1.4.0/Fixes-20221026-192327.yaml
diff --git a/.changes/unreleased/Under the Hood-20221207-103505.yaml b/.changes/1.4.0/Under the Hood-20221207-103505.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20221207-103505.yaml
rename to .changes/1.4.0/Under the Hood-20221207-103505.yaml
diff --git a/.changes/unreleased/Under the Hood-20221207-151813.yaml b/.changes/1.4.0/Under the Hood-20221207-151813.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20221207-151813.yaml
rename to .changes/1.4.0/Under the Hood-20221207-151813.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 73a284e72..f1e182733 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -4,7 +4,24 @@
 - Changes are listed under the (pre)release in which they first appear. Subsequent releases include changes from previous releases.
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
+## dbt-bigquery 1.4.0-b1 - December 15, 2022
+### Features
+- Support for ingestion time partition table on BigQuery as incremental materialization ([#75](https://github.com/dbt-labs/dbt-bigquery/issues/75), [#136](https://github.com/dbt-labs/dbt-bigquery/pull/136))
+- Migrate dbt-utils current_timestamp macros into core + adapters ([#324](https://github.com/dbt-labs/dbt-bigquery/issues/324), [#323](https://github.com/dbt-labs/dbt-bigquery/pull/323))
+- Optimize insert_overwrite incremental strategy with WRITE_TRUNCATE / Partition copy ([#77](https://github.com/dbt-labs/dbt-bigquery/issues/77), [#167](https://github.com/dbt-labs/dbt-bigquery/pull/167))
+### Fixes
+- use execution_project in python models ([#355](https://github.com/dbt-labs/dbt-bigquery/issues/355), [#356](https://github.com/dbt-labs/dbt-bigquery/pull/356))
+- fix args for console link logging ([#362](https://github.com/dbt-labs/dbt-bigquery/issues/362), [#363](https://github.com/dbt-labs/dbt-bigquery/pull/363))
+### Under the Hood
+- event logging changes to be in parity with dbt-core post pr#6291 ([#6139](https://github.com/dbt-labs/dbt-bigquery/issues/6139), [#410](https://github.com/dbt-labs/dbt-bigquery/pull/410))
+- fix issue with tox 4.0.0 release which changes passenv syntax for space-separated variables ([#411](https://github.com/dbt-labs/dbt-bigquery/issues/411), [#411](https://github.com/dbt-labs/dbt-bigquery/pull/411))
+### Dependencies
+- Bump google-cloud-bigquery dependency to < 3.3.3 ([#269](https://github.com/dbt-labs/dbt-bigquery/issues/269), [#332](https://github.com/dbt-labs/dbt-bigquery/pull/332))
 
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#136](https://github.com/dbt-labs/dbt-bigquery/pull/136), [#167](https://github.com/dbt-labs/dbt-bigquery/pull/167), [#363](https://github.com/dbt-labs/dbt-bigquery/pull/363))
+- [@eplus-aolalere](https://github.com/eplus-aolalere) ([#332](https://github.com/dbt-labs/dbt-bigquery/pull/332))
+- [@ericapetersson](https://github.com/ericapetersson) ([#356](https://github.com/dbt-labs/dbt-bigquery/pull/356))
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 - [1.3](https://github.com/dbt-labs/dbt-bigquery/blob/1.3.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 70ba273f5..27cfeecd9 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.4.0a1"
+version = "1.4.0b1"
diff --git a/setup.py b/setup.py
index b24bbe16f..766aace2d 100644
--- a/setup.py
+++ b/setup.py
@@ -50,7 +50,7 @@ def _get_dbt_core_version():
 
 
 package_name = "dbt-bigquery"
-package_version = "1.4.0a1"
+package_version = "1.4.0b1"
 dbt_core_version = _get_dbt_core_version()
 description = """The BigQuery adapter plugin for dbt"""
 

From 3ad7a42acb13259edfb5b10b802fdd2f5406f84d Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Thu, 15 Dec 2022 16:06:13 -0600
Subject: [PATCH 518/860] add pypi release to workflow (#436)

---
 .github/workflows/release.yml | 31 ++++++++++++++++---------------
 1 file changed, 16 insertions(+), 15 deletions(-)

diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml
index 2eea6e8cc..91c3114e4 100644
--- a/.github/workflows/release.yml
+++ b/.github/workflows/release.yml
@@ -190,22 +190,23 @@ jobs:
       changelog_path: ${{ inputs.changelog_path }}
       test_run: ${{ inputs.test_run }}
 
-# Skipping this for now until we've proven build work in the repos
-  # pypi-release:
-  #   name: Pypi release
+  pypi-release:
+    name: Pypi release
+    # only release to PyPi if we're not testing - will release to PyPi test when workflow gets rewritten
+    if: inputs.test_run  == 'false'
 
-  #   runs-on: ubuntu-latest
+    runs-on: ubuntu-latest
 
-  #   needs: github-release
+    needs: github-release
 
-  #   environment: PypiProd
-  #   steps:
-  #     - uses: actions/download-artifact@v2
-  #       with:
-  #         name: dist
-  #         path: 'dist'
+    environment: PypiProd
+    steps:
+      - uses: actions/download-artifact@v2
+        with:
+          name: dist
+          path: 'dist'
 
-  #     - name: Publish distribution to PyPI
-  #       uses: pypa/gh-action-pypi-publish@v1.4.2
-  #       with:
-  #         password: ${{ secrets.PYPI_API_TOKEN }}
+      - name: Publish distribution to PyPI
+        uses: pypa/gh-action-pypi-publish@v1.4.2
+        with:
+          password: ${{ secrets.PYPI_API_TOKEN }}

From 537b56d10fcd897660ad21b5dcd35575c04bbaa4 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Fri, 16 Dec 2022 11:22:40 -0600
Subject: [PATCH 519/860] dbt-bigquery python 3.11 support  (#431)

* initial pr for bigquery support for 3.11py

* update pr number

* add michelleark to core-team ignore list for changie

* trying to pin some requirements to fix install

* repin tox

* loosening pins

* loosening constraints in setup.py

* playing with pins in setup.py

* remove additions to team ignore

* loosening google-cloud-cor

* playing with setup.py ranges

* loosening dataproc ranges

* changing soft pin range for setup.py

* reverting setup.py requred installs

* remove duplicated protobuf

* trying to loosen google-cloud-core

* trying to loosen google-cloud-core

* unloosening google-cloud-core

* typo fix

* google-cloud-core range setting to 2.3.0 and up

* loosening google-cloud-bigquery

* loosening dataproc require installs in setup.py

* loosen all google required installs

* playing with ranges for requirements

* trying removing package versions to allow pip to attempt to solve dependency conflict

* trying to tighten requirement versions some

* loosen wheel install

* loosening refs in requirements

* removing caps in setup.py

* minor change for wheel

* trying to unbind installs

* lossening setup.py to make sure dev-requirements works as is

* typo

* changing range for common-protos

* changing range for common-protos

* setting lower bound of versions in setup.py

* double checking versioning

* testing changes for google-cloud-core

* testing changes for google-cloud-core

* change to google-api-core versioning

* changing versioning for google-cloud-core

* change to protobuf

* changing tox version

* changing versioning for google-cloud-core

* versioning

* dataproc changers

* uncapping dataproc in both settings and dev-requirements

* pinning to minor version

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .../Dependencies-20221213-135331.yaml         |  7 ++
 .github/scripts/integration-test-matrix.js    |  2 +-
 .github/workflows/main.yml                    |  4 +-
 dev-requirements.txt                          | 40 ++++-----
 setup.py                                      | 88 ++++++++++---------
 tox.ini                                       |  6 +-
 6 files changed, 79 insertions(+), 68 deletions(-)
 create mode 100644 .changes/unreleased/Dependencies-20221213-135331.yaml

diff --git a/.changes/unreleased/Dependencies-20221213-135331.yaml b/.changes/unreleased/Dependencies-20221213-135331.yaml
new file mode 100644
index 000000000..b26c5da1a
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20221213-135331.yaml
@@ -0,0 +1,7 @@
+kind: Dependencies
+body: Add supoort for python 3.11
+time: 2022-12-13T13:53:31.182619-06:00
+custom:
+  Author: McKnight-42
+  Issue: "407"
+  PR: "431"
diff --git a/.github/scripts/integration-test-matrix.js b/.github/scripts/integration-test-matrix.js
index 1e3bb0f0d..58acf364d 100644
--- a/.github/scripts/integration-test-matrix.js
+++ b/.github/scripts/integration-test-matrix.js
@@ -1,6 +1,6 @@
 module.exports = ({ context }) => {
   const defaultPythonVersion = "3.8";
-  const supportedPythonVersions = ["3.7", "3.8", "3.9", "3.10"];
+  const supportedPythonVersions = ["3.7", "3.8", "3.9", "3.10", "3.11"];
   const supportedAdapters = ["bigquery"];
 
   // if PR, generate matrix based on files changed and PR labels
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index bcd9d2677..0c7d51fa8 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -71,7 +71,7 @@ jobs:
     strategy:
       fail-fast: false
       matrix:
-        python-version: ['3.7', '3.8', '3.9', '3.10']
+        python-version: ['3.7', '3.8', '3.9', '3.10', '3.11']
 
     env:
       TOXENV: "unit"
@@ -175,7 +175,7 @@ jobs:
       fail-fast: false
       matrix:
         os: [ubuntu-latest, macos-latest, windows-latest]
-        python-version: ['3.7', '3.8', '3.9', '3.10']
+        python-version: ['3.7', '3.8', '3.9', '3.10', '3.11']
 
     steps:
       - name: Set up Python ${{ matrix.python-version }}
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 676703d3e..de146ae27 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -3,26 +3,22 @@
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
 
-black==22.8.0
-bumpversion
+black~=22.8.0
+bumpversion~=0.6.0
 flake8
-flaky
-freezegun==1.1.0
-ipdb
-mypy==0.971
-pip-tools
-pre-commit
-pytest
-pytest-dotenv
-pytest-logbook
-pytest-csv
-pytest-xdist
-pytz
-tox>=3.13
-types-requests
-twine
-wheel
-
-# For dataproc running
-google-cloud-storage[dataproc]>=2.4.0
-google-cloud-dataproc[dataproc]>=4.0.3
+flaky~=3.7.0
+freezegun~=1.1.0
+ipdb~=0.13.9
+mypy~=0.971.0
+pip-tools~=6.11.0
+pre-commit~=2.20.0
+pytest~=7.2.0
+pytest-dotenv~=0.5.2
+pytest-logbook~=1.2.0
+pytest-csv~=3.0.0
+pytest-xdist~=3.1.0
+pytz~=2022.6.0
+tox~=3.13
+types-requests~=2.28.11
+twine~=4.0.2
+wheel~=0.37.1
diff --git a/setup.py b/setup.py
index 766aace2d..07f501ce6 100644
--- a/setup.py
+++ b/setup.py
@@ -1,7 +1,5 @@
 #!/usr/bin/env python
-import os
 import sys
-import re
 
 # require python 3.7 or newer
 if sys.version_info < (3, 7):
@@ -9,56 +7,65 @@
     print("Please upgrade to Python 3.7 or higher.")
     sys.exit(1)
 
-
-# require version of setuptools that supports find_namespace_packages
-from setuptools import setup
-
 try:
     from setuptools import find_namespace_packages
 except ImportError:
     # the user has a downlevel version of setuptools.
     print("Error: dbt requires setuptools v40.1.0 or higher.")
-    print('Please upgrade setuptools with "pip install --upgrade setuptools" ' "and try again")
+    print('Please upgrade setuptools with "pip install --upgrade setuptools" and try again')
     sys.exit(1)
 
+from pathlib import Path
+from setuptools import setup
+
+
+# pull the long description from the README
+README = Path(__file__).parent / "README.md"
+
+# used for this adapter's version and in determining the compatible dbt-core version
+VERSION = Path(__file__).parent / "dbt/adapters/bigquery/__version__.py"
+
 
-# pull long description from README
-this_directory = os.path.abspath(os.path.dirname(__file__))
-with open(os.path.join(this_directory, "README.md")) as f:
-    long_description = f.read()
+def _dbt_bigquery_version() -> str:
+    """
+    Pull the package version from the main package version file
+    """
+    attributes = {}
+    exec(VERSION.read_text(), attributes)
+    return attributes["version"]
 
 
-# get this package's version from dbt/adapters/<name>/__version__.py
-def _get_plugin_version_dict():
-    _version_path = os.path.join(this_directory, "dbt", "adapters", "bigquery", "__version__.py")
-    _semver = r"""(?P<major>\d+)\.(?P<minor>\d+)\.(?P<patch>\d+)"""
-    _pre = r"""((?P<prekind>a|b|rc)(?P<pre>\d+))?"""
-    _version_pattern = rf"""version\s*=\s*["']{_semver}{_pre}["']"""
-    with open(_version_path) as f:
-        match = re.search(_version_pattern, f.read().strip())
-        if match is None:
-            raise ValueError(f"invalid version at {_version_path}")
-        return match.groupdict()
+# require a compatible minor version (~=) and prerelease if this is a prerelease
+def _dbt_core_version(plugin_version: str) -> str:
+    """
+    Determine the compatible version of dbt-core using this package's version
+    """
+    try:
+        major, minor, plugin_patch = plugin_version.split(".")
+    except ValueError:
+        raise ValueError(f"Invalid version: {plugin_version}")
 
+    pre_release_phase = "".join([i for i in plugin_patch if not i.isdigit()])
+    if pre_release_phase:
+        if pre_release_phase not in ["a", "b", "rc"]:
+            raise ValueError(f"Invalid version: {plugin_version}")
+        core_patch = f"0{pre_release_phase}1"
+    else:
+        core_patch = "0"
 
-# require a compatible minor version (~=), prerelease if this is a prerelease
-def _get_dbt_core_version():
-    parts = _get_plugin_version_dict()
-    minor = "{major}.{minor}.0".format(**parts)
-    pre = parts["prekind"] + "1" if parts["prekind"] else ""
-    return f"{minor}{pre}"
+    return f"{major}.{minor}.{core_patch}"
 
 
 package_name = "dbt-bigquery"
 package_version = "1.4.0b1"
-dbt_core_version = _get_dbt_core_version()
+dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 
 setup(
-    name=package_name,
-    version=package_version,
-    description=description,
-    long_description=long_description,
+    name="dbt-bigquery",
+    version=_dbt_bigquery_version(),
+    description="The Bigquery adapter plugin for dbt",
+    long_description=README.read_text(),
     long_description_content_type="text/markdown",
     author="dbt Labs",
     author_email="info@dbtlabs.com",
@@ -66,14 +73,14 @@ def _get_dbt_core_version():
     packages=find_namespace_packages(include=["dbt", "dbt.*"]),
     include_package_data=True,
     install_requires=[
-        "dbt-core~={}".format(dbt_core_version),
+        f"dbt-core~={_dbt_core_version(_dbt_bigquery_version())}",
         "protobuf>=3.13.0,<4",
-        "google-cloud-core>=1.3.0,<3",
-        "google-cloud-bigquery>=1.25.0,<3.3.3",
-        "google-api-core>=1.16.0,<3",
-        "googleapis-common-protos>=1.6.0,<2",
-        "google-cloud-storage>=2.4.0",
-        "google-cloud-dataproc>=4.0.3",
+        "google-cloud-core~=2.0",
+        "google-cloud-bigquery~=3.0",
+        "google-api-core~=2.0",
+        "googleapis-common-protos~=1.6",
+        "google-cloud-storage~=2.4",
+        "google-cloud-dataproc~=5.0",
     ],
     zip_safe=False,
     classifiers=[
@@ -86,6 +93,7 @@ def _get_dbt_core_version():
         "Programming Language :: Python :: 3.8",
         "Programming Language :: Python :: 3.9",
         "Programming Language :: Python :: 3.10",
+        "Programming Language :: Python :: 3.11",
     ],
     python_requires=">=3.7",
 )
diff --git a/tox.ini b/tox.ini
index de228d534..a04781985 100644
--- a/tox.ini
+++ b/tox.ini
@@ -1,8 +1,8 @@
 [tox]
 skipsdist = True
-envlist = py37,py38,py39,py310
+envlist = py37,py38,py39,py310,py311
 
-[testenv:{unit,py37,py38,py39,py310,py}]
+[testenv:{unit,py37,py38,py39,py310,py311,py}]
 description = unit testing
 skip_install = true
 passenv =
@@ -13,7 +13,7 @@ deps =
   -rdev-requirements.txt
   -e.
 
-[testenv:{integration,py37,py38,py39,py310,py}-{bigquery}]
+[testenv:{integration,py37,py38,py39,py310,py311,py}-{bigquery}]
 description = adapter plugin integration testing
 skip_install = true
 passenv =

From aa31c8b4f427e1f48a514fdaf07a8a098161a1f3 Mon Sep 17 00:00:00 2001
From: Darryl Ng <darryl.nmj@gmail.com>
Date: Thu, 22 Dec 2022 02:55:16 +1100
Subject: [PATCH 520/860] Support BigQuery 'labels' and 'hours_to_expiration'
 configs for seeds (#133)

* [CT-264] [CT-262] [Feature] Support BigQuery labels for seeded models

* Also test other seed config that may be added in this PR

* Add change log

* Revert "Add change log"

This reverts commit 78abb9fde7626d1e4f8a51b4a7514632eea52cca.

* Revert "Also test other seed config that may be added in this PR"

This reverts commit 42636221506118ab1013f7202a621263b655816e.

* Revert "[CT-264] [CT-262] [Feature] Support BigQuery labels for seeded models"

This reverts commit b5d0f2feaeb12bd5c5aeceb12b50015afdeb0f9c.

* [CT-264] [CT-262] [Feature] Support BigQuery labels and hours_to_expiration configs for seeded models

* Add changelog

* Add changes as created by changie

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .../unreleased/Features-20221119-115200.yaml  |  7 +++
 .../bigquery/macros/materializations/seed.sql |  5 ++
 .../simple_seed_test/models-bq/schema.yml     |  7 +++
 .../seeds-config/seed_configs.csv             |  3 ++
 ..._type_override.py => test_seed_configs.py} | 54 ++++++++++++++-----
 5 files changed, 64 insertions(+), 12 deletions(-)
 create mode 100644 .changes/unreleased/Features-20221119-115200.yaml
 create mode 100644 tests/integration/simple_seed_test/seeds-config/seed_configs.csv
 rename tests/integration/simple_seed_test/{test_seed_type_override.py => test_seed_configs.py} (56%)

diff --git a/.changes/unreleased/Features-20221119-115200.yaml b/.changes/unreleased/Features-20221119-115200.yaml
new file mode 100644
index 000000000..04189c19c
--- /dev/null
+++ b/.changes/unreleased/Features-20221119-115200.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: Support BigQuery 'labels' and 'hours_to_expiration' configs for seeds.
+time: 2022-11-19T11:52:00.80272+11:00
+custom:
+  Author: darrylng
+  Issue: "125"
+  PR: "133"
diff --git a/dbt/include/bigquery/macros/materializations/seed.sql b/dbt/include/bigquery/macros/materializations/seed.sql
index d95cc4e1b..6ac7337f3 100644
--- a/dbt/include/bigquery/macros/materializations/seed.sql
+++ b/dbt/include/bigquery/macros/materializations/seed.sql
@@ -12,6 +12,11 @@
   {%- set column_override = model['config'].get('column_types', {}) -%}
   {{ adapter.load_dataframe(model['database'], model['schema'], model['alias'],
   							agate_table, column_override) }}
+
+  {% call statement() %}
+    alter table {{ this.render() }} set {{ bigquery_table_options(config, model) }}
+  {% endcall %}
+
   {% if config.persist_relation_docs() and 'description' in model %}
 
   	{{ adapter.update_table_description(model['database'], model['schema'], model['alias'], model['description']) }}
diff --git a/tests/integration/simple_seed_test/models-bq/schema.yml b/tests/integration/simple_seed_test/models-bq/schema.yml
index 019a9524f..606794ce5 100644
--- a/tests/integration/simple_seed_test/models-bq/schema.yml
+++ b/tests/integration/simple_seed_test/models-bq/schema.yml
@@ -45,3 +45,10 @@ seeds:
     tests:
     - column_type:
         type: STRING
+
+- name: seed_configs
+  config:
+    hours_to_expiration: 2
+    labels:
+      contains_pii: 'yes'
+      contains_pie: 'no'
diff --git a/tests/integration/simple_seed_test/seeds-config/seed_configs.csv b/tests/integration/simple_seed_test/seeds-config/seed_configs.csv
new file mode 100644
index 000000000..a0d796378
--- /dev/null
+++ b/tests/integration/simple_seed_test/seeds-config/seed_configs.csv
@@ -0,0 +1,3 @@
+id,stuff
+1,a
+2,b
diff --git a/tests/integration/simple_seed_test/test_seed_type_override.py b/tests/integration/simple_seed_test/test_seed_configs.py
similarity index 56%
rename from tests/integration/simple_seed_test/test_seed_type_override.py
rename to tests/integration/simple_seed_test/test_seed_configs.py
index 4f97880d8..4dc9da9e1 100644
--- a/tests/integration/simple_seed_test/test_seed_type_override.py
+++ b/tests/integration/simple_seed_test/test_seed_configs.py
@@ -1,12 +1,16 @@
 from tests.integration.base import DBTIntegrationTest, use_profile
 
-
-class TestSimpleSeedColumnOverride(DBTIntegrationTest):
+class TestSimpleSeedConfigs(DBTIntegrationTest):
+    run_once = False
 
     @property
     def schema(self):
         return "simple_seed"
 
+    @property
+    def models(self):
+        return "models-bq"
+
     @property
     def project_config(self):
         return {
@@ -19,21 +23,22 @@ def project_config(self):
                     'quote_columns': True,
                     'seed_enabled': {
                         'enabled': True,
-                        '+column_types': self.seed_enabled_types()
+                        '+column_types': self.seed_enabled_types(),
                     },
                     'seed_tricky': {
                         'enabled': True,
                         '+column_types': self.seed_tricky_types(),
                     },
+                    'seed_configs': {
+                        'enabled': True,
+                    },
                 },
             },
         }
 
-
-class TestSimpleSeedColumnOverrideBQ(TestSimpleSeedColumnOverride):
     @property
-    def models(self):
-        return "models-bq"
+    def profile_config(self):
+        return self.bigquery_profile()
 
     def seed_enabled_types(self):
         return {
@@ -47,14 +52,39 @@ def seed_tricky_types(self):
             'looks_like_a_bool': 'STRING',
             'looks_like_a_date': 'STRING',
         }
+    
+    def seed_data(self):
+        if self.run_once:
+            return
+        results = self.run_dbt(["seed", "--show"])
+        self.assertEqual(len(results),  3)
+        self.run_once = True
 
     @property
-    def profile_config(self):
-        return self.bigquery_profile()
+    def table_labels(self):
+        return {
+            'contains_pii': 'yes',
+            'contains_pie': 'no'
+        }
 
     @use_profile('bigquery')
-    def test_bigquery_simple_seed_with_column_override_bigquery(self):
-        results = self.run_dbt(["seed", "--show"])
-        self.assertEqual(len(results),  2)
+    def test__bigquery_simple_seed_with_column_override_bigquery(self):
+        self.seed_data()
         results = self.run_dbt(["test"])
         self.assertEqual(len(results),  10)
+
+    @use_profile('bigquery')
+    def test__bigquery_seed_table_with_labels_config_bigquery(self):
+        self.seed_data()
+        with self.get_connection() as conn:
+            client = conn.handle
+
+            table = client.get_table(
+                self.adapter.connections.get_bq_table(
+                    self.default_database, self.unique_schema(), 'seed_configs')
+            )
+
+            self.assertTrue(table.labels)
+            self.assertEquals(table.labels, self.table_labels)
+            self.assertTrue(table.expires)
+

From a19f8e1a8ffa5edf1ca1a9255641c68ea2c10f95 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Thu, 22 Dec 2022 13:09:38 -0500
Subject: [PATCH 521/860] [CT-1691] Bug - Partition by validation failing
 (#445)

* exceptions are called correctly now

* replaced CompilationException with ValidationException where appropriate, also some whitespace pep-8 fixes

* added change log
---
 .../Under the Hood-20221221-124711.yaml       |  8 ++++++
 dbt/adapters/bigquery/impl.py                 | 27 +++++++++----------
 tests/unit/test_bigquery_adapter.py           | 20 ++++++--------
 3 files changed, 28 insertions(+), 27 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20221221-124711.yaml

diff --git a/.changes/unreleased/Under the Hood-20221221-124711.yaml b/.changes/unreleased/Under the Hood-20221221-124711.yaml
new file mode 100644
index 000000000..fcb83f9eb
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20221221-124711.yaml	
@@ -0,0 +1,8 @@
+kind: Under the Hood
+body: Partition By unit test was failing, referenced the incorrect exception to be
+  raised
+time: 2022-12-21T12:47:11.619388-05:00
+custom:
+  Author: mikealfare
+  Issue: "443"
+  PR: "445"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index b59aff6c3..ec749c316 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -55,8 +55,7 @@
 
 def sql_escape(string):
     if not isinstance(string, str):
-        dbt.exceptions.raise_compiler_exception(f"cannot escape a non-string: {string}")
-
+        raise dbt.exceptions.CompilationException(f"cannot escape a non-string: {string}")
     return json.dumps(string)[1:-1]
 
 
@@ -92,17 +91,16 @@ def render_wrapped(self, alias: Optional[str] = None):
             return self.render(alias)
 
     @classmethod
-    def parse(cls, raw_partition_by) -> Optional["PartitionConfig"]:  # type: ignore [return]
+    def parse(cls, raw_partition_by) -> Optional["PartitionConfig"]:
         if raw_partition_by is None:
             return None
         try:
             cls.validate(raw_partition_by)
             return cls.from_dict(raw_partition_by)
         except ValidationError as exc:
-            msg = dbt.exceptions.validator_error_message(exc)
-            dbt.exceptions.raise_compiler_error(f"Could not parse partition config: {msg}")
+            raise dbt.exceptions.ValidationException("Could not parse partition config") from exc
         except TypeError:
-            dbt.exceptions.raise_compiler_error(
+            raise dbt.exceptions.CompilationException(
                 f"Invalid partition_by config:\n"
                 f"  Got: {raw_partition_by}\n"
                 f'  Expected a dictionary with "field" and "data_type" keys'
@@ -494,7 +492,7 @@ def _bq_table_to_relation(self, bq_table):
         )
 
     @classmethod
-    def warning_on_hooks(hook_type):
+    def warning_on_hooks(cls, hook_type):
         msg = "{} is not supported in bigquery and will be ignored"
         warn_msg = dbt.ui.color(msg, ui.COLOR_FG_YELLOW)
         logger.info(warn_msg)
@@ -512,7 +510,8 @@ def add_query(self, sql, auto_begin=True, bindings=None, abridge_sql_log=False):
     # Special bigquery adapter methods
     ###
 
-    def _partitions_match(self, table, conf_partition: Optional[PartitionConfig]) -> bool:
+    @staticmethod
+    def _partitions_match(table, conf_partition: Optional[PartitionConfig]) -> bool:
         """
         Check if the actual and configured partitions for a table are a match.
         BigQuery tables can be replaced if:
@@ -551,7 +550,8 @@ def _partitions_match(self, table, conf_partition: Optional[PartitionConfig]) ->
         else:
             return False
 
-    def _clusters_match(self, table, conf_cluster) -> bool:
+    @staticmethod
+    def _clusters_match(table, conf_cluster) -> bool:
         """
         Check if the actual and configured clustering columns for a table
         are a match. BigQuery tables can be replaced if clustering columns
@@ -754,9 +754,7 @@ def get_common_options(
         opts = {}
 
         if (config.get("hours_to_expiration") is not None) and (not temporary):
-            expiration = ("TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL " "{} hour)").format(
-                config.get("hours_to_expiration")
-            )
+            expiration = f'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL {config.get("hours_to_expiration")} hour)'
             opts["expiration_timestamp"] = expiration
 
         if config.persist_relation_docs() and "description" in node:  # type: ignore[attr-defined]
@@ -776,11 +774,10 @@ def get_table_options(
         opts = self.get_common_options(config, node, temporary)
 
         if config.get("kms_key_name") is not None:
-            opts["kms_key_name"] = "'{}'".format(config.get("kms_key_name"))
+            opts["kms_key_name"] = f"'{config.get('kms_key_name')}'"
 
         if temporary:
-            expiration = "TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)"
-            opts["expiration_timestamp"] = expiration
+            opts["expiration_timestamp"] = "TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)"
         else:
             # It doesn't apply the `require_partition_filter` option for a temporary table
             # so that we avoid the error by not specifying a partition with a temporary table
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 83a2eb3a6..cd53f59b3 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -674,10 +674,10 @@ def test_copy_table_materialization_incremental(self):
     def test_parse_partition_by(self):
         adapter = self.get_adapter('oauth')
 
-        with self.assertRaises(dbt.exceptions.CompilationException):
+        with self.assertRaises(dbt.exceptions.ValidationException):
             adapter.parse_partition_by("date(ts)")
 
-        with self.assertRaises(dbt.exceptions.CompilationException):
+        with self.assertRaises(dbt.exceptions.ValidationException):
             adapter.parse_partition_by("ts")
 
         self.assertEqual(
@@ -756,8 +756,7 @@ def test_parse_partition_by(self):
                 "data_type": "timestamp",
                 "granularity": "MONTH"
 
-            }).to_dict(omit_none=True
-                ), {
+            }).to_dict(omit_none=True), {
                 "field": "ts",
                 "data_type": "timestamp",
                 "granularity": "MONTH",
@@ -842,7 +841,7 @@ def test_parse_partition_by(self):
         )
 
         # Invalid, should raise an error
-        with self.assertRaises(dbt.exceptions.CompilationException):
+        with self.assertRaises(dbt.exceptions.ValidationException):
             adapter.parse_partition_by({})
 
         # passthrough
@@ -883,12 +882,11 @@ def test_hours_to_expiration(self):
         actual = adapter.get_table_options(mock_config, node={}, temporary=False)
         self.assertEqual(expected, actual)
 
-
     def test_hours_to_expiration_temporary(self):
         adapter = self.get_adapter('oauth')
         mock_config = create_autospec(
             RuntimeConfigObject)
-        config={'hours_to_expiration': 4}
+        config = {'hours_to_expiration': 4}
         mock_config.get.side_effect = lambda name: config.get(name)
 
         expected = {
@@ -902,7 +900,7 @@ def test_table_kms_key_name(self):
         adapter = self.get_adapter('oauth')
         mock_config = create_autospec(
             RuntimeConfigObject)
-        config={'kms_key_name': 'some_key'}
+        config = {'kms_key_name': 'some_key'}
         mock_config.get.side_effect = lambda name: config.get(name)
 
         expected = {
@@ -911,12 +909,11 @@ def test_table_kms_key_name(self):
         actual = adapter.get_table_options(mock_config, node={}, temporary=False)
         self.assertEqual(expected, actual)
 
-
     def test_view_kms_key_name(self):
         adapter = self.get_adapter('oauth')
         mock_config = create_autospec(
             RuntimeConfigObject)
-        config={'kms_key_name': 'some_key'}
+        config = {'kms_key_name': 'some_key'}
         mock_config.get.side_effect = lambda name: config.get(name)
 
         expected = {}
@@ -924,7 +921,6 @@ def test_view_kms_key_name(self):
         self.assertEqual(expected, actual)
 
 
-
 class TestBigQueryFilterCatalog(unittest.TestCase):
     def test__catalog_filter_table(self):
         manifest = MagicMock()
@@ -1047,4 +1043,4 @@ def test_sanitize_label_length(label_length):
         )
     with pytest.raises(dbt.exceptions.RuntimeException) as error_info:
         _sanitize_label(random_string)
-    assert error_info.value.args[0] == test_error_msg
\ No newline at end of file
+    assert error_info.value.args[0] == test_error_msg

From da0d0d7e21c0bec09a9fd35baa264f915eddc55a Mon Sep 17 00:00:00 2001
From: Ernesto Ongaro <ernestoongaro@gmail.com>
Date: Tue, 3 Jan 2023 17:55:34 +0000
Subject: [PATCH 522/860]  truncate sanitized query labels to 63 characters 
 (#442)

* fixes https://github.com/dbt-labs/dbt-bigquery/issues/202
* add changie log
* remove unneeded import
* no need to check length of value before slicing, thx @mikealfare
---
 .../unreleased/Under the Hood-20221219-212421.yaml    |  7 +++++++
 dbt/adapters/bigquery/connections.py                  | 11 +----------
 tests/unit/test_bigquery_adapter.py                   |  9 +--------
 3 files changed, 9 insertions(+), 18 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20221219-212421.yaml

diff --git a/.changes/unreleased/Under the Hood-20221219-212421.yaml b/.changes/unreleased/Under the Hood-20221219-212421.yaml
new file mode 100644
index 000000000..c381a18c5
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20221219-212421.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: Truncates Job Labels to 63 Characters
+time: 2022-12-19T21:24:21.304277Z
+custom:
+  Author: ernestoongaro
+  Issue: "202"
+  PR: "442"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index e4643f660..c45355665 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -710,13 +710,4 @@ def _sanitize_label(value: str) -> str:
     """Return a legal value for a BigQuery label."""
     value = value.strip().lower()
     value = _SANITIZE_LABEL_PATTERN.sub("_", value)
-    value_length = len(value)
-    if value_length > _VALIDATE_LABEL_LENGTH_LIMIT:
-        error_msg = (
-            f"Job label length {value_length} is greater than length limit: "
-            f"{_VALIDATE_LABEL_LENGTH_LIMIT}\n"
-            f"Current sanitized label: {value}"
-        )
-        raise RuntimeException(error_msg)
-    else:
-        return value
+    return value[:_VALIDATE_LABEL_LENGTH_LIMIT]
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index cd53f59b3..cf8bcdfd8 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -1036,11 +1036,4 @@ def test_sanitize_label_length(label_length):
         random.choice(string.ascii_uppercase + string.digits)
         for i in range(label_length)
     )
-    test_error_msg = (
-            f"Job label length {label_length} is greater than length limit: "
-            f"{_VALIDATE_LABEL_LENGTH_LIMIT}\n"
-            f"Current sanitized label: {random_string.lower()}"
-        )
-    with pytest.raises(dbt.exceptions.RuntimeException) as error_info:
-        _sanitize_label(random_string)
-    assert error_info.value.args[0] == test_error_msg
+    assert len(_sanitize_label(random_string)) <= _VALIDATE_LABEL_LENGTH_LIMIT
\ No newline at end of file

From 778dc33868f42bdd8cdc81c5110f2ec3629f8ccc Mon Sep 17 00:00:00 2001
From: Nathaniel May <nathanieldmay@gmail.com>
Date: Thu, 5 Jan 2023 18:19:33 -0500
Subject: [PATCH 523/860] retry transient connection errors (#230)

* retry transient connection errors

* changelog

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .changes/unreleased/Under the Hood-20220805-165120.yaml | 7 +++++++
 dbt/adapters/bigquery/connections.py                    | 1 +
 2 files changed, 8 insertions(+)
 create mode 100644 .changes/unreleased/Under the Hood-20220805-165120.yaml

diff --git a/.changes/unreleased/Under the Hood-20220805-165120.yaml b/.changes/unreleased/Under the Hood-20220805-165120.yaml
new file mode 100644
index 000000000..dacf18dfe
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20220805-165120.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: use gcp retry logic for new connections
+time: 2022-08-05T16:51:20.590446-04:00
+custom:
+  Author: nathaniel-may
+  Issue: "229"
+  PR: "230"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index c45355665..7a09eb0af 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -325,6 +325,7 @@ def get_credentials(cls, profile_credentials):
             return cls.get_google_credentials(profile_credentials)
 
     @classmethod
+    @retry.Retry()  # google decorator. retries on transient errors with exponential backoff
     def get_bigquery_client(cls, profile_credentials):
         creds = cls.get_credentials(profile_credentials)
         execution_project = profile_credentials.execution_project

From 6841da72154202c9f759e7b3e2c424a51d7ae608 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Mon, 9 Jan 2023 15:49:52 -0800
Subject: [PATCH 524/860] fix whitespace issue with incremental sql (#458)

* fix whitespace issue with incremental sql

* add changie

* add setuptools install

* add twine and check-wheel-contents

* try installing pkgconfig-lite

* add --allow-empty-checksums

* check pkg-config installs

* install mac os pyicu dependencies

* fix syntax issue

* force agate 1.6.3

* add trailing comma
---
 .../unreleased/Fixes-20230109-105325.yaml     |  7 +++
 .github/workflows/main.yml                    |  3 +-
 .../incremental_strategy/merge.sql            |  2 +-
 setup.py                                      |  1 +
 .../test_incremental_materialization.py       | 44 +++++++++++++++++++
 5 files changed, 54 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230109-105325.yaml
 create mode 100644 tests/functional/test_incremental_materialization.py

diff --git a/.changes/unreleased/Fixes-20230109-105325.yaml b/.changes/unreleased/Fixes-20230109-105325.yaml
new file mode 100644
index 000000000..2a16883fc
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230109-105325.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: stop eliminating trailing whitespace in incremental merge sql
+time: 2023-01-09T10:53:25.837837-08:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "457"
+  PR: "458"
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 0c7d51fa8..140557beb 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -182,11 +182,10 @@ jobs:
         uses: actions/setup-python@v4.3.0
         with:
           python-version: ${{ matrix.python-version }}
-
       - name: Install python dependencies
         run: |
           python -m pip install --user --upgrade pip
-          python -m pip install --upgrade wheel
+          python -m pip install --upgrade wheel setuptools twine check-wheel-contents
           python -m pip --version
       - uses: actions/download-artifact@v2
         with:
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
index 5efb079b6..57c88dbc8 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
@@ -16,7 +16,7 @@
             {{ wrap_with_time_ingestion_partitioning_sql(build_partition_time_exp(partition_by), sql, True) }}
             {%- else -%}
             {{sql}}
-            {%- endif -%}
+            {%- endif %}
         )
         {%- endif -%}
     {%- endset -%}
diff --git a/setup.py b/setup.py
index 07f501ce6..52defb47d 100644
--- a/setup.py
+++ b/setup.py
@@ -81,6 +81,7 @@ def _dbt_core_version(plugin_version: str) -> str:
         "googleapis-common-protos~=1.6",
         "google-cloud-storage~=2.4",
         "google-cloud-dataproc~=5.0",
+        "agate>=1.6.3,<1.7",
     ],
     zip_safe=False,
     classifiers=[
diff --git a/tests/functional/test_incremental_materialization.py b/tests/functional/test_incremental_materialization.py
new file mode 100644
index 000000000..6932363b3
--- /dev/null
+++ b/tests/functional/test_incremental_materialization.py
@@ -0,0 +1,44 @@
+import pytest
+import os
+from dbt.tests.util import (
+    run_dbt
+)
+
+# This is a short term hack, we need to go back 
+# and make adapter implementations of:
+# https://github.com/dbt-labs/dbt-core/pull/6330
+
+_INCREMENTAL_MODEL = """
+{{
+    config(
+        materialized="incremental",
+    )
+}}
+
+{% if not is_incremental() %}
+
+    select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+    select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+
+{% else %}
+
+    select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+    select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+
+{% endif %}
+-- Test Comment To Prevent Reccurence of https://github.com/dbt-labs/dbt-core/issues/6485
+"""
+
+class BaseIncrementalModelConfig:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "test_incremental.sql": _INCREMENTAL_MODEL
+        }
+
+class TestIncrementalModel(BaseIncrementalModelConfig):
+    def test_incremental_model_succeeds(self, project):
+        results = run_dbt(["run"])
+        assert len(results) == 1
+        results = run_dbt(["run"])
+        assert len(results) == 1
\ No newline at end of file

From a5fcce91befc05949a3597e4bc84d526f6f1830e Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Tue, 10 Jan 2023 00:44:13 -0600
Subject: [PATCH 525/860] take over of PR #174 to get changelog in order to
 merge (#456)

* take over of PR #174 to get changelog in order to merge

* try and fix wheel

* trying to change order of operations for wheel install

* try manually installing PyICU

* reset to default

* test colin changes

* reset to default state

Co-authored-by: Halvor Lund <halvor.lund@gmail.com>
Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .changes/unreleased/Features-20230109-105921.yaml |  7 +++++++
 dbt/include/bigquery/macros/catalog.sql           | 10 ++++++++++
 2 files changed, 17 insertions(+)
 create mode 100644 .changes/unreleased/Features-20230109-105921.yaml

diff --git a/.changes/unreleased/Features-20230109-105921.yaml b/.changes/unreleased/Features-20230109-105921.yaml
new file mode 100644
index 000000000..0eaec0dcb
--- /dev/null
+++ b/.changes/unreleased/Features-20230109-105921.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: Fetch table description to `catalog.json`, so it can be shown in dbt docs
+time: 2023-01-09T10:59:21.213259-06:00
+custom:
+  Author: McKnight-42, halvorlu
+  Issue: "173"
+  PR: "174"
diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql
index 6822d88a6..d430b80f8 100644
--- a/dbt/include/bigquery/macros/catalog.sql
+++ b/dbt/include/bigquery/macros/catalog.sql
@@ -35,6 +35,14 @@
         )
     ),
 
+    table_options as (
+        select
+            concat(table_catalog, '.', table_schema, '.', table_name) as relation_id,
+            JSON_VALUE(option_value) as table_comment
+
+        from {{ information_schema.replace(information_schema_view='TABLE_OPTIONS') }}
+        where option_name = 'description'
+    ),
     extracted as (
 
         select *,
@@ -146,6 +154,7 @@
             else unsharded_tables.table_name
         end as table_name,
         unsharded_tables.table_type,
+        table_options.table_comment,
 
         -- coalesce name and type for External tables - these columns are not
         -- present in the COLUMN_FIELD_PATHS resultset
@@ -198,6 +207,7 @@
     -- sure that column metadata is picked up through the join. This will only
     -- return the column information for the "max" table in a date-sharded table set
     from unsharded_tables
+    left join table_options using (relation_id)
     left join columns using (relation_id)
     left join column_stats using (relation_id)
   {%- endset -%}

From 2e06977a118e07c5d7ad242b83bac913f9ee05d7 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Tue, 10 Jan 2023 12:50:42 -0800
Subject: [PATCH 526/860] fix release.yml pypi steps (#461)

---
 .github/workflows/release.yml | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)

diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml
index 91c3114e4..5fd0291e9 100644
--- a/.github/workflows/release.yml
+++ b/.github/workflows/release.yml
@@ -193,7 +193,7 @@ jobs:
   pypi-release:
     name: Pypi release
     # only release to PyPi if we're not testing - will release to PyPi test when workflow gets rewritten
-    if: inputs.test_run  == 'false'
+    if: ${{ inputs.test_run == false }}
 
     runs-on: ubuntu-latest
 
@@ -201,10 +201,11 @@ jobs:
 
     environment: PypiProd
     steps:
-      - uses: actions/download-artifact@v2
+      - name: "Download Build Artifact - ${{ inputs.version_number }}"
+        uses: actions/download-artifact@v3
         with:
-          name: dist
-          path: 'dist'
+          name: ${{ inputs.version_number }}
+          path: dist/
 
       - name: Publish distribution to PyPI
         uses: pypa/gh-action-pypi-publish@v1.4.2

From 3ce88d75ac78b9aac1dcaa184c94e235f661a6f0 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Tue, 10 Jan 2023 19:19:11 -0600
Subject: [PATCH 527/860] update exception names (#460)

* update exception names

* point back to main

* Update Under the Hood-20230110-110016.yaml
---
 .../Under the Hood-20230110-110016.yaml       |  7 ++++
 dbt/adapters/bigquery/connections.py          | 18 ++++-----
 dbt/adapters/bigquery/gcloud.py               |  2 +-
 dbt/adapters/bigquery/impl.py                 | 22 +++++------
 dbt/adapters/bigquery/relation.py             |  4 +-
 .../schema_tests_test/test_schema_v2_tests.py |  2 -
 .../test_simple_snapshot.py                   |  1 -
 .../integration/sources_test/test_sources.py  |  1 -
 tests/unit/test_bigquery_adapter.py           | 38 +++++++++----------
 9 files changed, 48 insertions(+), 47 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20230110-110016.yaml

diff --git a/.changes/unreleased/Under the Hood-20230110-110016.yaml b/.changes/unreleased/Under the Hood-20230110-110016.yaml
new file mode 100644
index 000000000..3f428a801
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20230110-110016.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: Update exception names to match dbt-core
+time: 2023-01-10T11:00:16.649793-06:00
+custom:
+  Author: emmyoop
+  Issue: "441"
+  PR: "460"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 7a09eb0af..715dfb36a 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -24,9 +24,9 @@
 from dbt.tracking import active_user
 from dbt.contracts.connection import ConnectionState, AdapterResponse
 from dbt.exceptions import (
-    FailedToConnectException,
-    RuntimeException,
-    DatabaseException,
+    FailedToConnectError,
+    DbtRuntimeError,
+    DbtDatabaseError,
     DbtProfileError,
 )
 from dbt.adapters.base import BaseConnectionManager, Credentials
@@ -196,7 +196,7 @@ def handle_error(cls, error, message):
                     error.query_job.location, error.query_job.project, error.query_job.job_id
                 )
             )
-        raise DatabaseException(error_msg)
+        raise DbtDatabaseError(error_msg)
 
     def clear_transaction(self):
         pass
@@ -223,12 +223,12 @@ def exception_handler(self, sql):
                 "account you are trying to impersonate.\n\n"
                 f"{str(e)}"
             )
-            raise RuntimeException(message)
+            raise DbtRuntimeError(message)
 
         except Exception as e:
             logger.debug("Unhandled error while running:\n{}".format(sql))
             logger.debug(e)
-            if isinstance(e, RuntimeException):
+            if isinstance(e, DbtRuntimeError):
                 # during a sql query, an internal to dbt exception was raised.
                 # this sounds a lot like a signal handler and probably has
                 # useful information, so raise it without modification.
@@ -238,7 +238,7 @@ def exception_handler(self, sql):
             # don't want to log. Hopefully they never change this!
             if BQ_QUERY_JOB_SPLIT in exc_message:
                 exc_message = exc_message.split(BQ_QUERY_JOB_SPLIT)[0].strip()
-            raise RuntimeException(exc_message)
+            raise DbtRuntimeError(exc_message)
 
     def cancel_open(self) -> None:
         pass
@@ -305,7 +305,7 @@ def get_google_credentials(cls, profile_credentials) -> GoogleCredentials:
             )
 
         error = 'Invalid `method` in profile: "{}"'.format(method)
-        raise FailedToConnectException(error)
+        raise FailedToConnectError(error)
 
     @classmethod
     def get_impersonated_credentials(cls, profile_credentials):
@@ -362,7 +362,7 @@ def open(cls, connection):
             connection.handle = None
             connection.state = "fail"
 
-            raise FailedToConnectException(str(e))
+            raise FailedToConnectError(str(e))
 
         connection.handle = handle
         connection.state = "open"
diff --git a/dbt/adapters/bigquery/gcloud.py b/dbt/adapters/bigquery/gcloud.py
index eb418e93b..c303097bc 100644
--- a/dbt/adapters/bigquery/gcloud.py
+++ b/dbt/adapters/bigquery/gcloud.py
@@ -25,4 +25,4 @@ def setup_default_credentials():
     if gcloud_installed():
         run_cmd(".", ["gcloud", "auth", "application-default", "login"])
     else:
-        raise dbt.exceptions.RuntimeException(NOT_INSTALLED_MSG)
+        raise dbt.exceptions.DbtRuntimeError(NOT_INSTALLED_MSG)
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index ec749c316..920b992a0 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -55,7 +55,7 @@
 
 def sql_escape(string):
     if not isinstance(string, str):
-        raise dbt.exceptions.CompilationException(f"cannot escape a non-string: {string}")
+        raise dbt.exceptions.CompilationError(f"cannot escape a non-string: {string}")
     return json.dumps(string)[1:-1]
 
 
@@ -98,9 +98,9 @@ def parse(cls, raw_partition_by) -> Optional["PartitionConfig"]:
             cls.validate(raw_partition_by)
             return cls.from_dict(raw_partition_by)
         except ValidationError as exc:
-            raise dbt.exceptions.ValidationException("Could not parse partition config") from exc
+            raise dbt.exceptions.DbtValidationError("Could not parse partition config") from exc
         except TypeError:
-            raise dbt.exceptions.CompilationException(
+            raise dbt.exceptions.CompilationError(
                 f"Invalid partition_by config:\n"
                 f"  Got: {raw_partition_by}\n"
                 f'  Expected a dictionary with "field" and "data_type" keys'
@@ -173,9 +173,7 @@ def drop_relation(self, relation: BigQueryRelation) -> None:
         conn.handle.delete_table(table_ref)
 
     def truncate_relation(self, relation: BigQueryRelation) -> None:
-        raise dbt.exceptions.NotImplementedException(
-            "`truncate` is not implemented for this adapter!"
-        )
+        raise dbt.exceptions.NotImplementedError("`truncate` is not implemented for this adapter!")
 
     def rename_relation(
         self, from_relation: BigQueryRelation, to_relation: BigQueryRelation
@@ -191,7 +189,7 @@ def rename_relation(
             or from_relation.type == RelationType.View
             or to_relation.type == RelationType.View
         ):
-            raise dbt.exceptions.RuntimeException(
+            raise dbt.exceptions.DbtRuntimeError(
                 "Renaming of views is not currently supported in BigQuery"
             )
 
@@ -440,7 +438,7 @@ def copy_table(self, source, destination, materialization):
         elif materialization == "table":
             write_disposition = WRITE_TRUNCATE
         else:
-            dbt.exceptions.raise_compiler_error(
+            raise dbt.exceptions.CompilationError(
                 'Copy table materialization must be "copy" or "table", but '
                 f"config.get('copy_materialization', 'table') was "
                 f"{materialization}"
@@ -473,11 +471,11 @@ def poll_until_job_completes(cls, job, timeout):
             job.reload()
 
         if job.state != "DONE":
-            raise dbt.exceptions.RuntimeException("BigQuery Timeout Exceeded")
+            raise dbt.exceptions.DbtRuntimeError("BigQuery Timeout Exceeded")
 
         elif job.error_result:
             message = "\n".join(error["message"].strip() for error in job.errors)
-            raise dbt.exceptions.RuntimeException(message)
+            raise dbt.exceptions.DbtRuntimeError(message)
 
     def _bq_table_to_relation(self, bq_table):
         if bq_table is None:
@@ -502,7 +500,7 @@ def add_query(self, sql, auto_begin=True, bindings=None, abridge_sql_log=False):
         if self.nice_connection_name() in ["on-run-start", "on-run-end"]:
             self.warning_on_hooks(self.nice_connection_name())
         else:
-            raise dbt.exceptions.NotImplementedException(
+            raise dbt.exceptions.NotImplementedError(
                 "`add_query` is not implemented for this adapter!"
             )
 
@@ -860,7 +858,7 @@ def string_add_sql(
         elif location == "prepend":
             return f"concat('{value}', {add_to})"
         else:
-            raise dbt.exceptions.RuntimeException(
+            raise dbt.exceptions.DbtRuntimeError(
                 f'Got an unexpected location value of "{location}"'
             )
 
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index 7224de8cf..d49677168 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -4,7 +4,7 @@
 from itertools import chain, islice
 
 from dbt.adapters.base.relation import BaseRelation, ComponentName, InformationSchema
-from dbt.exceptions import raise_compiler_error
+from dbt.exceptions import CompilationError
 from dbt.utils import filter_null_values
 from typing import TypeVar
 
@@ -94,7 +94,7 @@ def from_relation(cls, relation, information_schema_view):
                 msg = (
                     f'No location/region found when trying to retrieve "{information_schema_view}"'
                 )
-                raise raise_compiler_error(msg)
+                raise CompilationError(msg)
             info_schema = info_schema.incorporate(location=relation.location)
         return info_schema
 
diff --git a/tests/integration/schema_tests_test/test_schema_v2_tests.py b/tests/integration/schema_tests_test/test_schema_v2_tests.py
index 0709fdb44..7b213db8a 100644
--- a/tests/integration/schema_tests_test/test_schema_v2_tests.py
+++ b/tests/integration/schema_tests_test/test_schema_v2_tests.py
@@ -2,8 +2,6 @@
 import os
 
 from dbt.task.test import TestTask
-from dbt.exceptions import CompilationException
-from dbt.contracts.results import TestStatus
 
 class TestBQSchemaTests(DBTIntegrationTest):
     @property
diff --git a/tests/integration/simple_snapshot_test/test_simple_snapshot.py b/tests/integration/simple_snapshot_test/test_simple_snapshot.py
index 7f6c134c3..191dde72c 100644
--- a/tests/integration/simple_snapshot_test/test_simple_snapshot.py
+++ b/tests/integration/simple_snapshot_test/test_simple_snapshot.py
@@ -1,7 +1,6 @@
 from tests.integration.base import DBTIntegrationTest, use_profile
 from datetime import datetime
 import pytz
-import dbt.exceptions
 
 
 class BaseSimpleSnapshotTest(DBTIntegrationTest):
diff --git a/tests/integration/sources_test/test_sources.py b/tests/integration/sources_test/test_sources.py
index 21dbb19bf..83a586434 100644
--- a/tests/integration/sources_test/test_sources.py
+++ b/tests/integration/sources_test/test_sources.py
@@ -4,7 +4,6 @@
 
 import yaml
 
-from dbt.exceptions import CompilationException
 import dbt.tracking
 import dbt.version
 from dbt.events.functions import reset_metadata_vars
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index cf8bcdfd8..29c7a9e3d 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -156,8 +156,8 @@ def test_acquire_connection_oauth_no_project_validations(self, mock_open_connect
             connection = adapter.acquire_connection('dummy')
             self.assertEqual(connection.type, 'bigquery')
 
-        except dbt.exceptions.ValidationException as e:
-            self.fail('got ValidationException: {}'.format(str(e)))
+        except dbt.exceptions.DbtValidationError as e:
+            self.fail('got DbtValidationError: {}'.format(str(e)))
 
         except BaseException as e:
             raise
@@ -173,8 +173,8 @@ def test_acquire_connection_oauth_validations(self, mock_open_connection):
             connection = adapter.acquire_connection('dummy')
             self.assertEqual(connection.type, 'bigquery')
 
-        except dbt.exceptions.ValidationException as e:
-            self.fail('got ValidationException: {}'.format(str(e)))
+        except dbt.exceptions.DbtValidationError as e:
+            self.fail('got DbtValidationError: {}'.format(str(e)))
 
         except BaseException as e:
             raise
@@ -190,8 +190,8 @@ def test_acquire_connection_service_account_validations(self, mock_open_connecti
             connection = adapter.acquire_connection('dummy')
             self.assertEqual(connection.type, 'bigquery')
 
-        except dbt.exceptions.ValidationException as e:
-            self.fail('got ValidationException: {}'.format(str(e)))
+        except dbt.exceptions.DbtValidationError as e:
+            self.fail('got DbtValidationError: {}'.format(str(e)))
 
         except BaseException as e:
             raise
@@ -207,8 +207,8 @@ def test_acquire_connection_oauth_token_validations(self, mock_open_connection):
             connection = adapter.acquire_connection('dummy')
             self.assertEqual(connection.type, 'bigquery')
 
-        except dbt.exceptions.ValidationException as e:
-            self.fail('got ValidationException: {}'.format(str(e)))
+        except dbt.exceptions.DbtValidationError as e:
+            self.fail('got DbtValidationError: {}'.format(str(e)))
 
         except BaseException as e:
             raise
@@ -224,8 +224,8 @@ def test_acquire_connection_oauth_credentials_validations(self, mock_open_connec
             connection = adapter.acquire_connection('dummy')
             self.assertEqual(connection.type, 'bigquery')
 
-        except dbt.exceptions.ValidationException as e:
-            self.fail('got ValidationException: {}'.format(str(e)))
+        except dbt.exceptions.DbtValidationError as e:
+            self.fail('got DbtValidationError: {}'.format(str(e)))
 
         except BaseException as e:
             raise
@@ -241,8 +241,8 @@ def test_acquire_connection_impersonated_service_account_validations(self, mock_
             connection = adapter.acquire_connection('dummy')
             self.assertEqual(connection.type, 'bigquery')
 
-        except dbt.exceptions.ValidationException as e:
-            self.fail('got ValidationException: {}'.format(str(e)))
+        except dbt.exceptions.DbtValidationError as e:
+            self.fail('got DbtValidationError: {}'.format(str(e)))
 
         except BaseException as e:
             raise
@@ -259,8 +259,8 @@ def test_acquire_connection_priority(self, mock_open_connection):
             self.assertEqual(connection.type, 'bigquery')
             self.assertEqual(connection.credentials.priority, 'batch')
 
-        except dbt.exceptions.ValidationException as e:
-            self.fail('got ValidationException: {}'.format(str(e)))
+        except dbt.exceptions.DbtValidationError as e:
+            self.fail('got DbtValidationError: {}'.format(str(e)))
 
         mock_open_connection.assert_not_called()
         connection.handle
@@ -274,8 +274,8 @@ def test_acquire_connection_maximum_bytes_billed(self, mock_open_connection):
             self.assertEqual(connection.type, 'bigquery')
             self.assertEqual(connection.credentials.maximum_bytes_billed, 0)
 
-        except dbt.exceptions.ValidationException as e:
-            self.fail('got ValidationException: {}'.format(str(e)))
+        except dbt.exceptions.DbtValidationError as e:
+            self.fail('got DbtValidationError: {}'.format(str(e)))
 
         mock_open_connection.assert_not_called()
         connection.handle
@@ -674,10 +674,10 @@ def test_copy_table_materialization_incremental(self):
     def test_parse_partition_by(self):
         adapter = self.get_adapter('oauth')
 
-        with self.assertRaises(dbt.exceptions.ValidationException):
+        with self.assertRaises(dbt.exceptions.DbtValidationError):
             adapter.parse_partition_by("date(ts)")
 
-        with self.assertRaises(dbt.exceptions.ValidationException):
+        with self.assertRaises(dbt.exceptions.DbtValidationError):
             adapter.parse_partition_by("ts")
 
         self.assertEqual(
@@ -841,7 +841,7 @@ def test_parse_partition_by(self):
         )
 
         # Invalid, should raise an error
-        with self.assertRaises(dbt.exceptions.ValidationException):
+        with self.assertRaises(dbt.exceptions.DbtValidationError):
             adapter.parse_partition_by({})
 
         # passthrough

From 02132388891d2ab413b6ed10b4f4e1d5861a9c4e Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Thu, 12 Jan 2023 11:04:15 -0800
Subject: [PATCH 528/860] Bumping version to 1.5.0a1 and generate changelog
 (#465)

* Bumping version to 1.5.0a1 and generate CHANGELOG

* manual changie updates

Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Colin <colin.rogers@dbtlabs.com>
---
 .bumpversion.cfg                              |  2 +-
 .changes/0.0.0.md                             |  1 +
 .changes/1.4.0-b1.md                          | 18 ------------------
 .../1.4.0/Dependencies-20221010-181501.yaml   |  7 -------
 .changes/1.4.0/Features-20220807-164227.yaml  |  7 -------
 .changes/1.4.0/Features-20220926-105700.yaml  |  7 -------
 .changes/1.4.0/Features-20221020-223914.yaml  |  8 --------
 .changes/1.4.0/Fixes-20221019-115618.yaml     |  7 -------
 .changes/1.4.0/Fixes-20221026-192327.yaml     |  7 -------
 .../1.4.0/Under the Hood-20221207-103505.yaml |  7 -------
 .../1.4.0/Under the Hood-20221207-151813.yaml |  8 --------
 .../Dependencies-20221213-135331.yaml         |  7 -------
 .../unreleased/Features-20220823-085601.yaml  |  7 -------
 .../unreleased/Features-20221119-115200.yaml  |  7 -------
 .../unreleased/Features-20230109-105921.yaml  |  7 -------
 .../unreleased/Fixes-20230109-105325.yaml     |  7 -------
 .../Under the Hood-20220805-165120.yaml       |  7 -------
 .../Under the Hood-20221219-212421.yaml       |  7 -------
 .../Under the Hood-20221221-124711.yaml       |  8 --------
 .../Under the Hood-20230110-110016.yaml       |  7 -------
 CHANGELOG.md                                  | 19 +------------------
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 23 files changed, 5 insertions(+), 161 deletions(-)
 delete mode 100644 .changes/1.4.0-b1.md
 delete mode 100644 .changes/1.4.0/Dependencies-20221010-181501.yaml
 delete mode 100644 .changes/1.4.0/Features-20220807-164227.yaml
 delete mode 100644 .changes/1.4.0/Features-20220926-105700.yaml
 delete mode 100644 .changes/1.4.0/Features-20221020-223914.yaml
 delete mode 100644 .changes/1.4.0/Fixes-20221019-115618.yaml
 delete mode 100644 .changes/1.4.0/Fixes-20221026-192327.yaml
 delete mode 100644 .changes/1.4.0/Under the Hood-20221207-103505.yaml
 delete mode 100644 .changes/1.4.0/Under the Hood-20221207-151813.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20221213-135331.yaml
 delete mode 100644 .changes/unreleased/Features-20220823-085601.yaml
 delete mode 100644 .changes/unreleased/Features-20221119-115200.yaml
 delete mode 100644 .changes/unreleased/Features-20230109-105921.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230109-105325.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20220805-165120.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20221219-212421.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20221221-124711.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20230110-110016.yaml

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 487f15b2a..ba1f95c9a 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.4.0b1
+current_version = 1.5.0a1
 parse = (?P<major>\d+)
 	\.(?P<minor>\d+)
 	\.(?P<patch>\d+)
diff --git a/.changes/0.0.0.md b/.changes/0.0.0.md
index f27e1ab71..b719cfe93 100644
--- a/.changes/0.0.0.md
+++ b/.changes/0.0.0.md
@@ -1,5 +1,6 @@
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
+* [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
 - [1.3](https://github.com/dbt-labs/dbt-bigquery/blob/1.3.latest/CHANGELOG.md)
 - [1.2](https://github.com/dbt-labs/dbt-bigquery/blob/1.2.latest/CHANGELOG.md)
 - [1.1](https://github.com/dbt-labs/dbt-bigquery/blob/1.1.latest/CHANGELOG.md)
diff --git a/.changes/1.4.0-b1.md b/.changes/1.4.0-b1.md
deleted file mode 100644
index 22f898ceb..000000000
--- a/.changes/1.4.0-b1.md
+++ /dev/null
@@ -1,18 +0,0 @@
-## dbt-bigquery 1.4.0-b1 - December 15, 2022
-### Features
-- Support for ingestion time partition table on BigQuery as incremental materialization ([#75](https://github.com/dbt-labs/dbt-bigquery/issues/75), [#136](https://github.com/dbt-labs/dbt-bigquery/pull/136))
-- Migrate dbt-utils current_timestamp macros into core + adapters ([#324](https://github.com/dbt-labs/dbt-bigquery/issues/324), [#323](https://github.com/dbt-labs/dbt-bigquery/pull/323))
-- Optimize insert_overwrite incremental strategy with WRITE_TRUNCATE / Partition copy ([#77](https://github.com/dbt-labs/dbt-bigquery/issues/77), [#167](https://github.com/dbt-labs/dbt-bigquery/pull/167))
-### Fixes
-- use execution_project in python models ([#355](https://github.com/dbt-labs/dbt-bigquery/issues/355), [#356](https://github.com/dbt-labs/dbt-bigquery/pull/356))
-- fix args for console link logging ([#362](https://github.com/dbt-labs/dbt-bigquery/issues/362), [#363](https://github.com/dbt-labs/dbt-bigquery/pull/363))
-### Under the Hood
-- event logging changes to be in parity with dbt-core post pr#6291 ([#6139](https://github.com/dbt-labs/dbt-bigquery/issues/6139), [#410](https://github.com/dbt-labs/dbt-bigquery/pull/410))
-- fix issue with tox 4.0.0 release which changes passenv syntax for space-separated variables ([#411](https://github.com/dbt-labs/dbt-bigquery/issues/411), [#411](https://github.com/dbt-labs/dbt-bigquery/pull/411))
-### Dependencies
-- Bump google-cloud-bigquery dependency to < 3.3.3 ([#269](https://github.com/dbt-labs/dbt-bigquery/issues/269), [#332](https://github.com/dbt-labs/dbt-bigquery/pull/332))
-
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#136](https://github.com/dbt-labs/dbt-bigquery/pull/136), [#167](https://github.com/dbt-labs/dbt-bigquery/pull/167), [#363](https://github.com/dbt-labs/dbt-bigquery/pull/363))
-- [@eplus-aolalere](https://github.com/eplus-aolalere) ([#332](https://github.com/dbt-labs/dbt-bigquery/pull/332))
-- [@ericapetersson](https://github.com/ericapetersson) ([#356](https://github.com/dbt-labs/dbt-bigquery/pull/356))
diff --git a/.changes/1.4.0/Dependencies-20221010-181501.yaml b/.changes/1.4.0/Dependencies-20221010-181501.yaml
deleted file mode 100644
index 3fff057ad..000000000
--- a/.changes/1.4.0/Dependencies-20221010-181501.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Dependencies
-body: Bump google-cloud-bigquery dependency to < 3.3.3
-time: 2022-10-10T18:15:01.113219+01:00
-custom:
-  Author: eplus-aolalere
-  Issue: "269"
-  PR: "332"
diff --git a/.changes/1.4.0/Features-20220807-164227.yaml b/.changes/1.4.0/Features-20220807-164227.yaml
deleted file mode 100644
index 9352edc27..000000000
--- a/.changes/1.4.0/Features-20220807-164227.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: Support for ingestion time partition table on BigQuery as incremental materialization
-time: 2022-08-07T16:42:27.232818+02:00
-custom:
-  Author: Kayrnt
-  Issue: "75"
-  PR: "136"
diff --git a/.changes/1.4.0/Features-20220926-105700.yaml b/.changes/1.4.0/Features-20220926-105700.yaml
deleted file mode 100644
index 61e0ac741..000000000
--- a/.changes/1.4.0/Features-20220926-105700.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: Migrate dbt-utils current_timestamp macros into core + adapters
-time: 2022-09-26T10:57:00.942765-07:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "324"
-  PR: "323"
diff --git a/.changes/1.4.0/Features-20221020-223914.yaml b/.changes/1.4.0/Features-20221020-223914.yaml
deleted file mode 100644
index 58b8e60a3..000000000
--- a/.changes/1.4.0/Features-20221020-223914.yaml
+++ /dev/null
@@ -1,8 +0,0 @@
-kind: Features
-body: Optimize insert_overwrite incremental strategy with WRITE_TRUNCATE / Partition
-  copy
-time: 2022-10-20T22:39:14.091878+02:00
-custom:
-  Author: Kayrnt
-  Issue: "77"
-  PR: "167"
diff --git a/.changes/1.4.0/Fixes-20221019-115618.yaml b/.changes/1.4.0/Fixes-20221019-115618.yaml
deleted file mode 100644
index 4aef9955a..000000000
--- a/.changes/1.4.0/Fixes-20221019-115618.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: use execution_project in python models
-time: 2022-10-19T11:56:18.842401+02:00
-custom:
-  Author: ericapetersson
-  Issue: "355"
-  PR: "356"
diff --git a/.changes/1.4.0/Fixes-20221026-192327.yaml b/.changes/1.4.0/Fixes-20221026-192327.yaml
deleted file mode 100644
index 515095c98..000000000
--- a/.changes/1.4.0/Fixes-20221026-192327.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: fix args for console link logging
-time: 2022-10-26T19:23:27.916326+02:00
-custom:
-  Author: Kayrnt
-  Issue: "362"
-  PR: "363"
diff --git a/.changes/1.4.0/Under the Hood-20221207-103505.yaml b/.changes/1.4.0/Under the Hood-20221207-103505.yaml
deleted file mode 100644
index ea1e3a62f..000000000
--- a/.changes/1.4.0/Under the Hood-20221207-103505.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: event logging changes to be in parity with dbt-core post pr#6291
-time: 2022-12-07T10:35:05.567885-06:00
-custom:
-  Author: McKnight-42
-  Issue: "6139"
-  PR: "410"
diff --git a/.changes/1.4.0/Under the Hood-20221207-151813.yaml b/.changes/1.4.0/Under the Hood-20221207-151813.yaml
deleted file mode 100644
index 32084a6a7..000000000
--- a/.changes/1.4.0/Under the Hood-20221207-151813.yaml	
+++ /dev/null
@@ -1,8 +0,0 @@
-kind: Under the Hood
-body: fix issue with tox 4.0.0 release which changes passenv syntax for space-separated
-  variables
-time: 2022-12-07T15:18:13.996118-06:00
-custom:
-  Author: McKnight-42
-  Issue: "411"
-  PR: "411"
diff --git a/.changes/unreleased/Dependencies-20221213-135331.yaml b/.changes/unreleased/Dependencies-20221213-135331.yaml
deleted file mode 100644
index b26c5da1a..000000000
--- a/.changes/unreleased/Dependencies-20221213-135331.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Dependencies
-body: Add supoort for python 3.11
-time: 2022-12-13T13:53:31.182619-06:00
-custom:
-  Author: McKnight-42
-  Issue: "407"
-  PR: "431"
diff --git a/.changes/unreleased/Features-20220823-085601.yaml b/.changes/unreleased/Features-20220823-085601.yaml
deleted file mode 100644
index d8bcd51bc..000000000
--- a/.changes/unreleased/Features-20220823-085601.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: incremental predicates
-time: 2022-08-23T08:56:01.043831-05:00
-custom:
-  Author: dave-connors-3
-  Issue: "283"
-  PR: "284"
diff --git a/.changes/unreleased/Features-20221119-115200.yaml b/.changes/unreleased/Features-20221119-115200.yaml
deleted file mode 100644
index 04189c19c..000000000
--- a/.changes/unreleased/Features-20221119-115200.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: Support BigQuery 'labels' and 'hours_to_expiration' configs for seeds.
-time: 2022-11-19T11:52:00.80272+11:00
-custom:
-  Author: darrylng
-  Issue: "125"
-  PR: "133"
diff --git a/.changes/unreleased/Features-20230109-105921.yaml b/.changes/unreleased/Features-20230109-105921.yaml
deleted file mode 100644
index 0eaec0dcb..000000000
--- a/.changes/unreleased/Features-20230109-105921.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: Fetch table description to `catalog.json`, so it can be shown in dbt docs
-time: 2023-01-09T10:59:21.213259-06:00
-custom:
-  Author: McKnight-42, halvorlu
-  Issue: "173"
-  PR: "174"
diff --git a/.changes/unreleased/Fixes-20230109-105325.yaml b/.changes/unreleased/Fixes-20230109-105325.yaml
deleted file mode 100644
index 2a16883fc..000000000
--- a/.changes/unreleased/Fixes-20230109-105325.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: stop eliminating trailing whitespace in incremental merge sql
-time: 2023-01-09T10:53:25.837837-08:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "457"
-  PR: "458"
diff --git a/.changes/unreleased/Under the Hood-20220805-165120.yaml b/.changes/unreleased/Under the Hood-20220805-165120.yaml
deleted file mode 100644
index dacf18dfe..000000000
--- a/.changes/unreleased/Under the Hood-20220805-165120.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: use gcp retry logic for new connections
-time: 2022-08-05T16:51:20.590446-04:00
-custom:
-  Author: nathaniel-may
-  Issue: "229"
-  PR: "230"
diff --git a/.changes/unreleased/Under the Hood-20221219-212421.yaml b/.changes/unreleased/Under the Hood-20221219-212421.yaml
deleted file mode 100644
index c381a18c5..000000000
--- a/.changes/unreleased/Under the Hood-20221219-212421.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: Truncates Job Labels to 63 Characters
-time: 2022-12-19T21:24:21.304277Z
-custom:
-  Author: ernestoongaro
-  Issue: "202"
-  PR: "442"
diff --git a/.changes/unreleased/Under the Hood-20221221-124711.yaml b/.changes/unreleased/Under the Hood-20221221-124711.yaml
deleted file mode 100644
index fcb83f9eb..000000000
--- a/.changes/unreleased/Under the Hood-20221221-124711.yaml	
+++ /dev/null
@@ -1,8 +0,0 @@
-kind: Under the Hood
-body: Partition By unit test was failing, referenced the incorrect exception to be
-  raised
-time: 2022-12-21T12:47:11.619388-05:00
-custom:
-  Author: mikealfare
-  Issue: "443"
-  PR: "445"
diff --git a/.changes/unreleased/Under the Hood-20230110-110016.yaml b/.changes/unreleased/Under the Hood-20230110-110016.yaml
deleted file mode 100644
index 3f428a801..000000000
--- a/.changes/unreleased/Under the Hood-20230110-110016.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: Update exception names to match dbt-core
-time: 2023-01-10T11:00:16.649793-06:00
-custom:
-  Author: emmyoop
-  Issue: "441"
-  PR: "460"
diff --git a/CHANGELOG.md b/CHANGELOG.md
index f1e182733..56e5e7226 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -4,26 +4,9 @@
 - Changes are listed under the (pre)release in which they first appear. Subsequent releases include changes from previous releases.
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
-## dbt-bigquery 1.4.0-b1 - December 15, 2022
-### Features
-- Support for ingestion time partition table on BigQuery as incremental materialization ([#75](https://github.com/dbt-labs/dbt-bigquery/issues/75), [#136](https://github.com/dbt-labs/dbt-bigquery/pull/136))
-- Migrate dbt-utils current_timestamp macros into core + adapters ([#324](https://github.com/dbt-labs/dbt-bigquery/issues/324), [#323](https://github.com/dbt-labs/dbt-bigquery/pull/323))
-- Optimize insert_overwrite incremental strategy with WRITE_TRUNCATE / Partition copy ([#77](https://github.com/dbt-labs/dbt-bigquery/issues/77), [#167](https://github.com/dbt-labs/dbt-bigquery/pull/167))
-### Fixes
-- use execution_project in python models ([#355](https://github.com/dbt-labs/dbt-bigquery/issues/355), [#356](https://github.com/dbt-labs/dbt-bigquery/pull/356))
-- fix args for console link logging ([#362](https://github.com/dbt-labs/dbt-bigquery/issues/362), [#363](https://github.com/dbt-labs/dbt-bigquery/pull/363))
-### Under the Hood
-- event logging changes to be in parity with dbt-core post pr#6291 ([#6139](https://github.com/dbt-labs/dbt-bigquery/issues/6139), [#410](https://github.com/dbt-labs/dbt-bigquery/pull/410))
-- fix issue with tox 4.0.0 release which changes passenv syntax for space-separated variables ([#411](https://github.com/dbt-labs/dbt-bigquery/issues/411), [#411](https://github.com/dbt-labs/dbt-bigquery/pull/411))
-### Dependencies
-- Bump google-cloud-bigquery dependency to < 3.3.3 ([#269](https://github.com/dbt-labs/dbt-bigquery/issues/269), [#332](https://github.com/dbt-labs/dbt-bigquery/pull/332))
-
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#136](https://github.com/dbt-labs/dbt-bigquery/pull/136), [#167](https://github.com/dbt-labs/dbt-bigquery/pull/167), [#363](https://github.com/dbt-labs/dbt-bigquery/pull/363))
-- [@eplus-aolalere](https://github.com/eplus-aolalere) ([#332](https://github.com/dbt-labs/dbt-bigquery/pull/332))
-- [@ericapetersson](https://github.com/ericapetersson) ([#356](https://github.com/dbt-labs/dbt-bigquery/pull/356))
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
+* [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
 - [1.3](https://github.com/dbt-labs/dbt-bigquery/blob/1.3.latest/CHANGELOG.md)
 - [1.2](https://github.com/dbt-labs/dbt-bigquery/blob/1.2.latest/CHANGELOG.md)
 - [1.1](https://github.com/dbt-labs/dbt-bigquery/blob/1.1.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 27cfeecd9..219c289b1 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.4.0b1"
+version = "1.5.0a1"
diff --git a/setup.py b/setup.py
index 52defb47d..a7b278203 100644
--- a/setup.py
+++ b/setup.py
@@ -57,7 +57,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.4.0b1"
+package_version = "1.5.0a1"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From b24e09b6036bc0137a1f6f1e27845733fd758437 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Thu, 19 Jan 2023 00:10:26 -0600
Subject: [PATCH 529/860] add 1.3 and 1.4 to testing matrix (#467)

---
 .github/workflows/release-branch-tests.yml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/.github/workflows/release-branch-tests.yml b/.github/workflows/release-branch-tests.yml
index 4e29dd0f9..282efdf2c 100644
--- a/.github/workflows/release-branch-tests.yml
+++ b/.github/workflows/release-branch-tests.yml
@@ -39,7 +39,7 @@ jobs:
       max-parallel: 1
       fail-fast: false
       matrix:
-        branch: [1.0.latest, 1.1.latest, 1.2.latest, main]
+        branch: [1.0.latest, 1.1.latest, 1.2.latest, 1.3.latest, 1.4.latest, main]
         workflow_name: [main.yml, integration.yml]
 
     steps:

From 1d6b428fa44fc9528d9c31736e991e72098d20ad Mon Sep 17 00:00:00 2001
From: Jeremy Cohen <jeremy@dbtlabs.com>
Date: Thu, 19 Jan 2023 13:48:01 +0100
Subject: [PATCH 530/860] Convert incremental on_schema_change tests (#469)

* Convert incremental on_schema_change tests

* Switch to dbt-core main
---
 .../test_incremental_on_schema_change.py      | 154 +++++++++++++++++
 .../test_incremental_predicates.py            |   0
 .../test_incremental_unique_id.py             |   0
 .../models/incremental_append_new_columns.sql |  29 ----
 ...remental_append_new_columns_remove_one.sql |  28 ---
 ...l_append_new_columns_remove_one_target.sql |  19 --
 .../incremental_append_new_columns_target.sql |  19 --
 .../models/incremental_fail.sql               |  19 --
 .../models/incremental_ignore.sql             |  19 --
 .../models/incremental_ignore_target.sql      |  15 --
 .../models/incremental_sync_all_columns.sql   |  31 ----
 ...c_all_columns_dynamic_insert_overwrite.sql |  40 -----
 .../incremental_sync_all_columns_target.sql   |  20 ---
 ...ncremental_time_ingestion_partitioning.sql |  38 ----
 ...tal_time_ingestion_partitioning_target.sql |  24 ---
 .../models/model_a.sql                        |  22 ---
 .../models/schema.yml                         |  80 ---------
 .../test_incremental_schema.py                | 162 ------------------
 .../tests/select_from_a.sql                   |   1 -
 ...ct_from_incremental_append_new_columns.sql |   1 -
 ...remental_append_new_columns_remove_one.sql |   1 -
 ...l_append_new_columns_remove_one_target.sql |   1 -
 ..._incremental_append_new_columns_target.sql |   1 -
 .../tests/select_from_incremental_ignore.sql  |   1 -
 .../select_from_incremental_ignore_target.sql |   1 -
 ...lect_from_incremental_sync_all_columns.sql |   1 -
 ...om_incremental_sync_all_columns_target.sql |   1 -
 ...ncremental_time_ingestion_partitioning.sql |   1 -
 ...tal_time_ingestion_partitioning_target.sql |   1 -
 29 files changed, 154 insertions(+), 576 deletions(-)
 create mode 100644 tests/functional/adapter/incremental/test_incremental_on_schema_change.py
 rename tests/functional/adapter/{ => incremental}/test_incremental_predicates.py (100%)
 rename tests/functional/adapter/{ => incremental}/test_incremental_unique_id.py (100%)
 delete mode 100644 tests/integration/incremental_schema_tests/models/incremental_append_new_columns.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one_target.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/incremental_append_new_columns_target.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/incremental_fail.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/incremental_ignore.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/incremental_ignore_target.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/incremental_sync_all_columns.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_dynamic_insert_overwrite.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_target.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning_target.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/model_a.sql
 delete mode 100644 tests/integration/incremental_schema_tests/models/schema.yml
 delete mode 100644 tests/integration/incremental_schema_tests/test_incremental_schema.py
 delete mode 100644 tests/integration/incremental_schema_tests/tests/select_from_a.sql
 delete mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns.sql
 delete mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one.sql
 delete mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one_target.sql
 delete mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_target.sql
 delete mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore.sql
 delete mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore_target.sql
 delete mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns.sql
 delete mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns_target.sql
 delete mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning.sql
 delete mode 100644 tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning_target.sql

diff --git a/tests/functional/adapter/incremental/test_incremental_on_schema_change.py b/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
new file mode 100644
index 000000000..882b378a2
--- /dev/null
+++ b/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
@@ -0,0 +1,154 @@
+import pytest
+
+from dbt.tests.adapter.incremental.test_incremental_on_schema_change import (
+    BaseIncrementalOnSchemaChangeSetup,
+    BaseIncrementalOnSchemaChange,
+)
+
+from dbt.tests.adapter.incremental.fixtures import (
+    _MODELS__A,
+    _MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_TARGET,
+)
+
+
+class TestIncrementalOnSchemaChange(BaseIncrementalOnSchemaChange):
+    pass
+
+
+_MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_DYNAMIC_INSERT_OVERWRITE = """
+{{
+    config(
+        materialized='incremental',
+        unique_key='id',
+        on_schema_change='sync_all_columns',
+        partition_by={
+            "field": "id",
+            "data_type": "int64",
+            "range": {
+                "start": 1,
+                "end": 6,
+                "interval": 1
+            }
+        },
+        incremental_strategy='insert_overwrite'
+    )
+}}
+
+WITH source_data AS (SELECT * FROM {{ ref('model_a') }} )
+
+{% set string_type = 'string' %}
+
+{% if is_incremental() %}
+
+SELECT id, 
+       cast(field1 as {{string_type}}) as field1, 
+       cast(field3 as {{string_type}}) as field3, -- to validate new fields
+       cast(field4 as {{string_type}}) AS field4 -- to validate new fields
+
+FROM source_data WHERE id > _dbt_max_partition
+
+{% else %}
+
+select id, 
+       cast(field1 as {{string_type}}) as field1, 
+       cast(field2 as {{string_type}}) as field2
+
+from source_data where id <= 3
+
+{% endif %}
+"""
+
+_MODELS__INCREMENTAL_TIME_INGESTION_PARTITIONING = """
+
+{{
+    config(
+        materialized="incremental",
+        incremental_strategy='insert_overwrite',
+        partition_by={
+            "field": "date_hour",
+            "data_type": "datetime",
+            "granularity": "hour",
+            "time_ingestion_partitioning": true
+        }
+    )
+}}
+
+
+with data as (
+
+    {% if not is_incremental() %}
+
+        select 1 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 2 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 3 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 4 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour
+
+    {% else %}
+
+        -- we want to overwrite the 4 records in the 2020-01-01 01:00:00 partition
+        -- with the 2 records below, but add two more in the 2020-01-00 02:00:00 partition
+        select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour union all
+        select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+
+    {% endif %}
+
+)
+
+select * from data
+"""
+
+_MODELS__INCREMENTAL_TIME_INGESTION_PARTITIONING_TARGET = """
+{{
+    config(
+        materialized="incremental",
+        partition_by={
+            "field": "date_hour",
+            "data_type": "datetime",
+            "granularity": "hour",
+            "time_ingestion_partitioning": true
+        }
+    )
+}}
+
+{% if not is_incremental() %}
+
+    select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+    select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+
+{% else %}
+
+    select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+    select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+
+{% endif %}
+"""
+
+class TestIncrementalOnSchemaChangeBigQuerySpecific(BaseIncrementalOnSchemaChangeSetup):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "model_a.sql": _MODELS__A,
+            "incremental_sync_all_columns_dynamic_insert_overwrite.sql":
+                _MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_DYNAMIC_INSERT_OVERWRITE,
+            "incremental_sync_all_columns_target.sql":
+                _MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_TARGET,
+            "incremental_time_ingestion_partitioning.sql":
+                _MODELS__INCREMENTAL_TIME_INGESTION_PARTITIONING,
+            "incremental_time_ingestion_partitioning_target.sql":
+                _MODELS__INCREMENTAL_TIME_INGESTION_PARTITIONING_TARGET,
+        }
+    
+    def test_run_incremental_sync_all_columns_dynamic_insert_overwrite(self, project):
+        select = 'model_a incremental_sync_all_columns_dynamic_insert_overwrite incremental_sync_all_columns_target'
+        compare_source = 'incremental_sync_all_columns_dynamic_insert_overwrite'
+        compare_target = 'incremental_sync_all_columns_target'
+        self.run_twice_and_assert(select, compare_source, compare_target, project)
+    
+    # TODO: this test was added here, but it doesn't actually use 'on_schema_change'
+    def test_run_incremental_time_ingestion_partitioning(self, project):
+        select = 'model_a incremental_time_ingestion_partitioning incremental_time_ingestion_partitioning_target'
+        compare_source = 'incremental_time_ingestion_partitioning'
+        compare_target = 'incremental_time_ingestion_partitioning_target'
+        self.run_twice_and_assert(select, compare_source, compare_target, project)
diff --git a/tests/functional/adapter/test_incremental_predicates.py b/tests/functional/adapter/incremental/test_incremental_predicates.py
similarity index 100%
rename from tests/functional/adapter/test_incremental_predicates.py
rename to tests/functional/adapter/incremental/test_incremental_predicates.py
diff --git a/tests/functional/adapter/test_incremental_unique_id.py b/tests/functional/adapter/incremental/test_incremental_unique_id.py
similarity index 100%
rename from tests/functional/adapter/test_incremental_unique_id.py
rename to tests/functional/adapter/incremental/test_incremental_unique_id.py
diff --git a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns.sql b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns.sql
deleted file mode 100644
index 351a397b9..000000000
--- a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns.sql
+++ /dev/null
@@ -1,29 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key='id',
-        on_schema_change='append_new_columns'
-    )
-}}
-
-{% set string_type = 'string' %}
-
-WITH source_data AS (SELECT * FROM {{ ref('model_a') }} )
-
-{% if is_incremental()  %}
-
-SELECT id, 
-       cast(field1 as {{string_type}}) as field1,
-       cast(field2 as {{string_type}}) as field2,
-       cast(field3 as {{string_type}}) as field3,
-       cast(field4 as {{string_type}}) as field4 
-FROM source_data WHERE id NOT IN (SELECT id from {{ this }} )
-
-{% else %}
-
-SELECT id, 
-       cast(field1 as {{string_type}}) as field1, 
-       cast(field2 as {{string_type}}) as field2 
-FROM source_data where id <= 3
-
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one.sql b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one.sql
deleted file mode 100644
index 2ff6c6f48..000000000
--- a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one.sql
+++ /dev/null
@@ -1,28 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key='id',
-        on_schema_change='append_new_columns'
-    )
-}}
-
-{% set string_type = 'string' %}
-
-WITH source_data AS (SELECT * FROM {{ ref('model_a') }} )
-
-{% if is_incremental()  %}
-
-SELECT id,
-       cast(field1 as {{string_type}}) as field1,
-       cast(field3 as {{string_type}}) as field3,
-       cast(field4 as {{string_type}}) as field4
-FROM source_data WHERE id NOT IN (SELECT id from {{ this }} )
-
-{% else %}
-
-SELECT id,
-       cast(field1 as {{string_type}}) as field1,
-       cast(field2 as {{string_type}}) as field2
-FROM source_data where id <= 3
-
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one_target.sql b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one_target.sql
deleted file mode 100644
index c70029d9b..000000000
--- a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_remove_one_target.sql
+++ /dev/null
@@ -1,19 +0,0 @@
-{{
-    config(materialized='table')
-}}
-
-{% set string_type = 'string' %}
-
-with source_data as (
-
-    select * from {{ ref('model_a') }}
-
-)
-
-select id,
-       cast(field1 as {{string_type}}) as field1,
-       cast(CASE WHEN id >  3 THEN NULL ELSE field2 END as {{string_type}}) AS field2,
-       cast(CASE WHEN id <= 3 THEN NULL ELSE field3 END as {{string_type}}) AS field3,
-       cast(CASE WHEN id <= 3 THEN NULL ELSE field4 END as {{string_type}}) AS field4
-
-from source_data
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_target.sql b/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_target.sql
deleted file mode 100644
index 8f65ed71a..000000000
--- a/tests/integration/incremental_schema_tests/models/incremental_append_new_columns_target.sql
+++ /dev/null
@@ -1,19 +0,0 @@
-{{ 
-    config(materialized='table') 
-}}
-
-{% set string_type = 'string' %}
-
-with source_data as (
-
-    select * from {{ ref('model_a') }}
-
-)
-
-select id
-       ,cast(field1 as {{string_type}}) as field1
-       ,cast(field2 as {{string_type}}) as field2
-       ,cast(CASE WHEN id <= 3 THEN NULL ELSE field3 END as {{string_type}}) AS field3
-       ,cast(CASE WHEN id <= 3 THEN NULL ELSE field4 END as {{string_type}}) AS field4
-
-from source_data
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/incremental_fail.sql b/tests/integration/incremental_schema_tests/models/incremental_fail.sql
deleted file mode 100644
index 590f5b56d..000000000
--- a/tests/integration/incremental_schema_tests/models/incremental_fail.sql
+++ /dev/null
@@ -1,19 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key='id',
-        on_schema_change='fail'
-    )
-}}
-
-WITH source_data AS (SELECT * FROM {{ ref('model_a') }} )
-
-{% if is_incremental()  %}
-
-SELECT id, field1, field2 FROM source_data
-
-{% else %}
-
-SELECT id, field1, field3 FROm source_data
-
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/incremental_ignore.sql b/tests/integration/incremental_schema_tests/models/incremental_ignore.sql
deleted file mode 100644
index 51dee6022..000000000
--- a/tests/integration/incremental_schema_tests/models/incremental_ignore.sql
+++ /dev/null
@@ -1,19 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key='id',
-        on_schema_change='ignore'
-    )
-}}
-
-WITH source_data AS (SELECT * FROM {{ ref('model_a') }} )
-
-{% if is_incremental() %}
-
-SELECT id, field1, field2, field3, field4 FROM source_data WHERE id NOT IN (SELECT id from {{ this }} )
-
-{% else %}
-
-SELECT id, field1, field2 FROM source_data LIMIT 3
-
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/incremental_ignore_target.sql b/tests/integration/incremental_schema_tests/models/incremental_ignore_target.sql
deleted file mode 100644
index 92d4564e0..000000000
--- a/tests/integration/incremental_schema_tests/models/incremental_ignore_target.sql
+++ /dev/null
@@ -1,15 +0,0 @@
-{{ 
-    config(materialized='table') 
-}}
-
-with source_data as (
-
-    select * from {{ ref('model_a') }}
-
-)
-
-select id
-       ,field1
-       ,field2
-
-from source_data
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns.sql b/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns.sql
deleted file mode 100644
index 9c79d649c..000000000
--- a/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns.sql
+++ /dev/null
@@ -1,31 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key='id',
-        on_schema_change='sync_all_columns'
-        
-    )
-}}
-
-WITH source_data AS (SELECT * FROM {{ ref('model_a') }} )
-
-{% set string_type = 'string' %}
-
-{% if is_incremental() %}
-
-SELECT id, 
-       cast(field1 as {{string_type}}) as field1, 
-       cast(field3 as {{string_type}}) as field3, -- to validate new fields
-       cast(field4 as {{string_type}}) AS field4 -- to validate new fields
-
-FROM source_data WHERE id NOT IN (SELECT id from {{ this }} )
-
-{% else %}
-
-select id, 
-       cast(field1 as {{string_type}}) as field1, 
-       cast(field2 as {{string_type}}) as field2
-
-from source_data where id <= 3
-
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_dynamic_insert_overwrite.sql b/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_dynamic_insert_overwrite.sql
deleted file mode 100644
index d82e683a3..000000000
--- a/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_dynamic_insert_overwrite.sql
+++ /dev/null
@@ -1,40 +0,0 @@
-{{
-    config(
-        materialized='incremental',
-        unique_key='id',
-        on_schema_change='sync_all_columns',
-        partition_by={
-            "field": "id",
-            "data_type": "int64",
-            "range": {
-                "start": 1,
-                "end": 6,
-                "interval": 1
-            }
-        },
-        incremental_strategy='insert_overwrite'
-    )
-}}
-
-WITH source_data AS (SELECT * FROM {{ ref('model_a') }} )
-
-{% set string_type = 'string' %}
-
-{% if is_incremental() %}
-
-SELECT id, 
-       cast(field1 as {{string_type}}) as field1, 
-       cast(field3 as {{string_type}}) as field3, -- to validate new fields
-       cast(field4 as {{string_type}}) AS field4 -- to validate new fields
-
-FROM source_data WHERE id > _dbt_max_partition
-
-{% else %}
-
-select id, 
-       cast(field1 as {{string_type}}) as field1, 
-       cast(field2 as {{string_type}}) as field2
-
-from source_data where id <= 3
-
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_target.sql b/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_target.sql
deleted file mode 100644
index 2fcd88121..000000000
--- a/tests/integration/incremental_schema_tests/models/incremental_sync_all_columns_target.sql
+++ /dev/null
@@ -1,20 +0,0 @@
-{{ 
-    config(materialized='table') 
-}}
-
-with source_data as (
-
-    select * from {{ ref('model_a') }}
-
-)
-
-{% set string_type = 'string' %}
-
-select id
-       ,cast(field1 as {{string_type}}) as field1
-       --,field2
-       ,cast(case when id <= 3 then null else field3 end as {{string_type}}) as field3
-       ,cast(case when id <= 3 then null else field4 end as {{string_type}}) as field4
-
-from source_data
-order by id 
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning.sql b/tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning.sql
deleted file mode 100644
index ce064b33c..000000000
--- a/tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning.sql
+++ /dev/null
@@ -1,38 +0,0 @@
-
-{{
-    config(
-        materialized="incremental",
-        incremental_strategy='insert_overwrite',
-        partition_by={
-            "field": "date_hour",
-            "data_type": "datetime",
-            "granularity": "hour",
-            "time_ingestion_partitioning": true
-        }
-    )
-}}
-
-
-with data as (
-    
-    {% if not is_incremental() %}
-    
-        select 1 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 2 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 3 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 4 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour
-
-    {% else %}
-    
-        -- we want to overwrite the 4 records in the 2020-01-01 01:00:00 partition
-        -- with the 2 records below, but add two more in the 2020-01-00 02:00:00 partition
-        select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour union all
-        select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
-    
-    {% endif %}
-
-)
-
-select * from data
diff --git a/tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning_target.sql b/tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning_target.sql
deleted file mode 100644
index 2f25229de..000000000
--- a/tests/integration/incremental_schema_tests/models/incremental_time_ingestion_partitioning_target.sql
+++ /dev/null
@@ -1,24 +0,0 @@
-
-{{
-    config(
-        materialized="incremental",
-        partition_by={
-            "field": "date_hour",
-            "data_type": "datetime",
-            "granularity": "hour",
-            "time_ingestion_partitioning": true
-        }
-    )
-}}
-
-{% if not is_incremental() %}
-
-    select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-    select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
-
-{% else %}
-
-    select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-    select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
-
-{% endif %}
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/model_a.sql b/tests/integration/incremental_schema_tests/models/model_a.sql
deleted file mode 100644
index 2a0b2ddaf..000000000
--- a/tests/integration/incremental_schema_tests/models/model_a.sql
+++ /dev/null
@@ -1,22 +0,0 @@
-{{ 
-    config(materialized='table') 
-}}
-
-with source_data as (
-
-    select 1 as id, 'aaa' as field1, 'bbb' as field2, 111 as field3, 'TTT' as field4
-    union all select 2 as id, 'ccc' as field1, 'ddd' as field2, 222 as field3, 'UUU' as field4
-    union all select 3 as id, 'eee' as field1, 'fff' as field2, 333 as field3, 'VVV' as field4
-    union all select 4 as id, 'ggg' as field1, 'hhh' as field2, 444 as field3, 'WWW' as field4
-    union all select 5 as id, 'iii' as field1, 'jjj' as field2, 555 as field3, 'XXX' as field4
-    union all select 6 as id, 'kkk' as field1, 'lll' as field2, 666 as field3, 'YYY' as field4
-
-)
-
-select id
-       ,field1
-       ,field2
-       ,field3
-       ,field4
-
-from source_data
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/models/schema.yml b/tests/integration/incremental_schema_tests/models/schema.yml
deleted file mode 100644
index fd3136102..000000000
--- a/tests/integration/incremental_schema_tests/models/schema.yml
+++ /dev/null
@@ -1,80 +0,0 @@
-version: 2
-
-models:
-  - name: model_a
-    columns:
-      - name: id
-        tags: [column_level_tag]
-        tests:
-          - unique
-
-  - name: incremental_ignore
-    columns:
-      - name: id
-        tags: [column_level_tag]
-        tests:
-          - unique
-  
-  - name: incremental_ignore_target
-    columns:
-      - name: id
-        tags: [column_level_tag]
-        tests:
-          - unique
-  
-  - name: incremental_append_new_columns
-    columns:
-      - name: id
-        tags: [column_level_tag]
-        tests:
-          - unique
-
-  - name: incremental_append_new_columns_target
-    columns:
-      - name: id
-        tags: [column_level_tag]
-        tests:
-          - unique
-
-  - name: incremental_append_new_columns_remove_one
-    columns:
-      - name: id
-        tags: [column_level_tag]
-        tests:
-          - unique
-
-  - name: incremental_append_new_columns_remove_one_target
-    columns:
-      - name: id
-        tags: [column_level_tag]
-        tests:
-          - unique
- 
-  - name: incremental_sync_all_columns
-    columns:
-      - name: id
-        tags: [column_level_tag]
-        tests:
-          - unique
-          
-  - name: incremental_sync_all_columns_target
-    columns:
-      - name: id
-        tags: [column_leveL_tag]
-        tests:
-          - unique
-
-  - name: incremental_time_ingestion_partitioning
-    columns:
-      - name: id
-        tags: [column_level_tag]
-        tests:
-          - unique
-
-  - name: incremental_time_ingestion_partitioning_target
-    columns:
-      - name: id
-        tags: [column_level_tag]
-        tests:
-          - unique
-
diff --git a/tests/integration/incremental_schema_tests/test_incremental_schema.py b/tests/integration/incremental_schema_tests/test_incremental_schema.py
deleted file mode 100644
index 9592c500f..000000000
--- a/tests/integration/incremental_schema_tests/test_incremental_schema.py
+++ /dev/null
@@ -1,162 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-
-class TestSelectionExpansion(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "test_incremental_schema"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        return {
-            "config-version": 2,
-            "test-paths": ["tests"]
-        }
-
-    def list_tests_and_assert(self, include, exclude, expected_tests):
-        list_args = ['ls', '--resource-type', 'test']
-        if include:
-            list_args.extend(('--select', include))
-        if exclude:
-            list_args.extend(('--exclude', exclude))
-        listed = self.run_dbt(list_args)
-        print(listed)
-        assert len(listed) == len(expected_tests)
-        test_names = [name.split('.')[-1] for name in listed]
-        assert sorted(test_names) == sorted(expected_tests)
-
-    def run_tests_and_assert(
-        self, include, exclude, expected_tests, compare_source, compare_target
-    ):
-
-        run_args = ['run']
-        if include:
-            run_args.extend(('--models', include))
-        results_one = self.run_dbt(run_args)
-        results_two = self.run_dbt(run_args)
-
-        self.assertEqual(len(results_one), 3)
-        self.assertEqual(len(results_two), 3)
-
-        test_args = ['test']
-        if include:
-            test_args.extend(('--models', include))
-        if exclude:
-            test_args.extend(('--exclude', exclude))
-
-        results = self.run_dbt(test_args)
-        tests_run = [r.node.name for r in results]
-        assert len(tests_run) == len(expected_tests)
-        assert sorted(tests_run) == sorted(expected_tests)
-        self.assertTablesEqual(compare_source, compare_target)
-
-    def run_incremental_ignore(self):
-        select = 'model_a incremental_ignore incremental_ignore_target'
-        compare_source = 'incremental_ignore'
-        compare_target = 'incremental_ignore_target'
-        exclude = None
-        expected = [
-            'select_from_a',
-            'select_from_incremental_ignore',
-            'select_from_incremental_ignore_target',
-            'unique_model_a_id',
-            'unique_incremental_ignore_id',
-            'unique_incremental_ignore_target_id'
-        ]
-
-        self.list_tests_and_assert(select, exclude, expected)
-        self.run_tests_and_assert(select, exclude, expected, compare_source, compare_target)
-
-    def run_incremental_append_new_columns(self):
-        select = 'model_a incremental_append_new_columns incremental_append_new_columns_target'
-        compare_source = 'incremental_append_new_columns'
-        compare_target = 'incremental_append_new_columns_target'
-        exclude = None
-        expected = [
-            'select_from_a',
-            'select_from_incremental_append_new_columns',
-            'select_from_incremental_append_new_columns_target',
-            'unique_model_a_id',
-            'unique_incremental_append_new_columns_id',
-            'unique_incremental_append_new_columns_target_id'
-        ]
-        self.list_tests_and_assert(select, exclude, expected)
-        self.run_tests_and_assert(select, exclude, expected, compare_source, compare_target)
-
-    def run_incremental_append_new_columns_remove_one(self):
-        select = 'model_a incremental_append_new_columns_remove_one incremental_append_new_columns_remove_one_target'
-        compare_source = 'incremental_append_new_columns_remove_one'
-        compare_target = 'incremental_append_new_columns_remove_one_target'
-        exclude = None
-        expected = [
-            'select_from_a',
-            'select_from_incremental_append_new_columns_remove_one',
-            'select_from_incremental_append_new_columns_remove_one_target',
-            'unique_model_a_id',
-            'unique_incremental_append_new_columns_remove_one_id',
-            'unique_incremental_append_new_columns_remove_one_target_id'
-        ]
-        self.run_tests_and_assert(select, exclude, expected, compare_source, compare_target)
-
-    def run_incremental_sync_all_columns(self):
-        select = 'model_a incremental_sync_all_columns incremental_sync_all_columns_target'
-        compare_source = 'incremental_sync_all_columns'
-        compare_target = 'incremental_sync_all_columns_target'
-        exclude = None
-        expected = [
-            'select_from_a',
-            'select_from_incremental_sync_all_columns',
-            'select_from_incremental_sync_all_columns_target',
-            'unique_model_a_id',
-            'unique_incremental_sync_all_columns_id',
-            'unique_incremental_sync_all_columns_target_id'
-        ]
-        self.list_tests_and_assert(select, exclude, expected)
-        self.run_tests_and_assert(select, exclude, expected, compare_source, compare_target)
-
-    def run_incremental_fail_on_schema_change(self):
-        select = 'model_a incremental_fail'
-        results_one = self.run_dbt(['run', '--models', select, '--full-refresh'])
-        results_two = self.run_dbt(['run', '--models', select], expect_pass = False)
-        self.assertIn('Compilation Error', results_two[1].message)
-
-    def run_incremental_time_ingestion_partitioning(self):
-        select = 'model_a incremental_time_ingestion_partitioning incremental_time_ingestion_partitioning_target'
-        compare_source = 'incremental_time_ingestion_partitioning'
-        compare_target = 'incremental_time_ingestion_partitioning_target'
-        exclude = None
-        expected = [
-            'select_from_a',
-            'select_from_incremental_time_ingestion_partitioning',
-            'select_from_incremental_time_ingestion_partitioning_target',
-            'unique_model_a_id',
-            'unique_incremental_time_ingestion_partitioning_id',
-            'unique_incremental_time_ingestion_partitioning_target_id'
-        ]
-        self.list_tests_and_assert(select, exclude, expected)
-        self.run_tests_and_assert(select, exclude, expected, compare_source, compare_target)
-
-    @use_profile('bigquery')
-    def test__bigquery__run_incremental_ignore(self):
-        self.run_incremental_ignore()
-
-    @use_profile('bigquery')
-    def test__bigquery__run_incremental_append_new_columns(self):
-        self.run_incremental_append_new_columns()
-        self.run_incremental_append_new_columns_remove_one()
-
-    @use_profile('bigquery')
-    def test__bigquery__run_incremental_sync_all_columns(self):
-        self.run_incremental_sync_all_columns()
-
-    @use_profile('bigquery')
-    def test__bigquery__run_incremental_fail_on_schema_change(self):
-        self.run_incremental_fail_on_schema_change()
-
-    @use_profile('bigquery')
-    def test__bigquery__run_incremental_time_ingestion_partitioning(self):
-        self.run_incremental_time_ingestion_partitioning()
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_a.sql b/tests/integration/incremental_schema_tests/tests/select_from_a.sql
deleted file mode 100644
index 3dc8f2857..000000000
--- a/tests/integration/incremental_schema_tests/tests/select_from_a.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('model_a') }} where false
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns.sql
deleted file mode 100644
index 947e84588..000000000
--- a/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('incremental_append_new_columns') }} where false
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one.sql
deleted file mode 100644
index 06d52c6d6..000000000
--- a/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('incremental_append_new_columns_remove_one') }} where false
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one_target.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one_target.sql
deleted file mode 100644
index 07d2412b0..000000000
--- a/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_remove_one_target.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('incremental_append_new_columns_remove_one_target') }} where false
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_target.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_target.sql
deleted file mode 100644
index 8b86eddd7..000000000
--- a/tests/integration/incremental_schema_tests/tests/select_from_incremental_append_new_columns_target.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('incremental_append_new_columns_target') }} where false
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore.sql
deleted file mode 100644
index d565c8464..000000000
--- a/tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('incremental_ignore') }} where false
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore_target.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore_target.sql
deleted file mode 100644
index 35d535c5c..000000000
--- a/tests/integration/incremental_schema_tests/tests/select_from_incremental_ignore_target.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('incremental_ignore_target') }} where false
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns.sql
deleted file mode 100644
index aedc9f803..000000000
--- a/tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('incremental_sync_all_columns') }} where false
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns_target.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns_target.sql
deleted file mode 100644
index 4b703c988..000000000
--- a/tests/integration/incremental_schema_tests/tests/select_from_incremental_sync_all_columns_target.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('incremental_sync_all_columns_target') }} where false
\ No newline at end of file
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning.sql
deleted file mode 100644
index 85e653c11..000000000
--- a/tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('incremental_time_ingestion_partitioning') }} where false
diff --git a/tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning_target.sql b/tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning_target.sql
deleted file mode 100644
index e2533dff7..000000000
--- a/tests/integration/incremental_schema_tests/tests/select_from_incremental_time_ingestion_partitioning_target.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('incremental_time_ingestion_partitioning_target') }} where false

From 38fb796bf213217220ed4bcc6ce237b0b25d507c Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Thu, 26 Jan 2023 01:19:33 -0800
Subject: [PATCH 531/860] Remove test for CT-1629 in dbt-core. (#473)

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .../column_quoting/models-unquoted/model.sql  | 12 ---
 .../column_quoting/models/model.sql           | 12 ---
 .../integration/column_quoting/seeds/seed.csv |  4 -
 .../column_quoting/test_column_quotes.py      | 78 -------------------
 4 files changed, 106 deletions(-)
 delete mode 100644 tests/integration/column_quoting/models-unquoted/model.sql
 delete mode 100644 tests/integration/column_quoting/models/model.sql
 delete mode 100644 tests/integration/column_quoting/seeds/seed.csv
 delete mode 100644 tests/integration/column_quoting/test_column_quotes.py

diff --git a/tests/integration/column_quoting/models-unquoted/model.sql b/tests/integration/column_quoting/models-unquoted/model.sql
deleted file mode 100644
index a7fefd127..000000000
--- a/tests/integration/column_quoting/models-unquoted/model.sql
+++ /dev/null
@@ -1,12 +0,0 @@
-{% set col_a = '`col_a`' %}
-{% set col_b = '`col_b`' %}
-
-{{config(
-    materialized = 'incremental',
-    unique_key = col_a,
-    incremental_strategy = var('strategy')
-    )}}
-
-select
-{{ col_a }}, {{ col_b }}
-from {{ref('seed')}}
diff --git a/tests/integration/column_quoting/models/model.sql b/tests/integration/column_quoting/models/model.sql
deleted file mode 100644
index 6db19b06d..000000000
--- a/tests/integration/column_quoting/models/model.sql
+++ /dev/null
@@ -1,12 +0,0 @@
-{% set col_a = '`col_A`' %}
-{% set col_b = '`col_B`' %}
-
-{{config(
-    materialized = 'incremental',
-    unique_key = col_a,
-    incremental_strategy = var('strategy')
-    )}}
-
-select
-{{ col_a }}, {{ col_b }}
-from {{ref('seed')}}
diff --git a/tests/integration/column_quoting/seeds/seed.csv b/tests/integration/column_quoting/seeds/seed.csv
deleted file mode 100644
index d4a1e26ee..000000000
--- a/tests/integration/column_quoting/seeds/seed.csv
+++ /dev/null
@@ -1,4 +0,0 @@
-col_A,col_B
-1,2
-3,4
-5,6
diff --git a/tests/integration/column_quoting/test_column_quotes.py b/tests/integration/column_quoting/test_column_quotes.py
deleted file mode 100644
index c484317ac..000000000
--- a/tests/integration/column_quoting/test_column_quotes.py
+++ /dev/null
@@ -1,78 +0,0 @@
-from tests.integration.base import DBTIntegrationTest,  use_profile
-import os
-
-
-class BaseColumnQuotingTest(DBTIntegrationTest):
-    def column_quoting(self):
-        raise NotImplementedError('column_quoting not implemented')
-
-    @property
-    def schema(self):
-        return 'dbt_column_quoting'
-
-    @staticmethod
-    def dir(value):
-        return os.path.normpath(value)
-
-    def _run_columnn_quotes(self, strategy='delete+insert'):
-        strategy_vars = '{{"strategy": "{}"}}'.format(strategy)
-        self.run_dbt(['seed', '--vars', strategy_vars])
-        self.run_dbt(['run', '--vars', strategy_vars])
-        self.run_dbt(['run', '--vars', strategy_vars])
-
-
-class TestColumnQuotingDefault(BaseColumnQuotingTest):
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2
-        }
-
-    @property
-    def models(self):
-        return self.dir('models')
-
-    def run_dbt(self, *args, **kwargs):
-        return super().run_dbt(*args, **kwargs)
-
-    @use_profile('bigquery')
-    def test_bigquery_column_quotes(self):
-        self._run_columnn_quotes(strategy='merge')
-
-
-class TestColumnQuotingDisabled(BaseColumnQuotingTest):
-    @property
-    def models(self):
-        return self.dir('models-unquoted')
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seeds': {
-                'quote_columns': False,
-            },
-        }
-
-    @use_profile('bigquery')
-    def test_bigquery_column_quotes_merged(self):
-        self._run_columnn_quotes(strategy='merge')
-
-
-class TestColumnQuotingEnabled(BaseColumnQuotingTest):
-    @property
-    def models(self):
-        return self.dir('models')
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seeds': {
-                'quote_columns': True,
-            },
-        }
-
-    @use_profile('bigquery')
-    def test_bigquery_column_quotes_merged(self):
-        self._run_columnn_quotes(strategy='merge')

From e4ce8c1abf097d78f5a8b2709fa1844c9d85a579 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 26 Jan 2023 11:19:25 -0800
Subject: [PATCH 532/860] remove sources integration tests (#479)

---
 .../sources_test/error_models/model.sql       |   1 -
 .../sources_test/error_models/schema.yml      |  12 --
 .../sources_test/filtered_models/schema.yml   |  18 --
 .../integration/sources_test/macros/macro.sql |  14 --
 .../malformed_models/descendant_model.sql     |   1 -
 .../sources_test/malformed_models/schema.yml  |  14 --
 .../malformed_schema_tests/model.sql          |   1 -
 .../malformed_schema_tests/schema.yml         |  14 --
 .../sources_test/models/descendant_model.sql  |   1 -
 .../sources_test/models/ephemeral_model.sql   |   3 -
 .../models/multi_source_model.sql             |   2 -
 .../models/nonsource_descendant.sql           |   1 -
 .../sources_test/models/schema.yml            |  77 --------
 .../sources_test/models/view_model.sql        |   3 -
 tests/integration/sources_test/seed.sql       | 113 -----------
 .../seeds/expected_multi_source.csv           |   4 -
 .../sources_test/seeds/other_source_table.csv |   4 -
 .../sources_test/seeds/other_table.csv        |   4 -
 .../integration/sources_test/seeds/source.csv | 101 ----------
 .../integration/sources_test/test_sources.py  | 183 ------------------
 20 files changed, 571 deletions(-)
 delete mode 100644 tests/integration/sources_test/error_models/model.sql
 delete mode 100644 tests/integration/sources_test/error_models/schema.yml
 delete mode 100644 tests/integration/sources_test/filtered_models/schema.yml
 delete mode 100644 tests/integration/sources_test/macros/macro.sql
 delete mode 100644 tests/integration/sources_test/malformed_models/descendant_model.sql
 delete mode 100644 tests/integration/sources_test/malformed_models/schema.yml
 delete mode 100644 tests/integration/sources_test/malformed_schema_tests/model.sql
 delete mode 100644 tests/integration/sources_test/malformed_schema_tests/schema.yml
 delete mode 100644 tests/integration/sources_test/models/descendant_model.sql
 delete mode 100644 tests/integration/sources_test/models/ephemeral_model.sql
 delete mode 100644 tests/integration/sources_test/models/multi_source_model.sql
 delete mode 100644 tests/integration/sources_test/models/nonsource_descendant.sql
 delete mode 100644 tests/integration/sources_test/models/schema.yml
 delete mode 100644 tests/integration/sources_test/models/view_model.sql
 delete mode 100644 tests/integration/sources_test/seed.sql
 delete mode 100644 tests/integration/sources_test/seeds/expected_multi_source.csv
 delete mode 100644 tests/integration/sources_test/seeds/other_source_table.csv
 delete mode 100644 tests/integration/sources_test/seeds/other_table.csv
 delete mode 100644 tests/integration/sources_test/seeds/source.csv
 delete mode 100644 tests/integration/sources_test/test_sources.py

diff --git a/tests/integration/sources_test/error_models/model.sql b/tests/integration/sources_test/error_models/model.sql
deleted file mode 100644
index 55bbcba67..000000000
--- a/tests/integration/sources_test/error_models/model.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ source('test_source', 'test_table') }}
diff --git a/tests/integration/sources_test/error_models/schema.yml b/tests/integration/sources_test/error_models/schema.yml
deleted file mode 100644
index 69cf1f304..000000000
--- a/tests/integration/sources_test/error_models/schema.yml
+++ /dev/null
@@ -1,12 +0,0 @@
-version: 2
-sources:
-  - name: test_source
-    loader: custom
-    freshness:
-      warn_after: {count: 10, period: hour}
-      error_after: {count: 1, period: day}
-    schema: invalid
-    tables:
-      - name: test_table
-        identifier: source
-        loaded_at_field: updated_at
diff --git a/tests/integration/sources_test/filtered_models/schema.yml b/tests/integration/sources_test/filtered_models/schema.yml
deleted file mode 100644
index edad7f6ec..000000000
--- a/tests/integration/sources_test/filtered_models/schema.yml
+++ /dev/null
@@ -1,18 +0,0 @@
-version: 2
-sources:
-  - name: test_source
-    loader: custom
-    freshness:
-      warn_after: {count: 10, period: hour}
-      error_after: {count: 1, period: day}
-      filter: id > 1
-    schema: "{{ var(env_var('DBT_TEST_SCHEMA_NAME_VARIABLE')) }}"
-    quoting:
-      identifier: True
-    tables:
-      - name: test_table
-        identifier: source
-        loaded_at_field: updated_at
-        freshness:
-          error_after: {count: 18, period: hour}
-          filter: id > 101
diff --git a/tests/integration/sources_test/macros/macro.sql b/tests/integration/sources_test/macros/macro.sql
deleted file mode 100644
index a607a6e4c..000000000
--- a/tests/integration/sources_test/macros/macro.sql
+++ /dev/null
@@ -1,14 +0,0 @@
-{% macro override_me() -%}
-    {{ exceptions.raise_compiler_error('this is a bad macro') }}
-{%- endmacro %}
-
-{% macro happy_little_macro() -%}
-    {{ override_me() }}
-{%- endmacro %}
-
-
-{% macro vacuum_source(source_name, table_name) -%}
-    {% call statement('stmt', auto_begin=false, fetch_result=false) %}
-        vacuum {{ source(source_name, table_name) }}
-    {% endcall %}
-{%- endmacro %}
diff --git a/tests/integration/sources_test/malformed_models/descendant_model.sql b/tests/integration/sources_test/malformed_models/descendant_model.sql
deleted file mode 100644
index 55bbcba67..000000000
--- a/tests/integration/sources_test/malformed_models/descendant_model.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ source('test_source', 'test_table') }}
diff --git a/tests/integration/sources_test/malformed_models/schema.yml b/tests/integration/sources_test/malformed_models/schema.yml
deleted file mode 100644
index 544d18d65..000000000
--- a/tests/integration/sources_test/malformed_models/schema.yml
+++ /dev/null
@@ -1,14 +0,0 @@
-version: 2
-sources:
-  - name: test_source
-    loader: custom
-    schema: "{{ var('test_run_schema') }}"
-    tables:
-      - name: test_table
-        identifier: source
-        tests:
-          - relationships:
-            # this is invalid (list of 3 1-key dicts instead of a single 3-key dict)
-              - column_name: favorite_color
-              - to: ref('descendant_model')
-              - field: favorite_color
diff --git a/tests/integration/sources_test/malformed_schema_tests/model.sql b/tests/integration/sources_test/malformed_schema_tests/model.sql
deleted file mode 100644
index 55bbcba67..000000000
--- a/tests/integration/sources_test/malformed_schema_tests/model.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ source('test_source', 'test_table') }}
diff --git a/tests/integration/sources_test/malformed_schema_tests/schema.yml b/tests/integration/sources_test/malformed_schema_tests/schema.yml
deleted file mode 100644
index d72ab2eee..000000000
--- a/tests/integration/sources_test/malformed_schema_tests/schema.yml
+++ /dev/null
@@ -1,14 +0,0 @@
-version: 2
-sources:
-  - name: test_source
-    schema: "{{ var('test_run_schema') }}"
-    tables:
-      - name: test_table
-        identifier: source
-        columns:
-          - name: favorite_color
-            tests:
-              - relationships:
-                  to: ref('model')
-                  # this will get rendered as its literal
-                  field: "{{ 'favorite' ~ 'color' }}"
diff --git a/tests/integration/sources_test/models/descendant_model.sql b/tests/integration/sources_test/models/descendant_model.sql
deleted file mode 100644
index 55bbcba67..000000000
--- a/tests/integration/sources_test/models/descendant_model.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ source('test_source', 'test_table') }}
diff --git a/tests/integration/sources_test/models/ephemeral_model.sql b/tests/integration/sources_test/models/ephemeral_model.sql
deleted file mode 100644
index 8de35cd3e..000000000
--- a/tests/integration/sources_test/models/ephemeral_model.sql
+++ /dev/null
@@ -1,3 +0,0 @@
-{{ config(materialized='ephemeral') }}
-
-select 1 as id
diff --git a/tests/integration/sources_test/models/multi_source_model.sql b/tests/integration/sources_test/models/multi_source_model.sql
deleted file mode 100644
index e310206b0..000000000
--- a/tests/integration/sources_test/models/multi_source_model.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-select * from {{ source('test_source', 'other_test_table')}}
-	join {{ source('other_source', 'test_table')}} using (id)
diff --git a/tests/integration/sources_test/models/nonsource_descendant.sql b/tests/integration/sources_test/models/nonsource_descendant.sql
deleted file mode 100644
index 97f2151c7..000000000
--- a/tests/integration/sources_test/models/nonsource_descendant.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ schema }}.source
diff --git a/tests/integration/sources_test/models/schema.yml b/tests/integration/sources_test/models/schema.yml
deleted file mode 100644
index f02eb1345..000000000
--- a/tests/integration/sources_test/models/schema.yml
+++ /dev/null
@@ -1,77 +0,0 @@
-version: 2
-models:
-  - name: descendant_model
-    columns:
-      - name: favorite_color
-        tests:
-          - relationships:
-             to: source('test_source', 'test_table')
-             field: favorite_color
-
-sources:
-  - name: test_source
-    loader: custom
-    freshness:
-      warn_after: {count: 10, period: hour}
-      error_after: {count: 1, period: day}
-    schema: "{{ var(env_var('DBT_TEST_SCHEMA_NAME_VARIABLE')) }}"
-    quoting:
-      identifier: True
-    tags:
-      - my_test_source_tag
-    tables:
-      - name: test_table
-        identifier: source
-        loaded_at_field: "{{ var('test_loaded_at') | as_text }}"
-        freshness:
-          error_after: {count: 18, period: hour}
-        tags:
-          - my_test_source_table_tag
-        columns:
-          - name: favorite_color
-            description: The favorite color
-          - name: id
-            description: The user ID
-            tests:
-              - unique
-              - not_null
-            tags:
-              - id_column
-          - name: first_name
-            description: The first name of the user
-            tests: []
-          - name: email
-            description: The email address of the user
-          - name: ip_address
-            description: The last IP address the user logged in from
-          - name: updated_at
-            description: The last update time for this user
-        tests:
-          - relationships:
-              # do this as a table-level test, just to test out that aspect
-              column_name: favorite_color
-              to: ref('descendant_model')
-              field: favorite_color
-      - name: other_test_table
-        identifier: other_table
-        columns:
-          - name: id
-            tests:
-              - not_null
-              - unique
-            tags:
-              - id_column
-      - name: disabled_test_table
-        freshness: null
-        loaded_at_field: "{{ var('test_loaded_at') | as_text }}"
-  - name: other_source
-    schema: "{{ var('test_run_schema') }}"
-    quoting:
-      identifier: True
-    tables:
-      - name: test_table
-        identifier: other_source_table
-  - name: external_source
-    schema: "{{ var('test_run_alt_schema', var('test_run_schema')) }}"
-    tables:
-      - name: table
diff --git a/tests/integration/sources_test/models/view_model.sql b/tests/integration/sources_test/models/view_model.sql
deleted file mode 100644
index ecb330804..000000000
--- a/tests/integration/sources_test/models/view_model.sql
+++ /dev/null
@@ -1,3 +0,0 @@
-{# See here: https://github.com/dbt-labs/dbt/pull/1729 #}
-
-select * from {{ ref('ephemeral_model') }}
diff --git a/tests/integration/sources_test/seed.sql b/tests/integration/sources_test/seed.sql
deleted file mode 100644
index 40110b990..000000000
--- a/tests/integration/sources_test/seed.sql
+++ /dev/null
@@ -1,113 +0,0 @@
-create table {schema}.seed_expected (
-    favorite_color TEXT,
-	id INTEGER,
-	first_name TEXT,
-	email TEXT,
-	ip_address TEXT,
-	updated_at TIMESTAMP WITHOUT TIME ZONE
-);
-
-
-INSERT INTO {schema}.seed_expected
-    ("favorite_color","id","first_name","email","ip_address","updated_at")
-VALUES
-    ('blue',1,'Larry','lking0@miitbeian.gov.cn','''69.135.206.194''','2008-09-12 19:08:31'),
-    ('blue',2,'Larry','lperkins1@toplist.cz','''64.210.133.162''','1978-05-09 04:15:14'),
-    ('blue',3,'Anna','amontgomery2@miitbeian.gov.cn','''168.104.64.114''','2011-10-16 04:07:57'),
-    ('blue',4,'Sandra','sgeorge3@livejournal.com','''229.235.252.98''','1973-07-19 10:52:43'),
-    ('blue',5,'Fred','fwoods4@google.cn','''78.229.170.124''','2012-09-30 16:38:29'),
-    ('blue',6,'Stephen','shanson5@livejournal.com','''182.227.157.105''','1995-11-07 21:40:50'),
-    ('blue',7,'William','wmartinez6@upenn.edu','''135.139.249.50''','1982-09-05 03:11:59'),
-    ('blue',8,'Jessica','jlong7@hao123.com','''203.62.178.210''','1991-10-16 11:03:15'),
-    ('blue',9,'Douglas','dwhite8@tamu.edu','''178.187.247.1''','1979-10-01 09:49:48'),
-    ('blue',10,'Lisa','lcoleman9@nydailynews.com','''168.234.128.249''','2011-05-26 07:45:49'),
-    ('blue',11,'Ralph','rfieldsa@home.pl','''55.152.163.149''','1972-11-18 19:06:11'),
-    ('blue',12,'Louise','lnicholsb@samsung.com','''141.116.153.154''','2014-11-25 20:56:14'),
-    ('blue',13,'Clarence','cduncanc@sfgate.com','''81.171.31.133''','2011-11-17 07:02:36'),
-    ('blue',14,'Daniel','dfranklind@omniture.com','''8.204.211.37''','1980-09-13 00:09:04'),
-    ('blue',15,'Katherine','klanee@auda.org.au','''176.96.134.59''','1997-08-22 19:36:56'),
-    ('blue',16,'Billy','bwardf@wikia.com','''214.108.78.85''','2003-10-19 02:14:47'),
-    ('blue',17,'Annie','agarzag@ocn.ne.jp','''190.108.42.70''','1988-10-28 15:12:35'),
-    ('blue',18,'Shirley','scolemanh@fastcompany.com','''109.251.164.84''','1988-08-24 10:50:57'),
-    ('blue',19,'Roger','rfrazieri@scribd.com','''38.145.218.108''','1985-12-31 15:17:15'),
-    ('blue',20,'Lillian','lstanleyj@goodreads.com','''47.57.236.17''','1970-06-08 02:09:05'),
-    ('blue',21,'Aaron','arodriguezk@nps.gov','''205.245.118.221''','1985-10-11 23:07:49'),
-    ('blue',22,'Patrick','pparkerl@techcrunch.com','''19.8.100.182''','2006-03-29 12:53:56'),
-    ('blue',23,'Phillip','pmorenom@intel.com','''41.38.254.103''','2011-11-07 15:35:43'),
-    ('blue',24,'Henry','hgarcian@newsvine.com','''1.191.216.252''','2008-08-28 08:30:44'),
-    ('blue',25,'Irene','iturnero@opera.com','''50.17.60.190''','1994-04-01 07:15:02'),
-    ('blue',26,'Andrew','adunnp@pen.io','''123.52.253.176''','2000-11-01 06:03:25'),
-    ('blue',27,'David','dgutierrezq@wp.com','''238.23.203.42''','1988-01-25 07:29:18'),
-    ('blue',28,'Henry','hsanchezr@cyberchimps.com','''248.102.2.185''','1983-01-01 13:36:37'),
-    ('blue',29,'Evelyn','epetersons@gizmodo.com','''32.80.46.119''','1979-07-16 17:24:12'),
-    ('blue',30,'Tammy','tmitchellt@purevolume.com','''249.246.167.88''','2001-04-03 10:00:23'),
-    ('blue',31,'Jacqueline','jlittleu@domainmarket.com','''127.181.97.47''','1986-02-11 21:35:50'),
-    ('blue',32,'Earl','eortizv@opera.com','''166.47.248.240''','1996-07-06 08:16:27'),
-    ('blue',33,'Juan','jgordonw@sciencedirect.com','''71.77.2.200''','1987-01-31 03:46:44'),
-    ('blue',34,'Diane','dhowellx@nyu.edu','''140.94.133.12''','1994-06-11 02:30:05'),
-    ('blue',35,'Randy','rkennedyy@microsoft.com','''73.255.34.196''','2005-05-26 20:28:39'),
-    ('blue',36,'Janice','jriveraz@time.com','''22.214.227.32''','1990-02-09 04:16:52'),
-    ('blue',37,'Laura','lperry10@diigo.com','''159.148.145.73''','2015-03-17 05:59:25'),
-    ('blue',38,'Gary','gray11@statcounter.com','''40.193.124.56''','1970-01-27 10:04:51'),
-    ('blue',39,'Jesse','jmcdonald12@typepad.com','''31.7.86.103''','2009-03-14 08:14:29'),
-    ('blue',40,'Sandra','sgonzalez13@goodreads.com','''223.80.168.239''','1993-05-21 14:08:54'),
-    ('blue',41,'Scott','smoore14@archive.org','''38.238.46.83''','1980-08-30 11:16:56'),
-    ('blue',42,'Phillip','pevans15@cisco.com','''158.234.59.34''','2011-12-15 23:26:31'),
-    ('blue',43,'Steven','sriley16@google.ca','''90.247.57.68''','2011-10-29 19:03:28'),
-    ('blue',44,'Deborah','dbrown17@hexun.com','''179.125.143.240''','1995-04-10 14:36:07'),
-    ('blue',45,'Lori','lross18@ow.ly','''64.80.162.180''','1980-12-27 16:49:15'),
-    ('blue',46,'Sean','sjackson19@tumblr.com','''240.116.183.69''','1988-06-12 21:24:45'),
-    ('blue',47,'Terry','tbarnes1a@163.com','''118.38.213.137''','1997-09-22 16:43:19'),
-    ('blue',48,'Dorothy','dross1b@ebay.com','''116.81.76.49''','2005-02-28 13:33:24'),
-    ('blue',49,'Samuel','swashington1c@house.gov','''38.191.253.40''','1989-01-19 21:15:48'),
-    ('blue',50,'Ralph','rcarter1d@tinyurl.com','''104.84.60.174''','2007-08-11 10:21:49'),
-    ('green',51,'Wayne','whudson1e@princeton.edu','''90.61.24.102''','1983-07-03 16:58:12'),
-    ('green',52,'Rose','rjames1f@plala.or.jp','''240.83.81.10''','1995-06-08 11:46:23'),
-    ('green',53,'Louise','lcox1g@theglobeandmail.com','''105.11.82.145''','2016-09-19 14:45:51'),
-    ('green',54,'Kenneth','kjohnson1h@independent.co.uk','''139.5.45.94''','1976-08-17 11:26:19'),
-    ('green',55,'Donna','dbrown1i@amazon.co.uk','''19.45.169.45''','2006-05-27 16:51:40'),
-    ('green',56,'Johnny','jvasquez1j@trellian.com','''118.202.238.23''','1975-11-17 08:42:32'),
-    ('green',57,'Patrick','pramirez1k@tamu.edu','''231.25.153.198''','1997-08-06 11:51:09'),
-    ('green',58,'Helen','hlarson1l@prweb.com','''8.40.21.39''','1993-08-04 19:53:40'),
-    ('green',59,'Patricia','pspencer1m@gmpg.org','''212.198.40.15''','1977-08-03 16:37:27'),
-    ('green',60,'Joseph','jspencer1n@marriott.com','''13.15.63.238''','2005-07-23 20:22:06'),
-    ('green',61,'Phillip','pschmidt1o@blogtalkradio.com','''177.98.201.190''','1976-05-19 21:47:44'),
-    ('green',62,'Joan','jwebb1p@google.ru','''105.229.170.71''','1972-09-07 17:53:47'),
-    ('green',63,'Phyllis','pkennedy1q@imgur.com','''35.145.8.244''','2000-01-01 22:33:37'),
-    ('green',64,'Katherine','khunter1r@smh.com.au','''248.168.205.32''','1991-01-09 06:40:24'),
-    ('green',65,'Laura','lvasquez1s@wiley.com','''128.129.115.152''','1997-10-23 12:04:56'),
-    ('green',66,'Juan','jdunn1t@state.gov','''44.228.124.51''','2004-11-10 05:07:35'),
-    ('green',67,'Judith','jholmes1u@wiley.com','''40.227.179.115''','1977-08-02 17:01:45'),
-    ('green',68,'Beverly','bbaker1v@wufoo.com','''208.34.84.59''','2016-03-06 20:07:23'),
-    ('green',69,'Lawrence','lcarr1w@flickr.com','''59.158.212.223''','1988-09-13 06:07:21'),
-    ('green',70,'Gloria','gwilliams1x@mtv.com','''245.231.88.33''','1995-03-18 22:32:46'),
-    ('green',71,'Steven','ssims1y@cbslocal.com','''104.50.58.255''','2001-08-05 21:26:20'),
-    ('green',72,'Betty','bmills1z@arstechnica.com','''103.177.214.220''','1981-12-14 21:26:54'),
-    ('green',73,'Mildred','mfuller20@prnewswire.com','''151.158.8.130''','2000-04-19 10:13:55'),
-    ('green',74,'Donald','dday21@icq.com','''9.178.102.255''','1972-12-03 00:58:24'),
-    ('green',75,'Eric','ethomas22@addtoany.com','''85.2.241.227''','1992-11-01 05:59:30'),
-    ('green',76,'Joyce','jarmstrong23@sitemeter.com','''169.224.20.36''','1985-10-24 06:50:01'),
-    ('green',77,'Maria','mmartinez24@amazonaws.com','''143.189.167.135''','2005-10-05 05:17:42'),
-    ('green',78,'Harry','hburton25@youtube.com','''156.47.176.237''','1978-03-26 05:53:33'),
-    ('green',79,'Kevin','klawrence26@hao123.com','''79.136.183.83''','1994-10-12 04:38:52'),
-    ('green',80,'David','dhall27@prweb.com','''133.149.172.153''','1976-12-15 16:24:24'),
-    ('green',81,'Kathy','kperry28@twitter.com','''229.242.72.228''','1979-03-04 02:58:56'),
-    ('green',82,'Adam','aprice29@elegantthemes.com','''13.145.21.10''','1982-11-07 11:46:59'),
-    ('green',83,'Brandon','bgriffin2a@va.gov','''73.249.128.212''','2013-10-30 05:30:36'),
-    ('green',84,'Henry','hnguyen2b@discovery.com','''211.36.214.242''','1985-01-09 06:37:27'),
-    ('green',85,'Eric','esanchez2c@edublogs.org','''191.166.188.251''','2004-05-01 23:21:42'),
-    ('green',86,'Jason','jlee2d@jimdo.com','''193.92.16.182''','1973-01-08 09:05:39'),
-    ('green',87,'Diana','drichards2e@istockphoto.com','''19.130.175.245''','1994-10-05 22:50:49'),
-    ('green',88,'Andrea','awelch2f@abc.net.au','''94.155.233.96''','2002-04-26 08:41:44'),
-    ('green',89,'Louis','lwagner2g@miitbeian.gov.cn','''26.217.34.111''','2003-08-25 07:56:39'),
-    ('green',90,'Jane','jsims2h@seesaa.net','''43.4.220.135''','1987-03-20 20:39:04'),
-    ('green',91,'Larry','lgrant2i@si.edu','''97.126.79.34''','2000-09-07 20:26:19'),
-    ('green',92,'Louis','ldean2j@prnewswire.com','''37.148.40.127''','2011-09-16 20:12:14'),
-    ('green',93,'Jennifer','jcampbell2k@xing.com','''38.106.254.142''','1988-07-15 05:06:49'),
-    ('green',94,'Wayne','wcunningham2l@google.com.hk','''223.28.26.187''','2009-12-15 06:16:54'),
-    ('green',95,'Lori','lstevens2m@icq.com','''181.250.181.58''','1984-10-28 03:29:19'),
-    ('green',96,'Judy','jsimpson2n@marriott.com','''180.121.239.219''','1986-02-07 15:18:10'),
-    ('green',97,'Phillip','phoward2o@usa.gov','''255.247.0.175''','2002-12-26 08:44:45'),
-    ('green',98,'Gloria','gwalker2p@usa.gov','''156.140.7.128''','1997-10-04 07:58:58'),
-    ('green',99,'Paul','pjohnson2q@umn.edu','''183.59.198.197''','1991-11-14 12:33:55'),
-    ('green',100,'Frank','fgreene2r@blogspot.com','''150.143.68.121''','2010-06-12 23:55:39');
diff --git a/tests/integration/sources_test/seeds/expected_multi_source.csv b/tests/integration/sources_test/seeds/expected_multi_source.csv
deleted file mode 100644
index de9c1c01d..000000000
--- a/tests/integration/sources_test/seeds/expected_multi_source.csv
+++ /dev/null
@@ -1,4 +0,0 @@
-id,first_name,color
-1,Larry,blue
-2,Curly,red
-3,Moe,green
diff --git a/tests/integration/sources_test/seeds/other_source_table.csv b/tests/integration/sources_test/seeds/other_source_table.csv
deleted file mode 100644
index a92b2cb8e..000000000
--- a/tests/integration/sources_test/seeds/other_source_table.csv
+++ /dev/null
@@ -1,4 +0,0 @@
-id,color
-1,blue
-2,red
-3,green
diff --git a/tests/integration/sources_test/seeds/other_table.csv b/tests/integration/sources_test/seeds/other_table.csv
deleted file mode 100644
index 56bdda92b..000000000
--- a/tests/integration/sources_test/seeds/other_table.csv
+++ /dev/null
@@ -1,4 +0,0 @@
-id,first_name
-1,Larry
-2,Curly
-3,Moe
diff --git a/tests/integration/sources_test/seeds/source.csv b/tests/integration/sources_test/seeds/source.csv
deleted file mode 100644
index a8f87412e..000000000
--- a/tests/integration/sources_test/seeds/source.csv
+++ /dev/null
@@ -1,101 +0,0 @@
-favorite_color,id,first_name,email,ip_address,updated_at
-blue,1,Larry,lking0@miitbeian.gov.cn,'69.135.206.194',2008-09-12 19:08:31
-blue,2,Larry,lperkins1@toplist.cz,'64.210.133.162',1978-05-09 04:15:14
-blue,3,Anna,amontgomery2@miitbeian.gov.cn,'168.104.64.114',2011-10-16 04:07:57
-blue,4,Sandra,sgeorge3@livejournal.com,'229.235.252.98',1973-07-19 10:52:43
-blue,5,Fred,fwoods4@google.cn,'78.229.170.124',2012-09-30 16:38:29
-blue,6,Stephen,shanson5@livejournal.com,'182.227.157.105',1995-11-07 21:40:50
-blue,7,William,wmartinez6@upenn.edu,'135.139.249.50',1982-09-05 03:11:59
-blue,8,Jessica,jlong7@hao123.com,'203.62.178.210',1991-10-16 11:03:15
-blue,9,Douglas,dwhite8@tamu.edu,'178.187.247.1',1979-10-01 09:49:48
-blue,10,Lisa,lcoleman9@nydailynews.com,'168.234.128.249',2011-05-26 07:45:49
-blue,11,Ralph,rfieldsa@home.pl,'55.152.163.149',1972-11-18 19:06:11
-blue,12,Louise,lnicholsb@samsung.com,'141.116.153.154',2014-11-25 20:56:14
-blue,13,Clarence,cduncanc@sfgate.com,'81.171.31.133',2011-11-17 07:02:36
-blue,14,Daniel,dfranklind@omniture.com,'8.204.211.37',1980-09-13 00:09:04
-blue,15,Katherine,klanee@auda.org.au,'176.96.134.59',1997-08-22 19:36:56
-blue,16,Billy,bwardf@wikia.com,'214.108.78.85',2003-10-19 02:14:47
-blue,17,Annie,agarzag@ocn.ne.jp,'190.108.42.70',1988-10-28 15:12:35
-blue,18,Shirley,scolemanh@fastcompany.com,'109.251.164.84',1988-08-24 10:50:57
-blue,19,Roger,rfrazieri@scribd.com,'38.145.218.108',1985-12-31 15:17:15
-blue,20,Lillian,lstanleyj@goodreads.com,'47.57.236.17',1970-06-08 02:09:05
-blue,21,Aaron,arodriguezk@nps.gov,'205.245.118.221',1985-10-11 23:07:49
-blue,22,Patrick,pparkerl@techcrunch.com,'19.8.100.182',2006-03-29 12:53:56
-blue,23,Phillip,pmorenom@intel.com,'41.38.254.103',2011-11-07 15:35:43
-blue,24,Henry,hgarcian@newsvine.com,'1.191.216.252',2008-08-28 08:30:44
-blue,25,Irene,iturnero@opera.com,'50.17.60.190',1994-04-01 07:15:02
-blue,26,Andrew,adunnp@pen.io,'123.52.253.176',2000-11-01 06:03:25
-blue,27,David,dgutierrezq@wp.com,'238.23.203.42',1988-01-25 07:29:18
-blue,28,Henry,hsanchezr@cyberchimps.com,'248.102.2.185',1983-01-01 13:36:37
-blue,29,Evelyn,epetersons@gizmodo.com,'32.80.46.119',1979-07-16 17:24:12
-blue,30,Tammy,tmitchellt@purevolume.com,'249.246.167.88',2001-04-03 10:00:23
-blue,31,Jacqueline,jlittleu@domainmarket.com,'127.181.97.47',1986-02-11 21:35:50
-blue,32,Earl,eortizv@opera.com,'166.47.248.240',1996-07-06 08:16:27
-blue,33,Juan,jgordonw@sciencedirect.com,'71.77.2.200',1987-01-31 03:46:44
-blue,34,Diane,dhowellx@nyu.edu,'140.94.133.12',1994-06-11 02:30:05
-blue,35,Randy,rkennedyy@microsoft.com,'73.255.34.196',2005-05-26 20:28:39
-blue,36,Janice,jriveraz@time.com,'22.214.227.32',1990-02-09 04:16:52
-blue,37,Laura,lperry10@diigo.com,'159.148.145.73',2015-03-17 05:59:25
-blue,38,Gary,gray11@statcounter.com,'40.193.124.56',1970-01-27 10:04:51
-blue,39,Jesse,jmcdonald12@typepad.com,'31.7.86.103',2009-03-14 08:14:29
-blue,40,Sandra,sgonzalez13@goodreads.com,'223.80.168.239',1993-05-21 14:08:54
-blue,41,Scott,smoore14@archive.org,'38.238.46.83',1980-08-30 11:16:56
-blue,42,Phillip,pevans15@cisco.com,'158.234.59.34',2011-12-15 23:26:31
-blue,43,Steven,sriley16@google.ca,'90.247.57.68',2011-10-29 19:03:28
-blue,44,Deborah,dbrown17@hexun.com,'179.125.143.240',1995-04-10 14:36:07
-blue,45,Lori,lross18@ow.ly,'64.80.162.180',1980-12-27 16:49:15
-blue,46,Sean,sjackson19@tumblr.com,'240.116.183.69',1988-06-12 21:24:45
-blue,47,Terry,tbarnes1a@163.com,'118.38.213.137',1997-09-22 16:43:19
-blue,48,Dorothy,dross1b@ebay.com,'116.81.76.49',2005-02-28 13:33:24
-blue,49,Samuel,swashington1c@house.gov,'38.191.253.40',1989-01-19 21:15:48
-blue,50,Ralph,rcarter1d@tinyurl.com,'104.84.60.174',2007-08-11 10:21:49
-green,51,Wayne,whudson1e@princeton.edu,'90.61.24.102',1983-07-03 16:58:12
-green,52,Rose,rjames1f@plala.or.jp,'240.83.81.10',1995-06-08 11:46:23
-green,53,Louise,lcox1g@theglobeandmail.com,'105.11.82.145',2016-09-19 14:45:51
-green,54,Kenneth,kjohnson1h@independent.co.uk,'139.5.45.94',1976-08-17 11:26:19
-green,55,Donna,dbrown1i@amazon.co.uk,'19.45.169.45',2006-05-27 16:51:40
-green,56,Johnny,jvasquez1j@trellian.com,'118.202.238.23',1975-11-17 08:42:32
-green,57,Patrick,pramirez1k@tamu.edu,'231.25.153.198',1997-08-06 11:51:09
-green,58,Helen,hlarson1l@prweb.com,'8.40.21.39',1993-08-04 19:53:40
-green,59,Patricia,pspencer1m@gmpg.org,'212.198.40.15',1977-08-03 16:37:27
-green,60,Joseph,jspencer1n@marriott.com,'13.15.63.238',2005-07-23 20:22:06
-green,61,Phillip,pschmidt1o@blogtalkradio.com,'177.98.201.190',1976-05-19 21:47:44
-green,62,Joan,jwebb1p@google.ru,'105.229.170.71',1972-09-07 17:53:47
-green,63,Phyllis,pkennedy1q@imgur.com,'35.145.8.244',2000-01-01 22:33:37
-green,64,Katherine,khunter1r@smh.com.au,'248.168.205.32',1991-01-09 06:40:24
-green,65,Laura,lvasquez1s@wiley.com,'128.129.115.152',1997-10-23 12:04:56
-green,66,Juan,jdunn1t@state.gov,'44.228.124.51',2004-11-10 05:07:35
-green,67,Judith,jholmes1u@wiley.com,'40.227.179.115',1977-08-02 17:01:45
-green,68,Beverly,bbaker1v@wufoo.com,'208.34.84.59',2016-03-06 20:07:23
-green,69,Lawrence,lcarr1w@flickr.com,'59.158.212.223',1988-09-13 06:07:21
-green,70,Gloria,gwilliams1x@mtv.com,'245.231.88.33',1995-03-18 22:32:46
-green,71,Steven,ssims1y@cbslocal.com,'104.50.58.255',2001-08-05 21:26:20
-green,72,Betty,bmills1z@arstechnica.com,'103.177.214.220',1981-12-14 21:26:54
-green,73,Mildred,mfuller20@prnewswire.com,'151.158.8.130',2000-04-19 10:13:55
-green,74,Donald,dday21@icq.com,'9.178.102.255',1972-12-03 00:58:24
-green,75,Eric,ethomas22@addtoany.com,'85.2.241.227',1992-11-01 05:59:30
-green,76,Joyce,jarmstrong23@sitemeter.com,'169.224.20.36',1985-10-24 06:50:01
-green,77,Maria,mmartinez24@amazonaws.com,'143.189.167.135',2005-10-05 05:17:42
-green,78,Harry,hburton25@youtube.com,'156.47.176.237',1978-03-26 05:53:33
-green,79,Kevin,klawrence26@hao123.com,'79.136.183.83',1994-10-12 04:38:52
-green,80,David,dhall27@prweb.com,'133.149.172.153',1976-12-15 16:24:24
-green,81,Kathy,kperry28@twitter.com,'229.242.72.228',1979-03-04 02:58:56
-green,82,Adam,aprice29@elegantthemes.com,'13.145.21.10',1982-11-07 11:46:59
-green,83,Brandon,bgriffin2a@va.gov,'73.249.128.212',2013-10-30 05:30:36
-green,84,Henry,hnguyen2b@discovery.com,'211.36.214.242',1985-01-09 06:37:27
-green,85,Eric,esanchez2c@edublogs.org,'191.166.188.251',2004-05-01 23:21:42
-green,86,Jason,jlee2d@jimdo.com,'193.92.16.182',1973-01-08 09:05:39
-green,87,Diana,drichards2e@istockphoto.com,'19.130.175.245',1994-10-05 22:50:49
-green,88,Andrea,awelch2f@abc.net.au,'94.155.233.96',2002-04-26 08:41:44
-green,89,Louis,lwagner2g@miitbeian.gov.cn,'26.217.34.111',2003-08-25 07:56:39
-green,90,Jane,jsims2h@seesaa.net,'43.4.220.135',1987-03-20 20:39:04
-green,91,Larry,lgrant2i@si.edu,'97.126.79.34',2000-09-07 20:26:19
-green,92,Louis,ldean2j@prnewswire.com,'37.148.40.127',2011-09-16 20:12:14
-green,93,Jennifer,jcampbell2k@xing.com,'38.106.254.142',1988-07-15 05:06:49
-green,94,Wayne,wcunningham2l@google.com.hk,'223.28.26.187',2009-12-15 06:16:54
-green,95,Lori,lstevens2m@icq.com,'181.250.181.58',1984-10-28 03:29:19
-green,96,Judy,jsimpson2n@marriott.com,'180.121.239.219',1986-02-07 15:18:10
-green,97,Phillip,phoward2o@usa.gov,'255.247.0.175',2002-12-26 08:44:45
-green,98,Gloria,gwalker2p@usa.gov,'156.140.7.128',1997-10-04 07:58:58
-green,99,Paul,pjohnson2q@umn.edu,'183.59.198.197',1991-11-14 12:33:55
-green,100,Frank,fgreene2r@blogspot.com,'150.143.68.121',2010-06-12 23:55:39
diff --git a/tests/integration/sources_test/test_sources.py b/tests/integration/sources_test/test_sources.py
deleted file mode 100644
index 83a586434..000000000
--- a/tests/integration/sources_test/test_sources.py
+++ /dev/null
@@ -1,183 +0,0 @@
-import json
-import os
-from datetime import datetime, timedelta
-
-import yaml
-
-import dbt.tracking
-import dbt.version
-from dbt.events.functions import reset_metadata_vars
-from tests.integration.base import DBTIntegrationTest, use_profile, AnyFloat, \
-    AnyStringWith
-
-
-class BaseSourcesTest(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "sources"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds'],
-            'quoting': {'database': True, 'schema': True, 'identifier': True},
-            'seeds': {
-                'quote_columns': True,
-            },
-        }
-
-    def setUp(self):
-        super().setUp()
-        os.environ['DBT_TEST_SCHEMA_NAME_VARIABLE'] = 'test_run_schema'
-
-    def tearDown(self):
-        del os.environ['DBT_TEST_SCHEMA_NAME_VARIABLE']
-        super().tearDown()
-
-    def run_dbt_with_vars(self, cmd, *args, **kwargs):
-        vars_dict = {
-            'test_run_schema': self.unique_schema(),
-            'test_loaded_at': self.adapter.quote('updated_at'),
-        }
-        cmd.extend(['--vars', yaml.safe_dump(vars_dict)])
-        return self.run_dbt(cmd, *args, **kwargs)
-
-
-class TestSourceFreshness(BaseSourcesTest):
-    def setUp(self):
-        super().setUp()
-        self.run_dbt_with_vars(['seed'])
-        self.maxDiff = None
-        self._id = 101
-        # this is the db initial value
-        self.last_inserted_time = "2016-09-19T14:45:51+00:00"
-        os.environ['DBT_ENV_CUSTOM_ENV_key'] = 'value'
-
-    def tearDown(self):
-        super().tearDown()
-        reset_metadata_vars()
-        del os.environ['DBT_ENV_CUSTOM_ENV_key']
-
-    def _set_updated_at_to(self, delta):
-        insert_time = datetime.utcnow() + delta
-        timestr = insert_time.strftime("%Y-%m-%d %H:%M:%S")
-        # favorite_color,id,first_name,email,ip_address,updated_at
-        insert_id = self._id
-        self._id += 1
-        raw_sql = """INSERT INTO {schema}.{source}
-            ({quoted_columns})
-        VALUES (
-            'blue',{id},'Jake','abc@example.com','192.168.1.1','{time}'
-        )"""
-        quoted_columns = ','.join(
-            c for c in
-            ('favorite_color', 'id', 'first_name',
-             'email', 'ip_address', 'updated_at')
-        )
-        self.run_sql(
-            raw_sql,
-            kwargs={
-                'schema': self.unique_schema(),
-                'time': timestr,
-                'id': insert_id,
-                'source': self.adapter.quote('source'),
-                'quoted_columns': quoted_columns,
-            }
-        )
-        self.last_inserted_time = insert_time.strftime(
-            "%Y-%m-%dT%H:%M:%S+00:00")
-
-    def _assert_freshness_results(self, path, state):
-        self.assertTrue(os.path.exists(path))
-        with open(path) as fp:
-            data = json.load(fp)
-
-        assert set(data) == {'metadata', 'results', 'elapsed_time'}
-        assert 'generated_at' in data['metadata']
-        assert isinstance(data['elapsed_time'], float)
-        self.assertBetween(data['metadata']['generated_at'],
-                           self.freshness_start_time)
-        assert data['metadata']['dbt_schema_version'] == 'https://schemas.getdbt.com/dbt/sources/v3.json'
-        assert data['metadata']['dbt_version'] == dbt.version.__version__
-        assert data['metadata']['invocation_id'] == dbt.tracking.active_user.invocation_id
-        key = 'key'
-        if os.name == 'nt':
-            key = key.upper()
-        assert data['metadata']['env'] == {key: 'value'}
-
-        last_inserted_time = self.last_inserted_time
-
-        self.assertEqual(len(data['results']), 1)
-
-        self.assertEqual(data['results'], [
-            {
-                'unique_id': 'source.test.test_source.test_table',
-                'max_loaded_at': last_inserted_time,
-                'snapshotted_at': AnyStringWith(),
-                'max_loaded_at_time_ago_in_s': AnyFloat(),
-                'status': state,
-                'criteria': {
-                    'filter': None,
-                    'warn_after': {'count': 10, 'period': 'hour'},
-                    'error_after': {'count': 18, 'period': 'hour'},
-                },
-                'adapter_response': {},
-                'thread_id': AnyStringWith('Thread-'),
-                'execution_time': AnyFloat(),
-                'timing': [
-                    {
-                        'name': 'compile',
-                        'started_at': AnyStringWith(),
-                        'completed_at': AnyStringWith(),
-                    },
-                    {
-                        'name': 'execute',
-                        'started_at': AnyStringWith(),
-                        'completed_at': AnyStringWith(),
-                    }
-                ]
-            }
-        ])
-
-    def _run_source_freshness(self):
-        # test_source.test_table should have a loaded_at field of `updated_at`
-        # and a freshness of warn_after: 10 hours, error_after: 18 hours
-        # by default, our data set is way out of date!
-        self.freshness_start_time = datetime.utcnow()
-        reset_metadata_vars()
-        results = self.run_dbt_with_vars(
-            ['source', 'freshness', '-o', 'target/error_source.json'],
-            expect_pass=False
-        )
-        self.assertEqual(len(results), 1)
-        self.assertEqual(results[0].status, 'error')
-        self._assert_freshness_results('target/error_source.json', 'error')
-
-        self._set_updated_at_to(timedelta(hours=-12))
-        self.freshness_start_time = datetime.utcnow()
-        reset_metadata_vars()
-        results = self.run_dbt_with_vars(
-            ['source', 'freshness', '-o', 'target/warn_source.json'],
-        )
-        self.assertEqual(len(results), 1)
-        self.assertEqual(results[0].status, 'warn')
-        self._assert_freshness_results('target/warn_source.json', 'warn')
-
-        self._set_updated_at_to(timedelta(hours=-2))
-        self.freshness_start_time = datetime.utcnow()
-        reset_metadata_vars()
-        results = self.run_dbt_with_vars(
-            ['source', 'freshness', '-o', 'target/pass_source.json'],
-        )
-        self.assertEqual(len(results), 1)
-        self.assertEqual(results[0].status, 'pass')
-        self._assert_freshness_results('target/pass_source.json', 'pass')
-
-    @use_profile('bigquery')
-    def test_bigquery_source_freshness(self):
-        self._run_source_freshness()

From 368742d8bb91b6a8a1d57470945a9d8bcabdacf8 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 26 Jan 2023 15:58:27 -0600
Subject: [PATCH 533/860] [CT-1630] Convert column_types test for dbt-bigquery
 (#476)

* init conversion of columns_types test for bigquery, removal of old test, dev_requirements pin changed

* move bigquery specific stuff to fixture file to minimize contents of tests

* revert back to having test specific fixtuers part of test file as they are not overly large additions

* reset dev-requirement pointer

* move all fixtures to one single file
---
 .../adapter/column_types/fixtures.py          | 49 +++++++++++++
 .../column_types/test_alter_column_types.py   | 48 +++++++++++++
 .../adapter/column_types/test_column_types.py | 15 ++++
 .../column_type_tests/bq_models/model.sql     |  5 --
 .../column_type_tests/bq_models/schema.yml    | 10 ---
 .../bq_models_alter_type/altered_schema.yml   | 10 ---
 .../bq_models_alter_type/model.sql            |  6 --
 .../macros/test_alter_column_type.sql         |  5 --
 .../column_type_tests/macros/test_is_type.sql | 72 -------------------
 .../test_alter_column_types.py                | 28 --------
 .../column_type_tests/test_column_types.py    | 21 ------
 11 files changed, 112 insertions(+), 157 deletions(-)
 create mode 100644 tests/functional/adapter/column_types/fixtures.py
 create mode 100644 tests/functional/adapter/column_types/test_alter_column_types.py
 create mode 100644 tests/functional/adapter/column_types/test_column_types.py
 delete mode 100644 tests/integration/column_type_tests/bq_models/model.sql
 delete mode 100644 tests/integration/column_type_tests/bq_models/schema.yml
 delete mode 100644 tests/integration/column_type_tests/bq_models_alter_type/altered_schema.yml
 delete mode 100644 tests/integration/column_type_tests/bq_models_alter_type/model.sql
 delete mode 100644 tests/integration/column_type_tests/macros/test_alter_column_type.sql
 delete mode 100644 tests/integration/column_type_tests/macros/test_is_type.sql
 delete mode 100644 tests/integration/column_type_tests/test_alter_column_types.py
 delete mode 100644 tests/integration/column_type_tests/test_column_types.py

diff --git a/tests/functional/adapter/column_types/fixtures.py b/tests/functional/adapter/column_types/fixtures.py
new file mode 100644
index 000000000..b7be1e646
--- /dev/null
+++ b/tests/functional/adapter/column_types/fixtures.py
@@ -0,0 +1,49 @@
+_MACRO_TEST_ALTER_COLUMN_TYPE = """
+{% macro test_alter_column_type(model_name, column_name, new_column_type) %}
+  {% set relation = ref(model_name) %}
+  {{ alter_column_type(relation, column_name, new_column_type) }}
+{% endmacro %}
+"""
+
+_MODEL_SQL = """
+select
+    CAST(1 as int64) as int64_col,
+    CAST(2.0 as float64) as float64_col,
+    CAST(3.0 as numeric) as numeric_col,
+    CAST('3' as string) as string_col,
+"""
+
+_MODEL_ALT_SQL = """
+{{ config(materialized='table') }}
+select
+    CAST(1 as int64) as int64_col,
+    CAST(2.0 as float64) as float64_col,
+    CAST(3.0 as numeric) as numeric_col,
+    CAST('3' as string) as string_col,
+"""
+
+_SCHEMA_YML = """
+version: 2
+models:
+  - name: model
+    tests:
+      - is_type:
+          column_map:
+            int64_col: ['integer', 'number']
+            float64_col: ['float', 'number']
+            numeric_col: ['numeric', 'number']
+            string_col: ['string', 'not number']
+"""
+
+_ALT_SCHEMA_YML = """
+version: 2
+models:
+  - name: model
+    tests:
+      - is_type:
+          column_map:
+            int64_col: ['string', 'not number']
+            float64_col: ['float', 'number']
+            numeric_col: ['numeric', 'number']
+            string_col: ['string', 'not number']
+"""
diff --git a/tests/functional/adapter/column_types/test_alter_column_types.py b/tests/functional/adapter/column_types/test_alter_column_types.py
new file mode 100644
index 000000000..9bfb88dc1
--- /dev/null
+++ b/tests/functional/adapter/column_types/test_alter_column_types.py
@@ -0,0 +1,48 @@
+import pytest
+import yaml
+from dbt.tests.util import run_dbt
+from dbt.tests.adapter.column_types.test_column_types import BaseColumnTypes
+from dbt.tests.adapter.column_types.fixtures import macro_test_is_type_sql
+from tests.functional.adapter.column_types.fixtures import (
+    _MACRO_TEST_ALTER_COLUMN_TYPE,
+    _MODEL_ALT_SQL,
+    _ALT_SCHEMA_YML
+)
+
+
+
+class BaseAlterColumnTypes(BaseColumnTypes):
+
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {
+            "test_is_type.sql": macro_test_is_type_sql,
+            "test_alter_column_type.sql": _MACRO_TEST_ALTER_COLUMN_TYPE
+        }
+
+    def run_and_alter_and_test(self, alter_column_type_args):
+        results = run_dbt(["run"])
+        assert len(results) == 1
+        run_dbt(['run-operation', 'test_alter_column_type', '--args', alter_column_type_args])
+        results = run_dbt(["test"])
+        assert len(results) == 1
+
+
+
+class TestBigQueryAlterColumnTypes(BaseAlterColumnTypes):
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "model.sql": _MODEL_ALT_SQL,
+            "schema.yml": _ALT_SCHEMA_YML
+        }
+
+    def test_bigquery_alter_column_types(self, project):
+        alter_column_type_args = yaml.safe_dump({
+            'model_name': 'model',
+            'column_name': 'int64_col',
+            'new_column_type': 'string'
+        })
+
+        self.run_and_alter_and_test(alter_column_type_args)
\ No newline at end of file
diff --git a/tests/functional/adapter/column_types/test_column_types.py b/tests/functional/adapter/column_types/test_column_types.py
new file mode 100644
index 000000000..8f7ee4351
--- /dev/null
+++ b/tests/functional/adapter/column_types/test_column_types.py
@@ -0,0 +1,15 @@
+import pytest
+from dbt.tests.adapter.column_types.test_column_types import BaseColumnTypes
+from tests.functional.adapter.column_types.fixtures import _MODEL_SQL, _SCHEMA_YML
+
+class TestBigQueryColumnTypes(BaseColumnTypes):
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "model.sql": _MODEL_SQL,
+            "schema.yml": _SCHEMA_YML
+        }
+
+    def test_run_and_test(self, project):
+        self.run_and_test()
\ No newline at end of file
diff --git a/tests/integration/column_type_tests/bq_models/model.sql b/tests/integration/column_type_tests/bq_models/model.sql
deleted file mode 100644
index 94e4fba18..000000000
--- a/tests/integration/column_type_tests/bq_models/model.sql
+++ /dev/null
@@ -1,5 +0,0 @@
-select
-    CAST(1 as int64) as int64_col,
-    CAST(2.0 as float64) as float64_col,
-    CAST(3.0 as numeric) as numeric_col,
-    CAST('3' as string) as string_col,
diff --git a/tests/integration/column_type_tests/bq_models/schema.yml b/tests/integration/column_type_tests/bq_models/schema.yml
deleted file mode 100644
index 8eb8a9ae2..000000000
--- a/tests/integration/column_type_tests/bq_models/schema.yml
+++ /dev/null
@@ -1,10 +0,0 @@
-version: 2
-models:
-  - name: model
-    tests:
-      - is_type:
-          column_map:
-            int64_col: ['integer', 'number']
-            float64_col: ['float', 'number']
-            numeric_col: ['numeric', 'number']
-            string_col: ['string', 'not number']
diff --git a/tests/integration/column_type_tests/bq_models_alter_type/altered_schema.yml b/tests/integration/column_type_tests/bq_models_alter_type/altered_schema.yml
deleted file mode 100644
index 3a78da404..000000000
--- a/tests/integration/column_type_tests/bq_models_alter_type/altered_schema.yml
+++ /dev/null
@@ -1,10 +0,0 @@
-version: 2
-models:
-  - name: model
-    tests:
-      - is_type:
-          column_map:
-            int64_col: ['string', 'not number']
-            float64_col: ['float', 'number']
-            numeric_col: ['numeric', 'number']
-            string_col: ['string', 'not number']
diff --git a/tests/integration/column_type_tests/bq_models_alter_type/model.sql b/tests/integration/column_type_tests/bq_models_alter_type/model.sql
deleted file mode 100644
index 066252ae5..000000000
--- a/tests/integration/column_type_tests/bq_models_alter_type/model.sql
+++ /dev/null
@@ -1,6 +0,0 @@
-{{ config(materialized='table') }}
-select
-    CAST(1 as int64) as int64_col,
-    CAST(2.0 as float64) as float64_col,
-    CAST(3.0 as numeric) as numeric_col,
-    CAST('3' as string) as string_col,
diff --git a/tests/integration/column_type_tests/macros/test_alter_column_type.sql b/tests/integration/column_type_tests/macros/test_alter_column_type.sql
deleted file mode 100644
index 133d59fad..000000000
--- a/tests/integration/column_type_tests/macros/test_alter_column_type.sql
+++ /dev/null
@@ -1,5 +0,0 @@
--- Macro to alter a column type
-{% macro test_alter_column_type(model_name, column_name, new_column_type) %}
-  {% set relation = ref(model_name) %}
-  {{ alter_column_type(relation, column_name, new_column_type) }}
-{% endmacro %}
diff --git a/tests/integration/column_type_tests/macros/test_is_type.sql b/tests/integration/column_type_tests/macros/test_is_type.sql
deleted file mode 100644
index 2f1ffde2b..000000000
--- a/tests/integration/column_type_tests/macros/test_is_type.sql
+++ /dev/null
@@ -1,72 +0,0 @@
-
-{% macro simple_type_check_column(column, check) %}
-    {% if check == 'string' %}
-        {{ return(column.is_string()) }}
-    {% elif check == 'float' %}
-        {{ return(column.is_float()) }}
-    {% elif check == 'number' %}
-        {{ return(column.is_number()) }}
-    {% elif check == 'numeric' %}
-        {{ return(column.is_numeric()) }}
-    {% elif check == 'integer' %}
-        {{ return(column.is_integer()) }}
-    {% else %}
-        {% do exceptions.raise_compiler_error('invalid type check value: ' ~ check) %}
-    {% endif %}
-{% endmacro %}
-
-{% macro type_check_column(column, type_checks) %}
-    {% set failures = [] %}
-    {% for type_check in type_checks %}
-        {% if type_check.startswith('not ') %}
-            {% if simple_type_check_column(column, type_check[4:]) %}
-                {% do log('simple_type_check_column got ', True) %}
-                {% do failures.append(type_check) %}
-            {% endif %}
-        {% else %}
-            {% if not simple_type_check_column(column, type_check) %}
-                {% do failures.append(type_check) %}
-            {% endif %}
-        {% endif %}
-    {% endfor %}
-    {% if (failures | length) > 0 %}
-        {% do log('column ' ~ column.name ~ ' had failures: ' ~ failures, info=True) %}
-    {% endif %}
-    {% do return((failures | length) == 0) %}
-{% endmacro %}
-
-{% test is_type(model, column_map) %}
-    {% if not execute %}
-        {{ return(None) }}
-    {% endif %}
-    {% if not column_map %}
-        {% do exceptions.raise_compiler_error('test_is_type must have a column name') %}
-    {% endif %}
-    {% set columns = adapter.get_columns_in_relation(model) %}
-    {% if (column_map | length) != (columns | length) %}
-        {% set column_map_keys = (column_map | list | string) %}
-        {% set column_names = (columns | map(attribute='name') | list | string) %}
-        {% do exceptions.raise_compiler_error('did not get all the columns/all columns not specified:\n' ~ column_map_keys ~ '\nvs\n' ~ column_names) %}
-    {% endif %}
-    {% set bad_columns = [] %}
-    {% for column in columns %}
-        {% set column_key = (column.name | lower) %}
-        {% if column_key in column_map %}
-            {% set type_checks = column_map[column_key] %}
-            {% if not type_checks %}
-                {% do exceptions.raise_compiler_error('no type checks?') %}
-            {% endif %}
-            {% if not type_check_column(column, type_checks) %}
-                {% do bad_columns.append(column.name) %}
-            {% endif %}
-        {% else %}
-            {% do exceptions.raise_compiler_error('column key ' ~ column_key ~ ' not found in ' ~ (column_map | list | string)) %}
-        {% endif %}
-    {% endfor %}
-    {% do log('bad columns: ' ~ bad_columns, info=True) %}
-    {% for bad_column in bad_columns %}
-      select '{{ bad_column }}' as bad_column
-      {{ 'union all' if not loop.last }}
-    {% endfor %}
-      select * from (select 1 limit 0) as nothing
-{% endtest %}
diff --git a/tests/integration/column_type_tests/test_alter_column_types.py b/tests/integration/column_type_tests/test_alter_column_types.py
deleted file mode 100644
index 60bda6df1..000000000
--- a/tests/integration/column_type_tests/test_alter_column_types.py
+++ /dev/null
@@ -1,28 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-import yaml
-
-
-class TestAlterColumnTypes(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return 'alter_column_types'
-
-    def run_and_alter_and_test(self, alter_column_type_args):
-        self.assertEqual(len(self.run_dbt(['run'])), 1)
-        self.run_dbt(['run-operation', 'test_alter_column_type', '--args', alter_column_type_args])
-        self.assertEqual(len(self.run_dbt(['test'])), 1)
-
-
-class TestBigQueryAlterColumnTypes(TestAlterColumnTypes):
-    @property
-    def models(self):
-        return 'bq_models_alter_type'
-
-    @use_profile('bigquery')
-    def test_bigquery_column_types(self):
-        alter_column_type_args = yaml.safe_dump({
-            'model_name': 'model',
-            'column_name': 'int64_col',
-            'new_column_type': 'string'
-        })
-        self.run_and_alter_and_test(alter_column_type_args)
diff --git a/tests/integration/column_type_tests/test_column_types.py b/tests/integration/column_type_tests/test_column_types.py
deleted file mode 100644
index ab04cc8f3..000000000
--- a/tests/integration/column_type_tests/test_column_types.py
+++ /dev/null
@@ -1,21 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-
-class TestColumnTypes(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return 'column_types'
-
-    def run_and_test(self):
-        self.assertEqual(len(self.run_dbt(['run'])), 1)
-        self.assertEqual(len(self.run_dbt(['test'])), 1)
-
-
-class TestBigQueryColumnTypes(TestColumnTypes):
-    @property
-    def models(self):
-        return 'bq_models'
-
-    @use_profile('bigquery')
-    def test_bigquery_column_types(self):
-        self.run_and_test()

From 33b29dea13e00aa954a2ff771a42de0876a19216 Mon Sep 17 00:00:00 2001
From: Alexander Smolyakov <alexander.smolyakov.dev@gmail.com>
Date: Fri, 27 Jan 2023 19:05:23 +0400
Subject: [PATCH 534/860] [CI/CD] Update release workflow and introduce
 workflow for nightly releases (#466)

* Add workflows

* Set default `test_run` value to `true`

* Update .bumpversion.cfg

* Resolve review comments

- Update workflow docs
- Change workflow name
- Set `test_run` default value to `true`

* Update Slack secret

* Resolve review comments
---
 .bumpversion.cfg                      |   8 +-
 .github/workflows/nightly-release.yml | 109 +++++++++
 .github/workflows/release.yml         | 309 ++++++++++++++------------
 scripts/env-setup.sh                  |  13 ++
 4 files changed, 291 insertions(+), 148 deletions(-)
 create mode 100644 .github/workflows/nightly-release.yml
 create mode 100644 scripts/env-setup.sh

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index ba1f95c9a..b018b0474 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -3,8 +3,12 @@ current_version = 1.5.0a1
 parse = (?P<major>\d+)
 	\.(?P<minor>\d+)
 	\.(?P<patch>\d+)
-	((?P<prerelease>a|b|rc)(?P<num>\d+))?
+	((?P<prerelease>a|b|rc)
+	(?P<num>\d+)  # pre-release version num
+	)(\.(?P<nightly>[a-z..0-9]+)
+	)?
 serialize =
+	{major}.{minor}.{patch}{prerelease}{num}.{nightly}
 	{major}.{minor}.{patch}{prerelease}{num}
 	{major}.{minor}.{patch}
 commit = False
@@ -22,6 +26,8 @@ values =
 [bumpversion:part:num]
 first_value = 1
 
+[bumpversion:part:nightly]
+
 [bumpversion:file:setup.py]
 
 [bumpversion:file:dbt/adapters/bigquery/__version__.py]
diff --git a/.github/workflows/nightly-release.yml b/.github/workflows/nightly-release.yml
new file mode 100644
index 000000000..b668d62ec
--- /dev/null
+++ b/.github/workflows/nightly-release.yml
@@ -0,0 +1,109 @@
+# **what?**
+# Nightly releases to GitHub and PyPI. This workflow produces the following outcome:
+# - generate and validate data for night release (commit SHA, version number, release branch);
+# - pass data to release workflow;
+# - night release will be pushed to GitHub as a draft release;
+# - night build will be pushed to test PyPI;
+#
+# **why?**
+# Ensure an automated and tested release process for nightly builds
+#
+# **when?**
+# This workflow runs on schedule or can be run manually on demand.
+
+name: Nightly Test Release to GitHub and PyPI
+
+on:
+  workflow_dispatch: # for manual triggering
+  schedule:
+    - cron: 0 9 * * *
+
+permissions:
+  contents: write # this is the permission that allows creating a new release
+
+defaults:
+  run:
+    shell: bash
+
+env:
+  RELEASE_BRANCH: "main"
+
+jobs:
+  aggregate-release-data:
+    runs-on: ubuntu-latest
+
+    outputs:
+      commit_sha: ${{ steps.resolve-commit-sha.outputs.release_commit }}
+      version_number: ${{ steps.nightly-release-version.outputs.number }}
+      release_branch: ${{ steps.release-branch.outputs.name }}
+
+    steps:
+      - name: "Checkout ${{ github.repository }} Branch ${{ env.RELEASE_BRANCH }}"
+        uses: actions/checkout@v3
+        with:
+          ref: ${{ env.RELEASE_BRANCH }}
+
+      - name: "Resolve Commit To Release"
+        id: resolve-commit-sha
+        run: |
+          commit_sha=$(git rev-parse HEAD)
+          echo "release_commit=$commit_sha" >> $GITHUB_OUTPUT
+
+      - name: "Get Current Version Number"
+        id: version-number-sources
+        run: |
+          current_version=`awk -F"current_version = " '{print $2}' .bumpversion.cfg | tr '\n' ' '`
+          echo "current_version=$current_version" >> $GITHUB_OUTPUT
+
+      - name: "Audit Version And Parse Into Parts"
+        id: semver
+        uses: dbt-labs/actions/parse-semver@v1.1.0
+        with:
+          version: ${{ steps.version-number-sources.outputs.current_version }}
+
+      - name: "Get Current Date"
+        id: current-date
+        run: echo "date=$(date +'%m%d%Y')" >> $GITHUB_OUTPUT
+
+      - name: "Generate Nightly Release Version Number"
+        id: nightly-release-version
+        run: |
+          number="${{ steps.semver.outputs.version }}.dev${{ steps.current-date.outputs.date }}+nightly"
+          echo "number=$number" >> $GITHUB_OUTPUT
+
+      - name: "Audit Nightly Release Version And Parse Into Parts"
+        uses: dbt-labs/actions/parse-semver@v1.1.0
+        with:
+          version: ${{ steps.nightly-release-version.outputs.number }}
+
+      - name: "Set Release Branch"
+        id: release-branch
+        run: |
+          echo "name=${{ env.RELEASE_BRANCH }}" >> $GITHUB_OUTPUT
+
+  log-outputs-aggregate-release-data:
+    runs-on: ubuntu-latest
+    needs: [aggregate-release-data]
+
+    steps:
+      - name: "[DEBUG] Log Outputs"
+        run: |
+          echo commit_sha    : ${{ needs.aggregate-release-data.outputs.commit_sha }}
+          echo version_number: ${{ needs.aggregate-release-data.outputs.version_number }}
+          echo release_branch: ${{ needs.aggregate-release-data.outputs.release_branch }}
+
+  release-github-pypi:
+    needs: [aggregate-release-data]
+
+    uses: ./.github/workflows/release.yml
+    with:
+      sha: ${{ needs.aggregate-release-data.outputs.commit_sha }}
+      target_branch: ${{ needs.aggregate-release-data.outputs.release-branch }}
+      version_number: ${{ needs.aggregate-release-data.outputs.version_number }}
+      build_script_path: "scripts/build-dist.sh"
+      env_setup_script_path: "scripts/env-setup.sh"
+      s3_bucket_name: "core-team-artifacts"
+      package_test_command: "dbt --version"
+      test_run: true
+      nightly_release: true
+    secrets: inherit
diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml
index 5fd0291e9..a9f60734b 100644
--- a/.github/workflows/release.yml
+++ b/.github/workflows/release.yml
@@ -1,14 +1,19 @@
 # **what?**
-# Take the given commit, run unit tests specifically on that sha, build and
-# package it, and then release to GitHub with that specific build (PyPi to follow later)
-
+# Release workflow provides the following steps:
+# - checkout the given commit;
+# - validate version in sources and changelog file for given version;
+# - bump the version and generate a changelog if needed;
+# - merge all changes to the target branch if needed;
+# - run unit and integration tests against given commit;
+# - build and package that SHA;
+# - release it to GitHub and PyPI with that specific build;
+#
 # **why?**
 # Ensure an automated and tested release process
-
+#
 # **when?**
-# This will only run manually with a given sha and version
-
-name: Build, Test, and Package
+# This workflow can be run manually on demand or can be called by other workflows
+name: Release to GitHub and PyPI
 
 on:
   workflow_dispatch:
@@ -17,17 +22,85 @@ on:
         description: "The last commit sha in the release"
         type: string
         required: true
-      changelog_path:
-        description: "Path to changes log"
+      target_branch:
+        description: "The branch to release from"
+        type: string
+        required: true
+      version_number:
+        description: "The release version number (i.e. 1.0.0b1)"
+        type: string
+        required: true
+      build_script_path:
+        description: "Build script path"
+        type: string
+        default: "scripts/build-dist.sh"
+        required: true
+      env_setup_script_path:
+        description: "Environment setup script path"
         type: string
-        default: "./CHANGELOG.md"
+        default: "scripts/env-setup.sh"
+        required: false
+      s3_bucket_name:
+        description: "AWS S3 bucket name"
+        type: string
+        default: "core-team-artifacts"
+        required: true
+      package_test_command:
+        description: "Package test command"
+        type: string
+        default: "dbt --version"
+        required: true
+      test_run:
+        description: "Test run (Publish release as draft)"
+        type: boolean
+        default: true
+        required: false
+      nightly_release:
+        description: "Nightly release to dev environment"
+        type: boolean
+        default: false
         required: false
+  workflow_call:
+    inputs:
+      sha:
+        description: "The last commit sha in the release"
+        type: string
+        required: true
+      target_branch:
+        description: "The branch to release from"
+        type: string
+        required: true
       version_number:
         description: "The release version number (i.e. 1.0.0b1)"
         type: string
         required: true
+      build_script_path:
+        description: "Build script path"
+        type: string
+        default: "scripts/build-dist.sh"
+        required: true
+      env_setup_script_path:
+        description: "Environment setup script path"
+        type: string
+        default: "scripts/env-setup.sh"
+        required: false
+      s3_bucket_name:
+        description: "AWS S3 bucket name"
+        type: string
+        default: "core-team-artifacts"
+        required: true
+      package_test_command:
+        description: "Package test command"
+        type: string
+        default: "dbt --version"
+        required: true
       test_run:
-        description: "Test run (Publish release as draft to GitHub)"
+        description: "Test run (Publish release as draft)"
+        type: boolean
+        default: true
+        required: false
+      nightly_release:
+        description: "Nightly release to dev environment"
         type: boolean
         default: false
         required: false
@@ -35,10 +108,6 @@ on:
 permissions:
   contents: write # this is the permission that allows creating a new release
 
-env:
-  PYTHON_TARGET_VERSION: 3.8
-  ARTIFACT_RETENTION_DAYS: 2
-
 defaults:
   run:
     shell: bash
@@ -50,164 +119,110 @@ jobs:
     steps:
       - name: "[DEBUG] Print Variables"
         run: |
-          echo The last commit sha in the release:  ${{ inputs.sha }}
-          echo The release version number:          ${{ inputs.version_number }}
-          echo The path to the changelog markdpown: ${{ inputs.changelog_path }}
-          echo This is a test run:                  ${{ inputs.test_run }}
-          echo Python target version:               ${{ env.PYTHON_TARGET_VERSION }}
-          echo Artifact retention days:             ${{ env.ARTIFACT_RETENTION_DAYS }}
-
-  unit:
-    name: Unit Test
-    runs-on: ubuntu-latest
+          echo The last commit sha in the release: ${{ inputs.sha }}
+          echo The branch to release from:         ${{ inputs.target_branch }}
+          echo The release version number:         ${{ inputs.version_number }}
+          echo Build script path:                  ${{ inputs.build_script_path }}
+          echo Environment setup script path:      ${{ inputs.env_setup_script_path }}
+          echo AWS S3 bucket name:                 ${{ inputs.s3_bucket_name }}
+          echo Package test command:               ${{ inputs.package_test_command }}
+          echo Test run:                           ${{ inputs.test_run }}
+          echo Nightly release:                    ${{ inputs.nightly_release }}
 
-    env:
-      TOXENV: "unit"
+  bump-version-generate-changelog:
+    name: Bump package version, Generate changelog
 
-    steps:
-      - name: "Checkout Commit - ${{ inputs.sha }}"
-        uses: actions/checkout@v3
-        with:
-          persist-credentials: false
-          ref: ${{ github.event.inputs.sha }}
-
-      - name: "Set up Python - ${{ env.PYTHON_TARGET_VERSION }}"
-        uses: actions/setup-python@v4
-        with:
-          python-version: ${{ env.PYTHON_TARGET_VERSION }}
-
-      - name: "Install Python Dependencies"
-        run: |
-          python -m pip install --user --upgrade pip
-          python -m pip install tox
-          python -m pip --version
-          python -m tox --version
+    uses: dbt-labs/dbt-release/.github/workflows/release-prep.yml@main
 
-      - name: "Run Tox"
-        run: tox
-
-  build:
-    name: Build Packages
-
-    runs-on: ubuntu-latest
-
-    steps:
-      - name: "Checkout Commit - ${{ inputs.sha }}"
-        uses: actions/checkout@v3
-        with:
-          persist-credentials: false
-          ref: ${{ inputs.sha }}
-
-      - name: "Set up Python - ${{ env.PYTHON_TARGET_VERSION }}"
-        uses: actions/setup-python@v4
-        with:
-          python-version: ${{ env.PYTHON_TARGET_VERSION }}
-
-      - name: "Install Python Dependencies"
-        run: |
-          python -m pip install --user --upgrade pip
-          python -m pip install --upgrade setuptools wheel twine check-wheel-contents
-          python -m pip --version
-
-      - name: "Build Distributions"
-        run: ./scripts/build-dist.sh
-
-      - name: "[DEBUG] Show Distributions"
-        run: ls -lh dist/
-
-      - name: "Check Distribution Descriptions"
-        run: |
-          twine check dist/*
-
-      - name: "[DEBUG] Check Wheel Contents"
-        run: |
-          check-wheel-contents dist/*.whl --ignore W007,W008
+    with:
+      sha: ${{ inputs.sha }}
+      version_number: ${{ inputs.version_number }}
+      target_branch: ${{ inputs.target_branch }}
+      env_setup_script_path: ${{ inputs.env_setup_script_path }}
+      test_run: ${{ inputs.test_run }}
+      nightly_release: ${{ inputs.nightly_release }}
 
-      - name: "Upload Build Artifact - ${{ inputs.version_number }}"
-        uses: actions/upload-artifact@v3
-        with:
-          name: ${{ inputs.version_number }}
-          path: |
-            dist/
-            !dist/dbt-${{ inputs.version_number }}.tar.gz
-          retention-days: ${{ env.ARTIFACT_RETENTION_DAYS }}
+    secrets: inherit
 
-  test-build:
-    name: Verify Packages
+  log-outputs-bump-version-generate-changelog:
+    name: "[Log output] Bump package version, Generate changelog"
+    if: ${{ !failure() && !cancelled() }}
 
-    needs: [unit, build]
+    needs: [bump-version-generate-changelog]
 
     runs-on: ubuntu-latest
 
     steps:
-      - name: "Set up Python - ${{ env.PYTHON_TARGET_VERSION }}"
-        uses: actions/setup-python@v4
-        with:
-          python-version: ${{ env.PYTHON_TARGET_VERSION }}
-
-      - name: "Install Python Dependencies"
+      - name: Print variables
         run: |
-          python -m pip install --user --upgrade pip
-          python -m pip install --upgrade wheel
-          python -m pip --version
-
-      - name: "Download Build Artifact - ${{ inputs.version_number }}"
-        uses: actions/download-artifact@v3
-        with:
-          name: ${{ inputs.version_number }}
-          path: dist/
+          echo Final SHA     : ${{ needs.bump-version-generate-changelog.outputs.final_sha }}
+          echo Changelog path: ${{ needs.bump-version-generate-changelog.outputs.changelog_path }}
 
-      - name: "[DEBUG] Show Distributions"
-        run: ls -lh dist/
-
-      - name: "Install Wheel Distributions"
-        run: |
-          find ./dist/*.whl -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/
+  build-test-package:
+    name: Build, Test, Package
+    if: ${{ !failure() && !cancelled() }}
+    needs: [bump-version-generate-changelog]
 
-      - name: "[DEBUG] Check Wheel Distributions"
-        run: |
-          dbt --version
+    uses: dbt-labs/dbt-release/.github/workflows/build.yml@main
 
-      - name: "Install Source Distributions"
-        run: |
-          find ./dist/*.gz -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/
+    with:
+      sha: ${{ needs.bump-version-generate-changelog.outputs.final_sha }}
+      version_number: ${{ inputs.version_number }}
+      changelog_path: ${{ needs.bump-version-generate-changelog.outputs.changelog_path }}
+      build_script_path: ${{ inputs.build_script_path }}
+      s3_bucket_name: ${{ inputs.s3_bucket_name }}
+      package_test_command: ${{ inputs.package_test_command }}
+      test_run: ${{ inputs.test_run }}
+      nightly_release: ${{ inputs.nightly_release }}
 
-      - name: "[DEBUG] Check Source Distributions"
-        run: |
-          dbt --version
+    secrets:
+      AWS_ACCESS_KEY_ID: ${{ secrets.PRODUCTION_AWS_ACCESS_KEY_ID }}
+      AWS_SECRET_ACCESS_KEY: ${{ secrets.PRODUCTION_AWS_SECRET_ACCESS_KEY }}
 
   github-release:
     name: GitHub Release
     if: ${{ !failure() && !cancelled() }}
-    needs: test-build
 
-    # pin to commit since this is workflow is WIP but this commit has been tested as working
-    uses: dbt-labs/dbt-release/.github/workflows/github-release.yml@7b6e01d73d2c8454e06302cc66ef4c2dbd4dbe4e
+    needs: [bump-version-generate-changelog, build-test-package]
+
+    uses: dbt-labs/dbt-release/.github/workflows/github-release.yml@main
 
     with:
-      sha: ${{ inputs.sha }}
+      sha: ${{ needs.bump-version-generate-changelog.outputs.final_sha }}
       version_number: ${{ inputs.version_number }}
-      changelog_path: ${{ inputs.changelog_path }}
+      changelog_path: ${{ needs.bump-version-generate-changelog.outputs.changelog_path }}
       test_run: ${{ inputs.test_run }}
 
   pypi-release:
-    name: Pypi release
-    # only release to PyPi if we're not testing - will release to PyPi test when workflow gets rewritten
-    if: ${{ inputs.test_run == false }}
+    name: PyPI Release
 
-    runs-on: ubuntu-latest
+    needs: [github-release]
 
-    needs: github-release
+    uses: dbt-labs/dbt-release/.github/workflows/pypi-release.yml@main
 
-    environment: PypiProd
-    steps:
-      - name: "Download Build Artifact - ${{ inputs.version_number }}"
-        uses: actions/download-artifact@v3
-        with:
-          name: ${{ inputs.version_number }}
-          path: dist/
-
-      - name: Publish distribution to PyPI
-        uses: pypa/gh-action-pypi-publish@v1.4.2
-        with:
-          password: ${{ secrets.PYPI_API_TOKEN }}
+    with:
+      version_number: ${{ inputs.version_number }}
+      test_run: ${{ inputs.test_run }}
+
+    secrets:
+      PYPI_API_TOKEN: ${{ secrets.PYPI_API_TOKEN }}
+      TEST_PYPI_API_TOKEN: ${{ secrets.TEST_PYPI_API_TOKEN }}
+
+  slack-notification:
+    name: Slack Notification
+    if: ${{ failure() }}
+
+    needs:
+      [
+        bump-version-generate-changelog,
+        build-test-package,
+        github-release,
+        pypi-release,
+      ]
+
+    uses: dbt-labs/dbt-release/.github/workflows/slack-post-notification.yml@main
+    with:
+      status: "failure"
+
+    secrets:
+      SLACK_WEBHOOK_URL: ${{ secrets.SLACK_DEV_CORE_ALERTS }}
diff --git a/scripts/env-setup.sh b/scripts/env-setup.sh
new file mode 100644
index 000000000..726a00cf8
--- /dev/null
+++ b/scripts/env-setup.sh
@@ -0,0 +1,13 @@
+#!/bin/bash
+# Set TOXENV environment variable for subsequent steps
+echo "TOXENV=integration-bigquery" >> $GITHUB_ENV
+# Set INTEGRATION_TESTS_SECRETS_PREFIX environment variable for subsequent steps
+# All GH secrets that have this prefix will be set as environment variables
+echo "INTEGRATION_TESTS_SECRETS_PREFIX=BIGQUERY_TEST" >> $GITHUB_ENV
+# Set environment variables required for integration tests
+echo "DBT_TEST_USER_1=group:buildbot@dbtlabs.com" >> $GITHUB_ENV
+echo "DBT_TEST_USER_2=group:dev-core@dbtlabs.com" >> $GITHUB_ENV
+echo "DBT_TEST_USER_3=serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserviceaccount.com" >> $GITHUB_ENV
+echo "DATAPROC_REGION=us-central1" >> $GITHUB_ENV
+echo "DATAPROC_CLUSTER_NAME=dbt-test-1" >> $GITHUB_ENV
+echo "GCS_BUCKET=dbt-ci" >> $GITHUB_ENV

From 3f8bf458250236c07d7019025018b39a582ca4d0 Mon Sep 17 00:00:00 2001
From: Neelesh Salian <nssalian@users.noreply.github.com>
Date: Fri, 27 Jan 2023 10:21:05 -0800
Subject: [PATCH 535/860] Deleted redundant tests (#484)

---
 .../bigquery_test/copy-models/additional.sql  |  1 -
 .../copy-models/copy_as_incremental.sql       |  2 -
 .../copy-models/copy_as_several_tables.sql    |  1 -
 .../copy-models/copy_as_table.sql             |  1 -
 .../bigquery_test/copy-models/original.sql    |  1 -
 .../bigquery_test/copy-models/schema.yml      |  6 --
 .../execution-project-models/model.sql        |  1 -
 .../execution-project-models/schema.yml       | 10 ---
 .../test_bigquery_adapter_functions.py        | 84 -------------------
 .../test_bigquery_copy_models.py              | 41 ---------
 .../test_bigquery_execution_project.py        | 23 -----
 .../test_bigquery_query_results.py            | 24 ------
 .../test_bigquery_repeated_records.py         | 68 ---------------
 13 files changed, 263 deletions(-)
 delete mode 100644 tests/integration/bigquery_test/copy-models/additional.sql
 delete mode 100644 tests/integration/bigquery_test/copy-models/copy_as_incremental.sql
 delete mode 100644 tests/integration/bigquery_test/copy-models/copy_as_several_tables.sql
 delete mode 100644 tests/integration/bigquery_test/copy-models/copy_as_table.sql
 delete mode 100644 tests/integration/bigquery_test/copy-models/original.sql
 delete mode 100644 tests/integration/bigquery_test/copy-models/schema.yml
 delete mode 100644 tests/integration/bigquery_test/execution-project-models/model.sql
 delete mode 100644 tests/integration/bigquery_test/execution-project-models/schema.yml
 delete mode 100644 tests/integration/bigquery_test/test_bigquery_adapter_functions.py
 delete mode 100644 tests/integration/bigquery_test/test_bigquery_copy_models.py
 delete mode 100644 tests/integration/bigquery_test/test_bigquery_execution_project.py
 delete mode 100644 tests/integration/bigquery_test/test_bigquery_query_results.py
 delete mode 100644 tests/integration/bigquery_test/test_bigquery_repeated_records.py

diff --git a/tests/integration/bigquery_test/copy-models/additional.sql b/tests/integration/bigquery_test/copy-models/additional.sql
deleted file mode 100644
index 33560d6c0..000000000
--- a/tests/integration/bigquery_test/copy-models/additional.sql
+++ /dev/null
@@ -1 +0,0 @@
-select 2 as id
diff --git a/tests/integration/bigquery_test/copy-models/copy_as_incremental.sql b/tests/integration/bigquery_test/copy-models/copy_as_incremental.sql
deleted file mode 100644
index bbe8e5acd..000000000
--- a/tests/integration/bigquery_test/copy-models/copy_as_incremental.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-{{ config(copy_materialization='incremental') }}
-{{ ref('original') }}
\ No newline at end of file
diff --git a/tests/integration/bigquery_test/copy-models/copy_as_several_tables.sql b/tests/integration/bigquery_test/copy-models/copy_as_several_tables.sql
deleted file mode 100644
index 99b04e1b3..000000000
--- a/tests/integration/bigquery_test/copy-models/copy_as_several_tables.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('original') }}, {{ source('test_copy_several_tables', 'additional') }}
diff --git a/tests/integration/bigquery_test/copy-models/copy_as_table.sql b/tests/integration/bigquery_test/copy-models/copy_as_table.sql
deleted file mode 100644
index 3f2fe2550..000000000
--- a/tests/integration/bigquery_test/copy-models/copy_as_table.sql
+++ /dev/null
@@ -1 +0,0 @@
-{{ ref('original') }}
diff --git a/tests/integration/bigquery_test/copy-models/original.sql b/tests/integration/bigquery_test/copy-models/original.sql
deleted file mode 100644
index 26d9cae7b..000000000
--- a/tests/integration/bigquery_test/copy-models/original.sql
+++ /dev/null
@@ -1 +0,0 @@
-select 1 as id
\ No newline at end of file
diff --git a/tests/integration/bigquery_test/copy-models/schema.yml b/tests/integration/bigquery_test/copy-models/schema.yml
deleted file mode 100644
index defbd47b5..000000000
--- a/tests/integration/bigquery_test/copy-models/schema.yml
+++ /dev/null
@@ -1,6 +0,0 @@
-version: 2
-sources:
-  - name: test_copy_several_tables
-    schema: "{{ target.schema }}"
-    tables:
-      - name: additional
diff --git a/tests/integration/bigquery_test/execution-project-models/model.sql b/tests/integration/bigquery_test/execution-project-models/model.sql
deleted file mode 100644
index 43258a714..000000000
--- a/tests/integration/bigquery_test/execution-project-models/model.sql
+++ /dev/null
@@ -1 +0,0 @@
-select 1 as id
diff --git a/tests/integration/bigquery_test/execution-project-models/schema.yml b/tests/integration/bigquery_test/execution-project-models/schema.yml
deleted file mode 100644
index 390165a86..000000000
--- a/tests/integration/bigquery_test/execution-project-models/schema.yml
+++ /dev/null
@@ -1,10 +0,0 @@
-version: 2
-models:
-- name: model
-  description: |
-    I'm testing the profile execution_project
-  tests:
-    - project_for_job_id:
-        region: region-us
-        project_id: "{{ project_id}}"
-        unique_schema_id: "{{ unique_schema_id }}"
\ No newline at end of file
diff --git a/tests/integration/bigquery_test/test_bigquery_adapter_functions.py b/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
deleted file mode 100644
index 427470f1f..000000000
--- a/tests/integration/bigquery_test/test_bigquery_adapter_functions.py
+++ /dev/null
@@ -1,84 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, FakeArgs, use_profile
-import yaml
-
-
-class TestBigqueryAdapterFunctions(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "adapter-models"
-
-    @property
-    def profile_config(self):
-        return self.bigquery_profile()
-
-    @use_profile('bigquery')
-    def test__bigquery_adapter_functions(self):
-        results = self.run_dbt()
-        self.assertEqual(len(results), 3)
-
-        for result in results:
-            # all queries in adapter models are jobs that are expected to have a location/project_id/job_id
-            assert result.adapter_response["location"] is not None
-            assert result.adapter_response["project_id"] is not None
-            assert result.adapter_response["job_id"] is not None
-            assert result.adapter_response["slot_ms"] is not None
-
-        test_results = self.run_dbt(['test'])
-
-        self.assertTrue(len(test_results) > 0)
-        for result in test_results:
-            self.assertEqual(result.status, 'pass')
-            self.assertFalse(result.skipped)
-            self.assertEqual(result.failures, 0)
-
-
-class TestBigqueryAdapterMacros(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "models"
-
-    def _create_schema_named(self, database, schema):
-        # do not create the initial schema. We'll do this ourselves!
-        pass
-
-    @use_profile('bigquery')
-    def test__bigquery_run_create_drop_schema(self):
-        schema_args = yaml.safe_dump({
-            'db_name': self.default_database,
-            'schema_name': self.unique_schema(),
-        })
-        self.run_dbt(
-            ['run-operation', 'my_create_schema', '--args', schema_args])
-        relation_args = yaml.safe_dump({
-            'db_name': self.default_database,
-            'schema_name': self.unique_schema(),
-            'table_name': 'some_table',
-        })
-        self.run_dbt(['run-operation', 'my_create_table_as',
-                      '--args', relation_args])
-        # exercise list_relations_without_caching and get_columns_in_relation
-        self.run_dbt(
-            ['run-operation', 'ensure_one_relation_in', '--args', schema_args])
-        # now to drop the schema
-        schema_relation = self.adapter.Relation.create(
-            database=self.default_database, schema=self.unique_schema()).without_identifier()
-        with self.adapter.connection_named('test'):
-            results = self.adapter.list_relations_without_caching(
-                schema_relation)
-        assert len(results) == 1
-
-        self.run_dbt(
-            ['run-operation', 'my_drop_schema', '--args', schema_args])
-        with self.adapter.connection_named('test'):
-            results = self.adapter.list_relations_without_caching(
-                schema_relation)
-        assert len(results) == 0
diff --git a/tests/integration/bigquery_test/test_bigquery_copy_models.py b/tests/integration/bigquery_test/test_bigquery_copy_models.py
deleted file mode 100644
index f4362659b..000000000
--- a/tests/integration/bigquery_test/test_bigquery_copy_models.py
+++ /dev/null
@@ -1,41 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-import textwrap
-import yaml
-
-
-class TestBigqueryCopyTable(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "copy-models"
-
-    @property
-    def profile_config(self):
-        return self.bigquery_profile()
-
-    @property
-    def project_config(self):
-        return yaml.safe_load(textwrap.dedent('''\
-        config-version: 2
-        models:
-            test:
-                original:
-                    materialized: table
-                additional:
-                    materialized: table
-                copy_as_table:
-                    materialized: copy
-                copy_as_several_tables:
-                    materialized: copy
-                copy_as_incremental:
-                    materialized: copy
-        '''))
-
-    @use_profile('bigquery')
-    def test__bigquery_copy_table(self):
-        results = self.run_dbt()
-        self.assertEqual(len(results), 5)
diff --git a/tests/integration/bigquery_test/test_bigquery_execution_project.py b/tests/integration/bigquery_test/test_bigquery_execution_project.py
deleted file mode 100644
index 50c4c92f5..000000000
--- a/tests/integration/bigquery_test/test_bigquery_execution_project.py
+++ /dev/null
@@ -1,23 +0,0 @@
-import os
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-
-class TestAlternateExecutionProjectBigQueryRun(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "execution-project-models"
-
-    @use_profile('bigquery')
-    def test__bigquery_execute_project(self):
-        results = self.run_dbt(['run', '--models', 'model'])
-        self.assertEqual(len(results), 1)
-        execution_project = os.environ['BIGQUERY_TEST_ALT_DATABASE']
-        self.run_dbt(['test',
-                      '--target', 'alternate',
-                      '--vars', '{ project_id: %s, unique_schema_id: %s }'
-                      % (execution_project, self.unique_schema())],
-                     expect_pass=False)
diff --git a/tests/integration/bigquery_test/test_bigquery_query_results.py b/tests/integration/bigquery_test/test_bigquery_query_results.py
deleted file mode 100644
index 6cd32f383..000000000
--- a/tests/integration/bigquery_test/test_bigquery_query_results.py
+++ /dev/null
@@ -1,24 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-
-class TestBaseBigQueryResults(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'macro-paths': ['macros'],
-        }
-
-    @use_profile('bigquery')
-    def test__bigquery_type_inference(self):
-        result = self.run_dbt(['run-operation', 'test_int_inference'])
-        self.assertTrue(result.success)
diff --git a/tests/integration/bigquery_test/test_bigquery_repeated_records.py b/tests/integration/bigquery_test/test_bigquery_repeated_records.py
deleted file mode 100644
index 16ee69cda..000000000
--- a/tests/integration/bigquery_test/test_bigquery_repeated_records.py
+++ /dev/null
@@ -1,68 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-import json
-
-class TestBaseBigQueryRun(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'macro-paths': ['macros'],
-        }
-
-    @use_profile('bigquery')
-    def test__bigquery_fetch_nested_records(self):
-        sql = """
-        select
-          struct(
-            cast('Michael' as string) as fname,
-            cast('Stonebreaker' as string) as lname
-          ) as user,
-          [
-            struct(1 as val_1, cast(2.12 as numeric) as val_2),
-            struct(3 as val_1, cast(4.83 as numeric) as val_2)
-          ] as val
-
-        union all
-
-        select
-          struct(
-            cast('Johnny' as string) as fname,
-            cast('Brickmaker' as string) as lname
-          ) as user,
-          [
-            struct(7 as val_1, cast(8 as numeric) as val_2),
-            struct(9 as val_1, cast(null as numeric) as val_2)
-          ] as val
-        """
-
-
-        status, res = self.adapter.execute(sql, fetch=True)
-
-        self.assertEqual(len(res), 2, "incorrect row count")
-
-        expected = {
-            "user": [
-                '{"fname": "Michael", "lname": "Stonebreaker"}',
-                '{"fname": "Johnny", "lname": "Brickmaker"}'
-            ],
-            "val": [
-                '[{"val_1": 1, "val_2": 2.12}, {"val_1": 3, "val_2": 4.83}]',
-                '[{"val_1": 7, "val_2": 8}, {"val_1": 9, "val_2": null}]'
-            ]
-        }
-
-        for i, key in enumerate(expected):
-            line = "row {} for key {} ({} vs {})".format(i, key, expected[key][i], res[i][key])
-            # py2 serializes these in an unordered way - deserialize to compare
-            v1 = expected[key][i]
-            v2 = res[i][key]
-            self.assertEqual(json.loads(v1), json.loads(v2), line)

From 6f945febb0076ef74ccb1df69f65e8e9cbc5d291 Mon Sep 17 00:00:00 2001
From: Alexander Smolyakov <alexander.smolyakov.dev@gmail.com>
Date: Mon, 30 Jan 2023 19:19:06 +0400
Subject: [PATCH 536/860] Update release workflow (#497)

- Update AWS secrets
- Rework condition for Slack notification
---
 .github/workflows/release.yml | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml
index a9f60734b..1c0885001 100644
--- a/.github/workflows/release.yml
+++ b/.github/workflows/release.yml
@@ -176,8 +176,8 @@ jobs:
       nightly_release: ${{ inputs.nightly_release }}
 
     secrets:
-      AWS_ACCESS_KEY_ID: ${{ secrets.PRODUCTION_AWS_ACCESS_KEY_ID }}
-      AWS_SECRET_ACCESS_KEY: ${{ secrets.PRODUCTION_AWS_SECRET_ACCESS_KEY }}
+      AWS_ACCESS_KEY_ID: ${{ secrets.AWS_ACCESS_KEY_ID }}
+      AWS_SECRET_ACCESS_KEY: ${{ secrets.AWS_SECRET_ACCESS_KEY }}
 
   github-release:
     name: GitHub Release
@@ -210,7 +210,7 @@ jobs:
 
   slack-notification:
     name: Slack Notification
-    if: ${{ failure() }}
+    if: ${{ failure() && (!inputs.test_run || inputs.nightly_release) }}
 
     needs:
       [

From 446d30b63a160b66ac581673bf858beca85c7d69 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Mon, 30 Jan 2023 11:19:26 -0800
Subject: [PATCH 537/860] remove case sensitive and copy fail integ tests
 (#494)

---
 .../case-sensitive-models/fUnKyCaSe.sql       |  5 ---
 .../case-sensitive-schemas/model.sql          |  5 ---
 .../copy_bad_materialization.sql              |  2 --
 .../copy-failing-models/original.sql          |  1 -
 .../test_bigquery_case_sensitive.py           | 34 ------------------
 .../test_bigquery_copy_failing_models.py      | 36 -------------------
 6 files changed, 83 deletions(-)
 delete mode 100644 tests/integration/bigquery_test/case-sensitive-models/fUnKyCaSe.sql
 delete mode 100644 tests/integration/bigquery_test/case-sensitive-schemas/model.sql
 delete mode 100644 tests/integration/bigquery_test/copy-failing-models/copy_bad_materialization.sql
 delete mode 100644 tests/integration/bigquery_test/copy-failing-models/original.sql
 delete mode 100644 tests/integration/bigquery_test/test_bigquery_case_sensitive.py
 delete mode 100644 tests/integration/bigquery_test/test_bigquery_copy_failing_models.py

diff --git a/tests/integration/bigquery_test/case-sensitive-models/fUnKyCaSe.sql b/tests/integration/bigquery_test/case-sensitive-models/fUnKyCaSe.sql
deleted file mode 100644
index 1934d4cfc..000000000
--- a/tests/integration/bigquery_test/case-sensitive-models/fUnKyCaSe.sql
+++ /dev/null
@@ -1,5 +0,0 @@
-{{ config(materialized='incremental') }}
-select 1 as id
-{% if is_incremental() %}
-this is a syntax error!
-{% endif %}
diff --git a/tests/integration/bigquery_test/case-sensitive-schemas/model.sql b/tests/integration/bigquery_test/case-sensitive-schemas/model.sql
deleted file mode 100644
index 1934d4cfc..000000000
--- a/tests/integration/bigquery_test/case-sensitive-schemas/model.sql
+++ /dev/null
@@ -1,5 +0,0 @@
-{{ config(materialized='incremental') }}
-select 1 as id
-{% if is_incremental() %}
-this is a syntax error!
-{% endif %}
diff --git a/tests/integration/bigquery_test/copy-failing-models/copy_bad_materialization.sql b/tests/integration/bigquery_test/copy-failing-models/copy_bad_materialization.sql
deleted file mode 100644
index b6093645d..000000000
--- a/tests/integration/bigquery_test/copy-failing-models/copy_bad_materialization.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-{{ config(copy_materialization='view') }}
-{{ ref('original') }}
\ No newline at end of file
diff --git a/tests/integration/bigquery_test/copy-failing-models/original.sql b/tests/integration/bigquery_test/copy-failing-models/original.sql
deleted file mode 100644
index 26d9cae7b..000000000
--- a/tests/integration/bigquery_test/copy-failing-models/original.sql
+++ /dev/null
@@ -1 +0,0 @@
-select 1 as id
\ No newline at end of file
diff --git a/tests/integration/bigquery_test/test_bigquery_case_sensitive.py b/tests/integration/bigquery_test/test_bigquery_case_sensitive.py
deleted file mode 100644
index 60700eada..000000000
--- a/tests/integration/bigquery_test/test_bigquery_case_sensitive.py
+++ /dev/null
@@ -1,34 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-
-class TestCaseSensitiveModelBigQueryRun(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "case-sensitive-models"
-
-    @use_profile('bigquery')
-    def test__bigquery_double_run_fails(self):
-        results = self.run_dbt()
-        self.assertEqual(len(results), 1)
-        self.run_dbt(expect_pass=False)
-
-
-class TestCaseSensitiveSchemaBigQueryRun(TestCaseSensitiveModelBigQueryRun):
-    # same test, but the schema is funky instead of the model name
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    def unique_schema(self):
-        schema = self.schema
-
-        to_return = "{}_{}".format(self.prefix, schema)
-        return to_return
-
-    @property
-    def models(self):
-        return "case-sensitive-schemas"
diff --git a/tests/integration/bigquery_test/test_bigquery_copy_failing_models.py b/tests/integration/bigquery_test/test_bigquery_copy_failing_models.py
deleted file mode 100644
index 659525663..000000000
--- a/tests/integration/bigquery_test/test_bigquery_copy_failing_models.py
+++ /dev/null
@@ -1,36 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-import textwrap
-import yaml
-
-
-class TestBigqueryCopyTableFails(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "copy-failing-models"
-
-    @property
-    def profile_config(self):
-        return self.bigquery_profile()
-
-    @property
-    def project_config(self):
-        return yaml.safe_load(textwrap.dedent('''\
-        config-version: 2
-        models:
-            test:
-                original:
-                    materialized: table
-                copy_bad_materialization:
-                    materialized: copy
-        '''))
-
-    @use_profile('bigquery')
-    def test__bigquery_copy_table_fails(self):
-        results = self.run_dbt(expect_pass=False)
-        self.assertEqual(len(results), 2)
-        self.assertEqual(results[1].status, 'error')

From 73f36024d45a756e6bf88e934909f2c07cb217a7 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Mon, 30 Jan 2023 11:23:10 -0800
Subject: [PATCH 538/860] convert update columns tests (#488)

* convert update columns tests

* remove integ test fixtures
---
 tests/functional/test_update_column_policy.py | 61 +++++++++++++
 .../test_update_field_description.py          | 58 +++++++++++++
 .../test_bigquery_update_columns.py           | 85 -------------------
 .../description_table.sql                     |  9 --
 .../description_table.yml                     |  7 --
 .../policy_tag_table.sql                      |  9 --
 .../policy_tag_table.yml                      |  8 --
 7 files changed, 119 insertions(+), 118 deletions(-)
 create mode 100644 tests/functional/test_update_column_policy.py
 create mode 100644 tests/functional/test_update_field_description.py
 delete mode 100644 tests/integration/bigquery_test/test_bigquery_update_columns.py
 delete mode 100644 tests/integration/bigquery_test/update-column-description/description_table.sql
 delete mode 100644 tests/integration/bigquery_test/update-column-description/description_table.yml
 delete mode 100644 tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.sql
 delete mode 100644 tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.yml

diff --git a/tests/functional/test_update_column_policy.py b/tests/functional/test_update_column_policy.py
new file mode 100644
index 000000000..0599c0099
--- /dev/null
+++ b/tests/functional/test_update_column_policy.py
@@ -0,0 +1,61 @@
+import pytest
+from dbt.tests.util import (
+    run_dbt, get_connection, relation_from_name
+)
+
+from dbt.adapters.bigquery import BigQueryRelation
+
+_POLICY_TAG_MODEL = """{{
+  config(
+    materialized='table',
+    persist_docs={ 'columns': true }
+  )
+}}
+
+select
+  1 field
+"""
+
+_POLICY_TAG_YML = """version: 2
+
+models:
+- name: policy_tag_table
+  columns:
+  - name: field
+    policy_tags:
+      - '{{ var("policy_tag") }}'
+"""
+
+# Manually generated https://console.cloud.google.com/bigquery/policy-tags?project=dbt-test-env
+_POLICY_TAG = "projects/dbt-test-env/locations/us/taxonomies/5785568062805976401/policyTags/135489647357012267"
+_POLICY_TAG_MODEL_NAME = "policy_tag_table"
+
+class TestBigqueryUpdateColumnPolicy:
+    """See BQ docs for more info on policy tags:
+    https://cloud.google.com/bigquery/docs/column-level-security#work_with_policy_tags"""
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            'config-version': 2,
+            'vars': {
+                'policy_tag': _POLICY_TAG
+            }
+        }
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            f"{_POLICY_TAG_MODEL_NAME}.sql": _POLICY_TAG_MODEL,
+            "schema.yml": _POLICY_TAG_YML
+        }
+
+    def test_bigquery_update_column_policy_tag(self, project):
+        results = run_dbt(['run', '--models', 'policy_tag_table'])
+        assert len(results) == 1
+        relation: BigQueryRelation = relation_from_name(project.adapter, _POLICY_TAG_MODEL_NAME)
+        adapter = project.adapter
+        with get_connection(project.adapter) as conn:
+            table = conn.handle.get_table(
+                adapter.connections.get_bq_table(relation.database, relation.schema, relation.table))
+            for schema_field in table.schema:
+                assert schema_field.policy_tags.names == (_POLICY_TAG,)
diff --git a/tests/functional/test_update_field_description.py b/tests/functional/test_update_field_description.py
new file mode 100644
index 000000000..68be9c42f
--- /dev/null
+++ b/tests/functional/test_update_field_description.py
@@ -0,0 +1,58 @@
+import pytest
+from dbt.tests.util import (
+    relation_from_name,
+    get_connection,
+    run_dbt
+)
+
+from dbt.adapters.bigquery import BigQueryRelation
+
+_FIELD_DESCRIPTION_MODEL = """{{
+  config(
+    materialized='table',
+    persist_docs={ 'columns': true }
+  )
+}}
+
+select
+  1 field
+"""
+_FIELD_DESCRIPTION_MODEL_NAME = "field_description_model"
+_FIELD_DESCRIPTION = 'this is not a field'
+_FIELD_DESCRIPTION_MODEL_YML = """
+version: 2
+
+models:
+- name: field_description_model
+  columns:
+  - name: field
+    description: '{{ var("field_description") }}'
+"""
+
+class TestBigqueryUpdateColumnDescription:
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            'config-version': 2,
+            'vars': {
+                'field_description': _FIELD_DESCRIPTION
+            }
+        }
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            f"{_FIELD_DESCRIPTION_MODEL_NAME}.sql": _FIELD_DESCRIPTION_MODEL,
+            "schema.yml": _FIELD_DESCRIPTION_MODEL_YML
+        }
+
+    def test_bigquery_update_column_description(self, project):
+        results = run_dbt(['run'])
+        assert len(results) == 1
+        relation: BigQueryRelation = relation_from_name(project.adapter, _FIELD_DESCRIPTION_MODEL_NAME)
+        adapter = project.adapter
+        with get_connection(project.adapter) as conn:
+            table = conn.handle.get_table(
+                adapter.connections.get_bq_table(relation.database, relation.schema, relation.table))
+            for schema_field in table.schema:
+                assert schema_field.description == _FIELD_DESCRIPTION
diff --git a/tests/integration/bigquery_test/test_bigquery_update_columns.py b/tests/integration/bigquery_test/test_bigquery_update_columns.py
deleted file mode 100644
index ed3c93556..000000000
--- a/tests/integration/bigquery_test/test_bigquery_update_columns.py
+++ /dev/null
@@ -1,85 +0,0 @@
-import os
-
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-
-class TestBigqueryUpdateColumnPolicyTag(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "update-column-policy-tag"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'vars': {
-                'policy_tag': self.policy_tag
-            }
-        }
-
-    @property
-    def policy_tag(self):
-        return os.environ.get('BIGQUERY_POLICY_TAG')
-
-    @use_profile('bigquery')
-    def test__bigquery_update_column_policy_tag(self):
-        if self.policy_tag:
-            results = self.run_dbt(['run', '--models', 'policy_tag_table'])
-            self.assertEqual(len(results), 1)
-
-            with self.get_connection() as conn:
-                client = conn.handle
-
-                table = client.get_table(
-                    self.adapter.connections.get_bq_table(
-                        self.default_database, self.unique_schema(), 'policy_tag_table')
-                )
-
-                for schema_field in table.schema:
-                    self.assertEquals(schema_field.policy_tags.names,
-                                      (self.policy_tag,))
-
-
-class TestBigqueryUpdateColumnDescription(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "update-column-description"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'vars': {
-                'field_description': self.field_description
-            }
-        }
-
-    @property
-    def field_description(self):
-        return 'this is a field'
-
-    @use_profile('bigquery')
-    def test__bigquery_update_column_description(self):
-        results = self.run_dbt(['run', '--models', 'description_table'])
-        self.assertEqual(len(results), 1)
-
-        with self.get_connection() as conn:
-            client = conn.handle
-
-            table = client.get_table(
-                self.adapter.connections.get_bq_table(
-                    self.default_database, self.unique_schema(), 'description_table')
-            )
-
-            for schema_field in table.schema:
-                self.assertEquals(schema_field.description, self.field_description)
diff --git a/tests/integration/bigquery_test/update-column-description/description_table.sql b/tests/integration/bigquery_test/update-column-description/description_table.sql
deleted file mode 100644
index 7110ac8b0..000000000
--- a/tests/integration/bigquery_test/update-column-description/description_table.sql
+++ /dev/null
@@ -1,9 +0,0 @@
-{{
-  config(
-    materialized='table',
-    persist_docs={ 'columns': true }
-  )
-}}
-
-select
-  1 field
diff --git a/tests/integration/bigquery_test/update-column-description/description_table.yml b/tests/integration/bigquery_test/update-column-description/description_table.yml
deleted file mode 100644
index 28d68f007..000000000
--- a/tests/integration/bigquery_test/update-column-description/description_table.yml
+++ /dev/null
@@ -1,7 +0,0 @@
-version: 2
-
-models:
-- name: description_table
-  columns:
-  - name: field
-    description: '{{ var("field_description") }}'
diff --git a/tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.sql b/tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.sql
deleted file mode 100644
index 7110ac8b0..000000000
--- a/tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.sql
+++ /dev/null
@@ -1,9 +0,0 @@
-{{
-  config(
-    materialized='table',
-    persist_docs={ 'columns': true }
-  )
-}}
-
-select
-  1 field
diff --git a/tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.yml b/tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.yml
deleted file mode 100644
index 01a76c50d..000000000
--- a/tests/integration/bigquery_test/update-column-policy-tag/policy_tag_table.yml
+++ /dev/null
@@ -1,8 +0,0 @@
-version: 2
-
-models:
-- name: policy_tag_table
-  columns:
-  - name: field
-    policy_tags:
-      - '{{ var("policy_tag") }}'

From 8339a034929b12e027f0a143abf46582f3f6ffbc Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Mon, 30 Jan 2023 15:30:23 -0700
Subject: [PATCH 539/860] Use IEC standard abbreviations (GiB, TiB, etc) (#482)

* Use IEC standard abbreviations (GiB, TiB, etc)

* Changelog entry

---------

Co-authored-by: Florian Eiden <florian.eiden@fleid.fr>
---
 .changes/unreleased/Fixes-20230125-174159.yaml | 7 +++++++
 dbt/adapters/bigquery/connections.py           | 2 +-
 2 files changed, 8 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20230125-174159.yaml

diff --git a/.changes/unreleased/Fixes-20230125-174159.yaml b/.changes/unreleased/Fixes-20230125-174159.yaml
new file mode 100644
index 000000000..4100080d4
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230125-174159.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: Use IEC standard abbreviations (GiB, TiB, etc)
+time: 2023-01-25T17:41:59.921173-07:00
+custom:
+  Author: dbeatty10
+  Issue: "477"
+  PR: "477"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 715dfb36a..7bcc63c4b 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -257,7 +257,7 @@ def commit(self):
 
     def format_bytes(self, num_bytes):
         if num_bytes:
-            for unit in ["Bytes", "KB", "MB", "GB", "TB", "PB"]:
+            for unit in ["Bytes", "KiB", "MiB", "GiB", "TiB", "PiB"]:
                 if abs(num_bytes) < 1024.0:
                     return f"{num_bytes:3.1f} {unit}"
                 num_bytes /= 1024.0

From 92dba77f1a4472830e754fee9f090affe4ab5b1a Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Tue, 31 Jan 2023 11:47:40 -0600
Subject: [PATCH 540/860] Remove PR requirement from Changelog (#480)

* remove PR requirement, allow multiple issues for a single changelog

* add comment about multiple issues

* update input type

* update to list issues in contributors section
---
 .changie.yaml                       | 110 ++++++++++++++++++++++------
 .github/workflows/bot-changelog.yml |   5 +-
 2 files changed, 88 insertions(+), 27 deletions(-)

diff --git a/.changie.yaml b/.changie.yaml
index fc7a42cd5..37f9191de 100644
--- a/.changie.yaml
+++ b/.changie.yaml
@@ -6,57 +6,119 @@ changelogPath: CHANGELOG.md
 versionExt: md
 versionFormat: '## dbt-bigquery {{.Version}} - {{.Time.Format "January 02, 2006"}}'
 kindFormat: '### {{.Kind}}'
-changeFormat: '- {{.Body}} ([#{{.Custom.Issue}}](https://github.com/dbt-labs/dbt-bigquery/issues/{{.Custom.Issue}}), [#{{.Custom.PR}}](https://github.com/dbt-labs/dbt-bigquery/pull/{{.Custom.PR}}))'
+changeFormat: |-
+  {{- $IssueList := list }}
+  {{- $changes := splitList " " $.Custom.Issue }}
+  {{- range $issueNbr := $changes }}
+    {{- $changeLink := "[#nbr](https://github.com/dbt-labs/dbt-bigquery/issues/nbr)" | replace "nbr" $issueNbr }}
+    {{- $IssueList = append $IssueList $changeLink  }}
+  {{- end -}}
+  - {{.Body}} ({{ range $index, $element := $IssueList }}{{if $index}}, {{end}}{{$element}}{{end}})
+
 kinds:
 - label: Breaking Changes
 - label: Features
 - label: Fixes
 - label: Under the Hood
 - label: Dependencies
-  changeFormat: '- {{.Body}} ({{if ne .Custom.Issue ""}}[#{{.Custom.Issue}}](https://github.com/dbt-labs/dbt-bigquery/issues/{{.Custom.Issue}}), {{end}}[#{{.Custom.PR}}](https://github.com/dbt-labs/dbt-bigquery/pull/{{.Custom.PR}}))'
+  changeFormat: |-
+    {{- $PRList := list }}
+    {{- $changes := splitList " " $.Custom.PR }}
+    {{- range $pullrequest := $changes }}
+      {{- $changeLink := "[#nbr](https://github.com/dbt-labs/dbt-bigquery/pull/nbr)" | replace "nbr" $pullrequest }}
+      {{- $PRList = append $PRList $changeLink  }}
+    {{- end -}}
+    - {{.Body}} ({{ range $index, $element := $PRList }}{{if $index}}, {{end}}{{$element}}{{end}})
+  skipGlobalChoices: true
+  additionalChoices:
+    - key: Author
+      label: GitHub Username(s) (separated by a single space if multiple)
+      type: string
+      minLength: 3
+    - key: PR
+      label: GitHub Pull Request Number (separated by a single space if multiple)
+      type: string
+      minLength: 1
 - label: Security
-  changeFormat: '- {{.Body}} ({{if ne .Custom.Issue ""}}[#{{.Custom.Issue}}](https://github.com/dbt-labs/dbt-bigquery/issues/{{.Custom.Issue}}), {{end}}[#{{.Custom.PR}}](https://github.com/dbt-labs/dbt-bigquery/pull/{{.Custom.PR}}))'
+  changeFormat: |-
+    {{- $PRList := list }}
+    {{- $changes := splitList " " $.Custom.PR }}
+    {{- range $pullrequest := $changes }}
+      {{- $changeLink := "[#nbr](https://github.com/dbt-labs/dbt-bigquery/pull/nbr)" | replace "nbr" $pullrequest }}
+      {{- $PRList = append $PRList $changeLink  }}
+    {{- end -}}
+    - {{.Body}} ({{ range $index, $element := $PRList }}{{if $index}}, {{end}}{{$element}}{{end}})
+  skipGlobalChoices: true
+  additionalChoices:
+    - key: Author
+      label: GitHub Username(s) (separated by a single space if multiple)
+      type: string
+      minLength: 3
+    - key: PR
+      label: GitHub Pull Request Number (separated by a single space if multiple)
+      type: string
+      minLength: 1
+
+newlines:
+  afterChangelogHeader: 1
+  afterKind: 1
+  afterChangelogVersion: 1
+  beforeKind: 1
+  endOfVersion: 1
+
 custom:
 - key: Author
   label: GitHub Username(s) (separated by a single space if multiple)
   type: string
   minLength: 3
 - key: Issue
-  label: GitHub Issue Number
-  type: int
-  minLength: 4
-- key: PR
-  label: GitHub Pull Request Number
-  type: int
-  minLength: 4
+  label: GitHub Issue Number (separated by a single space if multiple)
+  type: string
+  minLength: 1
+
 footerFormat: |
   {{- $contributorDict := dict }}
   {{- /* any names added to this list should be all lowercase for later matching purposes */}}
-  {{- $core_team := list "michelleark" "peterallenwebb" "emmyoop" "nathaniel-may" "gshank" "leahwicz" "chenyulinx" "stu-k" "iknox-fa" "versusfacit" "mcknight-42" "jtcohen6" "aranke" "mikealfare" "dependabot[bot]" "snyk-bot" "colin-rogers-dbt" }}
+  {{- $core_team := list "michelleark" "peterallenwebb" "emmyoop" "nathaniel-may" "gshank" "leahwicz" "chenyulinx" "stu-k" "iknox-fa" "versusfacit" "mcknight-42" "jtcohen6" "aranke" "dependabot[bot]" "snyk-bot" "colin-rogers-dbt" }}
   {{- range $change := .Changes }}
     {{- $authorList := splitList " " $change.Custom.Author }}
-    {{- /* loop through all authors for a PR */}}
+    {{- /* loop through all authors for a single changelog */}}
     {{- range $author := $authorList }}
       {{- $authorLower := lower $author }}
       {{- /* we only want to include non-core team contributors */}}
       {{- if not (has $authorLower $core_team)}}
-        {{- $pr := $change.Custom.PR }}
-        {{- /* check if this contributor has other PRs associated with them already */}}
-        {{- if hasKey $contributorDict $author }}
-          {{- $prList := get $contributorDict $author }}
-          {{- $prList = append $prList $pr  }}
-          {{- $contributorDict := set $contributorDict $author $prList }}
-        {{- else }}
-          {{- $prList := list $change.Custom.PR }}
-          {{- $contributorDict := set $contributorDict $author $prList }}
-        {{- end }}
-      {{- end}}
+        {{- $changeList := splitList " " $change.Custom.Author }}
+          {{- $IssueList := list }}
+          {{- $changeLink := $change.Kind }}
+          {{- if or (eq $change.Kind "Dependencies") (eq $change.Kind "Security") }}
+            {{- $changes := splitList " " $change.Custom.PR }}
+            {{- range $issueNbr := $changes }}
+              {{- $changeLink := "[#nbr](https://github.com/dbt-labs/dbt-bigquery/pull/nbr)" | replace "nbr" $issueNbr }}
+              {{- $IssueList = append $IssueList $changeLink  }}
+            {{- end -}}
+          {{- else }}
+            {{- $changes := splitList " " $change.Custom.Issue }}
+            {{- range $issueNbr := $changes }}
+              {{- $changeLink := "[#nbr](https://github.com/dbt-labs/dbt-bigquery/issues/nbr)" | replace "nbr" $issueNbr }}
+              {{- $IssueList = append $IssueList $changeLink  }}
+            {{- end -}}
+          {{- end }}
+          {{- /* check if this contributor has other changes associated with them already */}}
+          {{- if hasKey $contributorDict $author }}
+            {{- $contributionList := get $contributorDict $author }}
+            {{- $contributionList = concat $contributionList $IssueList  }}
+            {{- $contributorDict := set $contributorDict $author $contributionList }}
+          {{- else }}
+            {{- $contributionList := $IssueList }}
+            {{- $contributorDict := set $contributorDict $author $contributionList }}
+          {{- end }}
+        {{- end}}
     {{- end}}
   {{- end }}
   {{- /* no indentation here for formatting so the final markdown doesn't have unneeded indentations */}}
   {{- if $contributorDict}}
   ### Contributors
   {{- range $k,$v := $contributorDict }}
-  - [@{{$k}}](https://github.com/{{$k}}) ({{ range $index, $element := $v }}{{if $index}}, {{end}}[#{{$element}}](https://github.com/dbt-labs/dbt-bigquery/pull/{{$element}}){{end}})
+  - [@{{$k}}](https://github.com/{{$k}}) ({{ range $index, $element := $v }}{{if $index}}, {{end}}{{$element}}{{end}})
   {{- end }}
   {{- end }}
diff --git a/.github/workflows/bot-changelog.yml b/.github/workflows/bot-changelog.yml
index 2ac7091bf..8122ab8b4 100644
--- a/.github/workflows/bot-changelog.yml
+++ b/.github/workflows/bot-changelog.yml
@@ -9,7 +9,6 @@
 # time: <current timestamp>
 # custom:
 #   Author: <PR User Login (generally the bot)>
-#   Issue: 4904
 #   PR: <PR number>
 #
 # **why?**
@@ -40,7 +39,7 @@ jobs:
       matrix:
         include:
           - label: "dependencies"
-            changie_kind: "Dependency"
+            changie_kind: "Dependencies"
           - label: "snyk"
             changie_kind: "Security"
     runs-on: ubuntu-latest
@@ -58,4 +57,4 @@ jobs:
         commit_message: "Add automated changelog yaml from template for bot PR"
         changie_kind: ${{ matrix.changie_kind }}
         label: ${{ matrix.label }}
-        custom_changelog_string: "custom:\n  Author: ${{ github.event.pull_request.user.login }}\n  Issue: 254\n  PR: ${{ github.event.pull_request.number }}\n"
+        custom_changelog_string: "custom:\n  Author: ${{ github.event.pull_request.user.login }}\n  PR: ${{ github.event.pull_request.number }}\n"

From edca37abfa21d492b75f1fa03d20e51da55304bb Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Tue, 31 Jan 2023 13:24:29 -0800
Subject: [PATCH 541/860] Migrate test and use adapter zone base classes.
 (#495)

* Migrate test and use adapter zone base classes.

* Remove shas now that base version of conversion is in

---------

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .../adapter}/test_persist_docs.py             | 222 ++++++++++--------
 .../models-bigquery-nested/schema.yml         |  19 --
 .../table_model_nested.sql                    |   8 -
 .../view_model_nested.sql                     |   8 -
 .../models-column-missing/missing_column.sql  |   2 -
 .../models-column-missing/schema.yml          |   8 -
 .../persist_docs_tests/models/my_fun_docs.md  |  10 -
 .../models/no_docs_model.sql                  |   1 -
 .../persist_docs_tests/models/schema.yml      |  70 ------
 .../persist_docs_tests/models/table_model.sql |   2 -
 .../persist_docs_tests/models/view_model.sql  |   2 -
 .../persist_docs_tests/seeds/seed.csv         |   3 -
 12 files changed, 119 insertions(+), 236 deletions(-)
 rename tests/{integration/persist_docs_tests => functional/adapter}/test_persist_docs.py (54%)
 delete mode 100644 tests/integration/persist_docs_tests/models-bigquery-nested/schema.yml
 delete mode 100644 tests/integration/persist_docs_tests/models-bigquery-nested/table_model_nested.sql
 delete mode 100644 tests/integration/persist_docs_tests/models-bigquery-nested/view_model_nested.sql
 delete mode 100644 tests/integration/persist_docs_tests/models-column-missing/missing_column.sql
 delete mode 100644 tests/integration/persist_docs_tests/models-column-missing/schema.yml
 delete mode 100644 tests/integration/persist_docs_tests/models/my_fun_docs.md
 delete mode 100644 tests/integration/persist_docs_tests/models/no_docs_model.sql
 delete mode 100644 tests/integration/persist_docs_tests/models/schema.yml
 delete mode 100644 tests/integration/persist_docs_tests/models/table_model.sql
 delete mode 100644 tests/integration/persist_docs_tests/models/view_model.sql
 delete mode 100644 tests/integration/persist_docs_tests/seeds/seed.csv

diff --git a/tests/integration/persist_docs_tests/test_persist_docs.py b/tests/functional/adapter/test_persist_docs.py
similarity index 54%
rename from tests/integration/persist_docs_tests/test_persist_docs.py
rename to tests/functional/adapter/test_persist_docs.py
index 5079c035e..169d1074e 100644
--- a/tests/integration/persist_docs_tests/test_persist_docs.py
+++ b/tests/functional/adapter/test_persist_docs.py
@@ -1,70 +1,89 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-import os
-
 import json
-
-
-class BasePersistDocsTest(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "persist_docs"
-
-    @property
-    def models(self):
-        return "models"
-
-    def _assert_common_comments(self, *comments):
-        for comment in comments:
-            assert '"with double quotes"' in comment
-            assert """'''abc123'''""" in comment
-            assert '\n' in comment
-            assert 'Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting' in comment
-            assert '/* comment */' in comment
-            if os.name == 'nt':
-                assert '--\r\n' in comment or '--\n' in comment
-            else:
-                assert '--\n' in comment
-
-    def _assert_has_table_comments(self, table_node):
-        table_comment = table_node['metadata']['comment']
-        assert table_comment.startswith('Table model description')
-
-        table_id_comment = table_node['columns']['id']['comment']
-        assert table_id_comment.startswith('id Column description')
-
-        table_name_comment = table_node['columns']['name']['comment']
-        assert table_name_comment.startswith(
-            'Some stuff here and then a call to')
-
-        self._assert_common_comments(
-            table_comment, table_id_comment, table_name_comment
-        )
-
-    def _assert_has_view_comments(self, view_node, has_node_comments=True,
-                                  has_column_comments=True):
-        view_comment = view_node['metadata']['comment']
+import os
+import pytest
+
+from dbt.tests.util import (
+    run_dbt
+)
+
+from dbt.tests.adapter.persist_docs.test_persist_docs import (
+    BasePersistDocsBase,
+    BasePersistDocs,
+    BasePersistDocsColumnMissing,
+    BasePersistDocsCommentOnQuotedColumn,
+)
+
+_MODELS__TABLE_MODEL_NESTED = """
+{{ config(materialized='table') }}
+SELECT
+    STRUCT(
+        STRUCT(
+            1 AS level_3_a,
+            2 AS level_3_b
+        ) AS level_2
+    ) AS level_1
+"""
+
+_MODELS__VIEW_MODEL_NESTED = """
+{{ config(materialized='view') }}
+SELECT
+    STRUCT(
+        STRUCT(
+            1 AS level_3_a,
+            2 AS level_3_b
+        ) AS level_2
+    ) AS level_1
+"""
+
+_PROPERTIES__MODEL_COMMENTS = """
+version: 2
+
+models:
+  - name: table_model_nested
+    columns:
+      - name: level_1
+        description: level_1 column description
+      - name: level_1.level_2
+        description: level_2 column description
+      - name: level_1.level_2.level_3_a
+        description: level_3 column description
+  - name: view_model_nested
+    columns:
+      - name: level_1
+        description: level_1 column description
+      - name: level_1.level_2
+        description: level_2 column description
+      - name: level_1.level_2.level_3_a
+        description: level_3 column description
+"""
+
+
+class TestBasePersistDocs(BasePersistDocs):
+    def _assert_has_view_comments(
+        self, view_node, has_node_comments=True, has_column_comments=True
+    ):
+        view_comment = view_node["metadata"]["comment"]
         if has_node_comments:
-            assert view_comment.startswith('View model description')
+            assert view_comment.startswith("View model description")
             self._assert_common_comments(view_comment)
         else:
-            assert view_comment is None
+            assert not view_comment
 
-        view_id_comment = view_node['columns']['id']['comment']
+        view_id_comment = view_node["columns"]["id"]["comment"]
         if has_column_comments:
-            assert view_id_comment.startswith('id Column description')
+            assert view_id_comment.startswith("id Column description")
             self._assert_common_comments(view_id_comment)
         else:
-            assert view_id_comment is None
+            assert not view_id_comment
 
-        view_name_comment = view_node['columns']['name']['comment']
-        assert view_name_comment is None
+        view_name_comment = view_node["columns"]["name"]["comment"]
+        assert not view_name_comment
 
 
-class TestPersistDocsSimple(BasePersistDocsTest):
-    @property
-    def project_config(self):
+class TestPersistDocsSimple(BasePersistDocsBase):
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
         return {
-            'config-version': 2,
             'models': {
                 'test': {
                     '+persist_docs': {
@@ -82,24 +101,22 @@ def project_config(self):
                 }
             },
         }
-
-    @use_profile('bigquery')
-    def test_bigquery_persist_docs(self):
-        self.run_dbt(['seed'])
-        self.run_dbt()
+    def test_persist_docs(self, project):
+        run_dbt(['seed'])
+        run_dbt()
         desc_map = {
             'seed': 'Seed model description',
             'table_model': 'Table model description',
             'view_model': 'View model description',
         }
         for node_id in ['seed', 'table_model', 'view_model']:
-            with self.adapter.connection_named('_test'):
-                client = self.adapter.connections \
+            with project.adapter.connection_named('_test'):
+                client = project.adapter.connections \
                     .get_thread_connection().handle
 
                 table_id = "{}.{}.{}".format(
-                    self.default_database,
-                    self.unique_schema(),
+                    project.database,
+                    project.test_schema,
                     node_id
                 )
                 bq_table = client.get_table(table_id)
@@ -112,37 +129,59 @@ def test_bigquery_persist_docs(self):
                     assert bq_schema[1].description.startswith('Some stuff here and then a call to')
 
 
-class TestPersistDocsNested(BasePersistDocsTest):
-    @property
-    def project_config(self):
+class TestPersistDocsColumnMissing(BasePersistDocsBase):
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
         return {
-            'config-version': 2,
             'models': {
                 'test': {
                     '+persist_docs': {
-                        "relation": True,
                         "columns": True,
                     },
                 }
             }
         }
 
-    @property
+    def test_missing_column(self, project):
+        run_dbt()
+
+
+class TestPersistDocsNested(BasePersistDocsBase):
+    @pytest.fixture(scope="class")
+    def properties(self):
+        return {"schema.yml": _PROPERTIES__MODEL_COMMENTS}
+
+    @pytest.fixture(scope="class")
     def models(self):
-        return 'models-bigquery-nested'
+        return {
+            "table_model_nested.sql": _MODELS__TABLE_MODEL_NESTED,
+            "view_model_nested.sql": _MODELS__VIEW_MODEL_NESTED
+        }
 
-    @use_profile('bigquery')
-    def test_bigquery_persist_docs(self):
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            'models': {
+                'test': {
+                    '+persist_docs': {
+                        "relation": True,
+                        "columns": True,
+                    },
+                }
+            }
+        }
+
+    def test_persist_docs(self, project):
         """
         run dbt and use the bigquery client from the adapter to check if the
         colunmn descriptions are persisted on the test model table and view.
 
         Next, generate the catalog and check if the comments are also included.
         """
-        self.run_dbt(['seed'])
-        self.run_dbt()
+        run_dbt(['seed'])
+        run_dbt()
 
-        self.run_dbt(['docs', 'generate'])
+        run_dbt(['docs', 'generate'])
         with open('target/catalog.json') as fp:
             catalog_data = json.load(fp)
         assert 'nodes' in catalog_data
@@ -150,13 +189,13 @@ def test_bigquery_persist_docs(self):
 
         for node_id in ['table_model_nested', 'view_model_nested']:
             # check the descriptions using the api
-            with self.adapter.connection_named('_test'):
-                client = self.adapter.connections \
+            with project.adapter.connection_named('_test'):
+                client = project.adapter.connections \
                     .get_thread_connection().handle
 
                 table_id = "{}.{}.{}".format(
-                    self.default_database,
-                    self.unique_schema(),
+                    project.database,
+                    project.test_schema,
                     node_id
                 )
                 bq_schema = client.get_table(table_id).schema
@@ -184,26 +223,3 @@ def test_bigquery_persist_docs(self):
 
             level_3_column = node['columns']['level_1.level_2.level_3_a']
             assert level_3_column['comment'] == "level_3 column description"
-
-
-class TestPersistDocsColumnMissing(BasePersistDocsTest):
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'models': {
-                'test': {
-                    '+persist_docs': {
-                        "columns": True,
-                    },
-                }
-            }
-        }
-
-    @property
-    def models(self):
-        return 'models-column-missing'
-
-    @use_profile('bigquery')
-    def test_bigquery_missing_column(self):
-        self.run_dbt()
diff --git a/tests/integration/persist_docs_tests/models-bigquery-nested/schema.yml b/tests/integration/persist_docs_tests/models-bigquery-nested/schema.yml
deleted file mode 100644
index 0311dcb14..000000000
--- a/tests/integration/persist_docs_tests/models-bigquery-nested/schema.yml
+++ /dev/null
@@ -1,19 +0,0 @@
-version: 2
-
-models:
-  - name: table_model_nested
-    columns:
-      - name: level_1
-        description: level_1 column description
-      - name: level_1.level_2
-        description: level_2 column description
-      - name: level_1.level_2.level_3_a
-        description: level_3 column description
-  - name: view_model_nested
-    columns:
-      - name: level_1
-        description: level_1 column description
-      - name: level_1.level_2
-        description: level_2 column description
-      - name: level_1.level_2.level_3_a
-        description: level_3 column description
\ No newline at end of file
diff --git a/tests/integration/persist_docs_tests/models-bigquery-nested/table_model_nested.sql b/tests/integration/persist_docs_tests/models-bigquery-nested/table_model_nested.sql
deleted file mode 100644
index c2936d4f1..000000000
--- a/tests/integration/persist_docs_tests/models-bigquery-nested/table_model_nested.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-{{ config(materialized='table') }}
-SELECT
-    STRUCT(
-        STRUCT(
-            1 AS level_3_a,
-            2 AS level_3_b
-        ) AS level_2
-    ) AS level_1
\ No newline at end of file
diff --git a/tests/integration/persist_docs_tests/models-bigquery-nested/view_model_nested.sql b/tests/integration/persist_docs_tests/models-bigquery-nested/view_model_nested.sql
deleted file mode 100644
index e3323ddf4..000000000
--- a/tests/integration/persist_docs_tests/models-bigquery-nested/view_model_nested.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-{{ config(materialized='view') }}
-SELECT
-    STRUCT(
-        STRUCT(
-            1 AS level_3_a,
-            2 AS level_3_b
-        ) AS level_2
-    ) AS level_1
\ No newline at end of file
diff --git a/tests/integration/persist_docs_tests/models-column-missing/missing_column.sql b/tests/integration/persist_docs_tests/models-column-missing/missing_column.sql
deleted file mode 100644
index 642b0f14a..000000000
--- a/tests/integration/persist_docs_tests/models-column-missing/missing_column.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-{{ config(materialized='table') }}
-select 1 as id, 'Ed' as name
diff --git a/tests/integration/persist_docs_tests/models-column-missing/schema.yml b/tests/integration/persist_docs_tests/models-column-missing/schema.yml
deleted file mode 100644
index aa7b4f888..000000000
--- a/tests/integration/persist_docs_tests/models-column-missing/schema.yml
+++ /dev/null
@@ -1,8 +0,0 @@
-version: 2
-models:
-  - name: missing_column
-    columns:
-      - name: id
-        description: "test id column description"
-      - name: column_that_does_not_exist
-        description: "comment that cannot be created"
diff --git a/tests/integration/persist_docs_tests/models/my_fun_docs.md b/tests/integration/persist_docs_tests/models/my_fun_docs.md
deleted file mode 100644
index f3c0fbf55..000000000
--- a/tests/integration/persist_docs_tests/models/my_fun_docs.md
+++ /dev/null
@@ -1,10 +0,0 @@
-{% docs my_fun_doc %}
-name Column description "with double quotes"
-and with 'single  quotes' as welll as other;
-'''abc123'''
-reserved -- characters
---
-/* comment */
-Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting
-
-{% enddocs %}
diff --git a/tests/integration/persist_docs_tests/models/no_docs_model.sql b/tests/integration/persist_docs_tests/models/no_docs_model.sql
deleted file mode 100644
index e39a7a156..000000000
--- a/tests/integration/persist_docs_tests/models/no_docs_model.sql
+++ /dev/null
@@ -1 +0,0 @@
-select 1 as id, 'Alice' as name
diff --git a/tests/integration/persist_docs_tests/models/schema.yml b/tests/integration/persist_docs_tests/models/schema.yml
deleted file mode 100644
index 5a9091624..000000000
--- a/tests/integration/persist_docs_tests/models/schema.yml
+++ /dev/null
@@ -1,70 +0,0 @@
-version: 2
-
-models:
-  - name: table_model
-    description: |
-      Table model description "with double quotes"
-      and with 'single  quotes' as welll as other;
-      '''abc123'''
-      reserved -- characters
-      --
-      /* comment */
-      Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting
-    columns:
-      - name: id
-        description: |
-          id Column description "with double quotes"
-          and with 'single  quotes' as welll as other;
-          '''abc123'''
-          reserved -- characters
-          --
-          /* comment */
-          Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting
-      - name: name
-        description: |
-          Some stuff here and then a call to
-          {{ doc('my_fun_doc')}}
-  - name: view_model
-    description: |
-      View model description "with double quotes"
-      and with 'single  quotes' as welll as other;
-      '''abc123'''
-      reserved -- characters
-      --
-      /* comment */
-      Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting
-    columns:
-      - name: id
-        description: |
-          id Column description "with double quotes"
-          and with 'single  quotes' as welll as other;
-          '''abc123'''
-          reserved -- characters
-          --
-          /* comment */
-          Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting
-
-seeds:
-  - name: seed
-    description: |
-      Seed model description "with double quotes"
-      and with 'single  quotes' as welll as other;
-      '''abc123'''
-      reserved -- characters
-      --
-      /* comment */
-      Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting
-    columns:
-      - name: id
-        description: |
-          id Column description "with double quotes"
-          and with 'single  quotes' as welll as other;
-          '''abc123'''
-          reserved -- characters
-          --
-          /* comment */
-          Some $lbl$ labeled $lbl$ and $$ unlabeled $$ dollar-quoting
-      - name: name
-        description: |
-          Some stuff here and then a call to
-          {{ doc('my_fun_doc')}}
diff --git a/tests/integration/persist_docs_tests/models/table_model.sql b/tests/integration/persist_docs_tests/models/table_model.sql
deleted file mode 100644
index c0e93c3f3..000000000
--- a/tests/integration/persist_docs_tests/models/table_model.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-{{ config(materialized='table') }}
-select 1 as id, 'Joe' as name
diff --git a/tests/integration/persist_docs_tests/models/view_model.sql b/tests/integration/persist_docs_tests/models/view_model.sql
deleted file mode 100644
index a6f96a16d..000000000
--- a/tests/integration/persist_docs_tests/models/view_model.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-{{ config(materialized='view') }}
-select 2 as id, 'Bob' as name
diff --git a/tests/integration/persist_docs_tests/seeds/seed.csv b/tests/integration/persist_docs_tests/seeds/seed.csv
deleted file mode 100644
index 1a728c8ab..000000000
--- a/tests/integration/persist_docs_tests/seeds/seed.csv
+++ /dev/null
@@ -1,3 +0,0 @@
-id,name
-1,Alice
-2,Bob

From 339f56a3ccfd6c08d34031e850d5ef4ef8d71bc7 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Thu, 2 Feb 2023 12:21:45 -0600
Subject: [PATCH 542/860] update regex for version bump (#511)

---
 .bumpversion.cfg | 17 +++++++++++------
 1 file changed, 11 insertions(+), 6 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index b018b0474..2f645d9e2 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,11 +1,16 @@
 [bumpversion]
 current_version = 1.5.0a1
-parse = (?P<major>\d+)
-	\.(?P<minor>\d+)
-	\.(?P<patch>\d+)
-	((?P<prerelease>a|b|rc)
-	(?P<num>\d+)  # pre-release version num
-	)(\.(?P<nightly>[a-z..0-9]+)
+# `parse` allows parsing the version into the parts we need to check.  There are some
+# unnamed groups and that's okay because they do not need to be audited.  If any part
+# of the version passed and does not match the regex, it will fail.
+# expected matches: `1.5.0`, `1.5.0a1`, `1.5.0a1.dev123457+nightly`
+# excepted failures: `1`, `1.5`, `1.5.2-a1`, `text1.5.0`
+parse = (?P<major>[\d]+) # major version number
+	\.(?P<minor>[\d]+) # minor version number
+	\.(?P<patch>[\d]+) # patch version number
+	(((?P<prekind>a|b|rc) # optional pre-release type
+	?(?P<num>[\d]+?)) # optional pre-release version number
+	\.?(?P<nightly>[a-z0-9]+\+[a-z]+)? # optional nightly release indicator
 	)?
 serialize =
 	{major}.{minor}.{patch}{prerelease}{num}.{nightly}

From 36f5f105cde7674c6410c9448fca3cb170439ea8 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 2 Feb 2023 23:53:29 -0600
Subject: [PATCH 543/860] BigQuery_tests conversion (#492)

* init push for tests conversions in bigquery_tests

* add changing_partitions test conversion and update to main

* move tests outside of sub folder

* adding a revert on the location param to not break next test

* spacing fix

* modify fixture for test to have way to revert to default location

* add a try, finally condition as possible extra protection

* remove unneeded try finally condtion

* remove unused arg
---
 .../test_changing_partitions.py}              | 149 +++++++++++-------
 tests/functional/test_hours_to_expiration.py  |  34 ++++
 tests/functional/test_location_change.py      |  42 +++++
 .../expiring_table.sql                        |   1 -
 .../bigquery_test/location-models/model.sql   |   1 -
 .../macros/partition_metadata.sql             |  19 ---
 .../partition-models/my_model.sql             |  15 --
 .../bigquery_test/partition-models/schema.yml |   6 -
 .../test_bigquery_adapter_specific.py         |  40 -----
 .../test_bigquery_location_change.py          |  31 ----
 10 files changed, 172 insertions(+), 166 deletions(-)
 rename tests/{integration/bigquery_test/test_bigquery_changing_partitions.py => functional/test_changing_partitions.py} (65%)
 create mode 100644 tests/functional/test_hours_to_expiration.py
 create mode 100644 tests/functional/test_location_change.py
 delete mode 100644 tests/integration/bigquery_test/adapter-specific-models/expiring_table.sql
 delete mode 100644 tests/integration/bigquery_test/location-models/model.sql
 delete mode 100644 tests/integration/bigquery_test/macros/partition_metadata.sql
 delete mode 100644 tests/integration/bigquery_test/partition-models/my_model.sql
 delete mode 100644 tests/integration/bigquery_test/partition-models/schema.yml
 delete mode 100644 tests/integration/bigquery_test/test_bigquery_adapter_specific.py
 delete mode 100644 tests/integration/bigquery_test/test_bigquery_location_change.py

diff --git a/tests/integration/bigquery_test/test_bigquery_changing_partitions.py b/tests/functional/test_changing_partitions.py
similarity index 65%
rename from tests/integration/bigquery_test/test_bigquery_changing_partitions.py
rename to tests/functional/test_changing_partitions.py
index 7d543397e..ad6977556 100644
--- a/tests/integration/bigquery_test/test_bigquery_changing_partitions.py
+++ b/tests/functional/test_changing_partitions.py
@@ -1,34 +1,88 @@
-from tests.integration.base import DBTIntegrationTest, FakeArgs, use_profile
+import pytest
 import json
-
-
-class TestChangingPartitions(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
+from dbt.tests.util import run_dbt
+
+_MACRO_SQL = """
+{% test number_partitions(model, expected) %}
+
+    {%- set result = get_partitions_metadata(model) %}
+
+    {% if result %}
+        {% set partitions = result.columns['partition_id'].values() %}
+    {% else %}
+        {% set partitions = () %}
+    {% endif %}
+
+    {% set actual = partitions | length %}
+    {% set success = 1 if model and actual == expected else 0 %}
+
+    select 'Expected {{ expected }}, but got {{ actual }}' as validation_error
+    from (select true)
+    where {{ success }} = 0
+
+{% endtest %}
+"""
+
+_MODEL_SQL = """
+{{
+    config(
+        materialized="table",
+        partition_by=var('partition_by'),
+        cluster_by=var('cluster_by'),
+        partition_expiration_days=var('partition_expiration_days'),
+        require_partition_filter=var('require_partition_filter')
+    )
+}}
+
+select 1 as id, 'dr. bigquery' as name, current_timestamp() as cur_time, current_date() as cur_date
+union all
+select 2 as id, 'prof. bigquery' as name, current_timestamp() as cur_time, current_date() as cur_date
+"""
+
+_SCHEMA_YML = """
+version: 2
+models:
+- name: my_model
+  tests:
+  - number_partitions:
+      expected: "{{ var('expected', 1) }}"
+"""
+
+
+class BaseBigQueryChangingPartition:
+
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {
+            "partition_metadata.sql": _MACRO_SQL
+        }
+
+    @pytest.fixture(scope='class')
     def models(self):
-        return "partition-models"
+        return {
+            "my_model.sql": _MODEL_SQL,
+            "schema.yml": _SCHEMA_YML
+        }
+
 
     def run_changes(self, before, after):
-        results = self.run_dbt(['run', '--vars', json.dumps(before)])
-        self.assertEqual(len(results), 1)
+        results = run_dbt(['run', '--vars', json.dumps(before)])
+        assert len(results) == 1
 
-        results = self.run_dbt(['run', '--vars', json.dumps(after)])
-        self.assertEqual(len(results), 1)
+        results = run_dbt(['run', '--vars', json.dumps(after)])
+        assert len(results) == 1
 
-    def test_partitions(self, expected):
-        test_results = self.run_dbt(['test', '--vars', json.dumps(expected)])
+    def partitions_test(self, expected):
+        test_results = run_dbt(['test', '--vars', json.dumps(expected)])
 
         for result in test_results:
-            self.assertEqual(result.status, 'pass')
-            self.assertFalse(result.skipped)
-            self.assertEqual(result.failures, 0)
+            assert result.status == "pass"
+            assert result.skipped == False
+            assert result.failures == 0
+
+class TestBigQueryChangingPartition(BaseBigQueryChangingPartition):
 
-    @use_profile('bigquery')
-    def test_bigquery_add_partition(self):
+    def test_bigquery_add_partition(self, project):
         before = {"partition_by": None,
                   "cluster_by": None,
                   'partition_expiration_days': None,
@@ -38,10 +92,9 @@ def test_bigquery_add_partition(self):
                  'partition_expiration_days': 7,
                  'require_partition_filter': True}
         self.run_changes(before, after)
-        self.test_partitions({"expected": 1})
+        self.partitions_test({"expected": 1})
 
-    @use_profile('bigquery')
-    def test_bigquery_add_partition_year(self):
+    def test_bigquery_add_partition_year(self, project):
         before = {"partition_by": None,
                   "cluster_by": None,
                   'partition_expiration_days': None,
@@ -51,10 +104,9 @@ def test_bigquery_add_partition_year(self):
                  'partition_expiration_days': None,
                  'require_partition_filter': None}
         self.run_changes(before, after)
-        self.test_partitions({"expected": 1})
+        self.partitions_test({"expected": 1})
 
-    @use_profile('bigquery')
-    def test_bigquery_add_partition_month(self):
+    def test_bigquery_add_partition_month(self, project):
         before = {"partition_by": None,
                   "cluster_by": None,
                   'partition_expiration_days': None,
@@ -64,10 +116,9 @@ def test_bigquery_add_partition_month(self):
                  'partition_expiration_days': None,
                  'require_partition_filter': None}
         self.run_changes(before, after)
-        self.test_partitions({"expected": 1})
+        self.partitions_test({"expected": 1})
 
-    @use_profile('bigquery')
-    def test_bigquery_add_partition_hour(self):
+    def test_bigquery_add_partition_hour(self, project):
         before = {"partition_by": None,
                   "cluster_by": None,
                   'partition_expiration_days': None,
@@ -77,10 +128,9 @@ def test_bigquery_add_partition_hour(self):
                  'partition_expiration_days': None,
                  'require_partition_filter': None}
         self.run_changes(before, after)
-        self.test_partitions({"expected": 1})
+        self.partitions_test({"expected": 1})
 
-    @use_profile('bigquery')
-    def test_bigquery_add_partition_hour(self):
+    def test_bigquery_add_partition_hour(self, project):
         before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp', 'granularity': 'day'},
                   "cluster_by": None,
                   'partition_expiration_days': None,
@@ -90,10 +140,9 @@ def test_bigquery_add_partition_hour(self):
                  'partition_expiration_days': None,
                  'require_partition_filter': None}
         self.run_changes(before, after)
-        self.test_partitions({"expected": 1})
+        self.partitions_test({"expected": 1})
 
-    @use_profile('bigquery')
-    def test_bigquery_remove_partition(self):
+    def test_bigquery_remove_partition(self, project):
         before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'},
                   "cluster_by": None,
                   'partition_expiration_days': None,
@@ -104,8 +153,7 @@ def test_bigquery_remove_partition(self):
                  'require_partition_filter': None}
         self.run_changes(before, after)
 
-    @use_profile('bigquery')
-    def test_bigquery_change_partitions(self):
+    def test_bigquery_change_partitions(self, project):
         before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'},
                   "cluster_by": None,
                   'partition_expiration_days': None,
@@ -115,12 +163,11 @@ def test_bigquery_change_partitions(self):
                  'partition_expiration_days': 7,
                  'require_partition_filter': True}
         self.run_changes(before, after)
-        self.test_partitions({"expected": 1})
+        self.partitions_test({"expected": 1})
         self.run_changes(after, before)
-        self.test_partitions({"expected": 1})
+        self.partitions_test({"expected": 1})
 
-    @use_profile('bigquery')
-    def test_bigquery_change_partitions_from_int(self):
+    def test_bigquery_change_partitions_from_int(self, project):
         before = {"partition_by": {"field": "id", "data_type": "int64", "range": {"start": 0, "end": 10, "interval": 1}},
                   "cluster_by": None,
                   'partition_expiration_days': None,
@@ -130,12 +177,11 @@ def test_bigquery_change_partitions_from_int(self):
                  'partition_expiration_days': None,
                  'require_partition_filter': None}
         self.run_changes(before, after)
-        self.test_partitions({"expected": 1})
+        self.partitions_test({"expected": 1})
         self.run_changes(after, before)
-        self.test_partitions({"expected": 2})
+        self.partitions_test({"expected": 2})
 
-    @use_profile('bigquery')
-    def test_bigquery_add_clustering(self):
+    def test_bigquery_add_clustering(self, project):
         before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'},
                   "cluster_by": None,
                   'partition_expiration_days': None,
@@ -146,8 +192,7 @@ def test_bigquery_add_clustering(self):
                  'require_partition_filter': None}
         self.run_changes(before, after)
 
-    @use_profile('bigquery')
-    def test_bigquery_remove_clustering(self):
+    def test_bigquery_remove_clustering(self, project):
         before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'},
                   "cluster_by": "id",
                   'partition_expiration_days': None,
@@ -158,8 +203,7 @@ def test_bigquery_remove_clustering(self):
                  'require_partition_filter': None}
         self.run_changes(before, after)
 
-    @use_profile('bigquery')
-    def test_bigquery_change_clustering(self):
+    def test_bigquery_change_clustering(self, project):
         before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'},
                   "cluster_by": "id",
                   'partition_expiration_days': None,
@@ -170,8 +214,7 @@ def test_bigquery_change_clustering(self):
                  'require_partition_filter': None}
         self.run_changes(before, after)
 
-    @use_profile('bigquery')
-    def test_bigquery_change_clustering_strict(self):
+    def test_bigquery_change_clustering_strict(self, project):
         before = {'partition_by': {'field': 'cur_time', 'data_type': 'timestamp'},
                   'cluster_by': 'id',
                   'partition_expiration_days': None,
diff --git a/tests/functional/test_hours_to_expiration.py b/tests/functional/test_hours_to_expiration.py
new file mode 100644
index 000000000..e38917756
--- /dev/null
+++ b/tests/functional/test_hours_to_expiration.py
@@ -0,0 +1,34 @@
+import pytest
+from dbt.tests.util import run_dbt_and_capture
+
+_MODEL_SQL = """
+select 1 as id
+"""
+
+class BaseBigQueryHoursToExpiration:
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "model.sql": _MODEL_SQL,
+        }
+
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            "models": {
+                "test": {
+                    "materialized": "table",
+                    "model": {
+                        "hours_to_expiration": "4"
+                    }
+                }
+            }
+        }
+
+class TestBigQueryHoursToExpiration(BaseBigQueryHoursToExpiration):
+
+    def test_bigquery_hours_to_expiration(self, project):
+        _, stdout = run_dbt_and_capture(["--debug", "run"])
+        assert 'expiration_timestamp=TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 4 hour)' in stdout
+
diff --git a/tests/functional/test_location_change.py b/tests/functional/test_location_change.py
new file mode 100644
index 000000000..805d06daf
--- /dev/null
+++ b/tests/functional/test_location_change.py
@@ -0,0 +1,42 @@
+import pytest
+import os
+from dbt.tests.util import run_dbt
+
+_MODEL_SQL = """
+select 1 as id
+"""
+
+_INVALID_LOCATION = os.getenv('DBT_TEST_BIGQUERY_BAD_LOCATION', 'northamerica-northeast1')
+_VALID_LOCATION = os.getenv('DBT_TEST_BIGQUERY_INITIAL_LOCATION', 'US')
+
+class BaseBigQueryLocation:
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "model.sql": _MODEL_SQL,
+        }
+
+
+class TestBigqueryValidLocation(BaseBigQueryLocation):
+
+    def test_bigquery_valid_location(self, project):
+        results = run_dbt()
+        for result in results:
+            assert "US" == result.adapter_response["location"]
+
+
+class TestBigqueryInvalidLocation(BaseBigQueryLocation):
+
+    @pytest.fixture(scope="class")
+    def profiles_config_update(self, dbt_profile_target):
+        outputs = {"default": dbt_profile_target}
+        outputs["default"]["location"] = _INVALID_LOCATION
+        yield
+        outputs = {"default": dbt_profile_target}
+        outputs["default"]["location"] = _VALID_LOCATION
+
+    def test_bigquery_location_invalid(self, project):
+        results = run_dbt()
+        for result in results:
+            assert "northamerica-northeast1" == result.adapter_response["location"]
diff --git a/tests/integration/bigquery_test/adapter-specific-models/expiring_table.sql b/tests/integration/bigquery_test/adapter-specific-models/expiring_table.sql
deleted file mode 100644
index 43258a714..000000000
--- a/tests/integration/bigquery_test/adapter-specific-models/expiring_table.sql
+++ /dev/null
@@ -1 +0,0 @@
-select 1 as id
diff --git a/tests/integration/bigquery_test/location-models/model.sql b/tests/integration/bigquery_test/location-models/model.sql
deleted file mode 100644
index 43258a714..000000000
--- a/tests/integration/bigquery_test/location-models/model.sql
+++ /dev/null
@@ -1 +0,0 @@
-select 1 as id
diff --git a/tests/integration/bigquery_test/macros/partition_metadata.sql b/tests/integration/bigquery_test/macros/partition_metadata.sql
deleted file mode 100644
index 4a4131c9e..000000000
--- a/tests/integration/bigquery_test/macros/partition_metadata.sql
+++ /dev/null
@@ -1,19 +0,0 @@
-
-{% test number_partitions(model, expected) %}
-
-    {%- set result = get_partitions_metadata(model) %}
-
-    {% if result %}
-        {% set partitions = result.columns['partition_id'].values() %}
-    {% else %}
-        {% set partitions = () %}
-    {% endif %}
-
-    {% set actual = partitions | length %}
-    {% set success = 1 if model and actual == expected else 0 %}
-
-    select 'Expected {{ expected }}, but got {{ actual }}' as validation_error
-    from (select true)
-    where {{ success }} = 0
-
-{% endtest %}
diff --git a/tests/integration/bigquery_test/partition-models/my_model.sql b/tests/integration/bigquery_test/partition-models/my_model.sql
deleted file mode 100644
index 554db0932..000000000
--- a/tests/integration/bigquery_test/partition-models/my_model.sql
+++ /dev/null
@@ -1,15 +0,0 @@
-
-
-{{
-    config(
-        materialized="table",
-        partition_by=var('partition_by'),
-        cluster_by=var('cluster_by'),
-        partition_expiration_days=var('partition_expiration_days'),
-        require_partition_filter=var('require_partition_filter')
-    )
-}}
-
-select 1 as id, 'dr. bigquery' as name, current_timestamp() as cur_time, current_date() as cur_date
-union all
-select 2 as id, 'prof. bigquery' as name, current_timestamp() as cur_time, current_date() as cur_date
diff --git a/tests/integration/bigquery_test/partition-models/schema.yml b/tests/integration/bigquery_test/partition-models/schema.yml
deleted file mode 100644
index 208701eaa..000000000
--- a/tests/integration/bigquery_test/partition-models/schema.yml
+++ /dev/null
@@ -1,6 +0,0 @@
-version: 2
-models:
-- name: my_model
-  tests:
-  - number_partitions:
-      expected: "{{ var('expected', 1) }}"
diff --git a/tests/integration/bigquery_test/test_bigquery_adapter_specific.py b/tests/integration/bigquery_test/test_bigquery_adapter_specific.py
deleted file mode 100644
index 03170378b..000000000
--- a/tests/integration/bigquery_test/test_bigquery_adapter_specific.py
+++ /dev/null
@@ -1,40 +0,0 @@
-""""Test adapter specific config options."""
-from pprint import pprint
-
-from tests.integration.base import DBTIntegrationTest, use_profile
-import textwrap
-import yaml
-
-
-class TestBigqueryAdapterSpecific(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "adapter-specific-models"
-
-    @property
-    def profile_config(self):
-        return self.bigquery_profile()
-
-    @property
-    def project_config(self):
-        return yaml.safe_load(textwrap.dedent('''\
-        config-version: 2
-        models:
-            test:
-                materialized: table
-                expiring_table:
-                    hours_to_expiration: 4    
-        '''))
-
-    @use_profile('bigquery')
-    def test_bigquery_hours_to_expiration(self):
-        _, stdout = self.run_dbt_and_capture(['--debug', 'run'])
-
-        self.assertIn(
-            'expiration_timestamp=TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL '
-            '4 hour)', stdout)
diff --git a/tests/integration/bigquery_test/test_bigquery_location_change.py b/tests/integration/bigquery_test/test_bigquery_location_change.py
deleted file mode 100644
index 02911f7d3..000000000
--- a/tests/integration/bigquery_test/test_bigquery_location_change.py
+++ /dev/null
@@ -1,31 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-import os
-
-
-class TestBigqueryErrorHandling(DBTIntegrationTest):
-    def setUp(self):
-        self.valid_location = os.getenv('DBT_TEST_BIGQUERY_INITIAL_LOCATION', 'US')
-        self.invalid_location = os.getenv('DBT_TEST_BIGQUERY_BAD_LOCATION', 'northamerica-northeast1')
-        self.location = self.valid_location
-        super().setUp()
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "location-models"
-
-    def bigquery_profile(self):
-        result = super().bigquery_profile()
-        result['test']['outputs']['default2']['location'] = self.location
-        return result
-
-    @use_profile('bigquery')
-    def test_bigquery_location_invalid(self):
-        self.run_dbt()
-        self.location = self.invalid_location
-        self.use_profile('bigquery')
-        _, stdout = self.run_dbt_and_capture(expect_pass=False)
-        assert 'Query Job SQL Follows' not in stdout

From 256acaa6909b4d548b0c65629d7b5231f81aa271 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Fri, 3 Feb 2023 10:20:45 -0600
Subject: [PATCH 544/860] update prerelease -> prekind (#512)

* update regex for version bump

* update to prekind

* more renaming
---
 .bumpversion.cfg | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 2f645d9e2..9fe96617b 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -13,13 +13,13 @@ parse = (?P<major>[\d]+) # major version number
 	\.?(?P<nightly>[a-z0-9]+\+[a-z]+)? # optional nightly release indicator
 	)?
 serialize =
-	{major}.{minor}.{patch}{prerelease}{num}.{nightly}
-	{major}.{minor}.{patch}{prerelease}{num}
+	{major}.{minor}.{patch}{prekind}{num}.{nightly}
+	{major}.{minor}.{patch}{prekind}{num}
 	{major}.{minor}.{patch}
 commit = False
 tag = False
 
-[bumpversion:part:prerelease]
+[bumpversion:part:prekind]
 first_value = a
 optional_value = final
 values =

From 4fc8efd88942358b618ceb5c87416b19a29f4ed4 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Fri, 3 Feb 2023 16:39:40 -0800
Subject: [PATCH 545/860] convert test_store_test_failures to functional test
 (#509)

* convert test_store_test_failures to functional test

* temp update dev reqs

* remove dev requirements override
---
 .../test_store_test_failures.py               |  9 +++
 .../models/fine_model.sql                     |  1 -
 ...odel_but_with_a_no_good_very_long_name.sql |  1 -
 .../models/problematic_model.sql              | 11 ----
 .../models/schema.yml                         | 40 -------------
 .../expected/expected_accepted_values.csv     |  3 -
 .../seeds/expected/expected_failing_test.csv  | 11 ----
 ...expected_not_null_problematic_model_id.csv |  3 -
 .../expected_unique_problematic_model_id.csv  |  3 -
 .../seeds/people.csv                          | 11 ----
 .../test_store_test_failures.py               | 60 -------------------
 .../tests/failing_test.sql                    |  1 -
 .../tests/passing_test.sql                    |  2 -
 13 files changed, 9 insertions(+), 147 deletions(-)
 create mode 100644 tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
 delete mode 100644 tests/integration/store_test_failures_tests/models/fine_model.sql
 delete mode 100644 tests/integration/store_test_failures_tests/models/fine_model_but_with_a_no_good_very_long_name.sql
 delete mode 100644 tests/integration/store_test_failures_tests/models/problematic_model.sql
 delete mode 100644 tests/integration/store_test_failures_tests/models/schema.yml
 delete mode 100644 tests/integration/store_test_failures_tests/seeds/expected/expected_accepted_values.csv
 delete mode 100644 tests/integration/store_test_failures_tests/seeds/expected/expected_failing_test.csv
 delete mode 100644 tests/integration/store_test_failures_tests/seeds/expected/expected_not_null_problematic_model_id.csv
 delete mode 100644 tests/integration/store_test_failures_tests/seeds/expected/expected_unique_problematic_model_id.csv
 delete mode 100644 tests/integration/store_test_failures_tests/seeds/people.csv
 delete mode 100644 tests/integration/store_test_failures_tests/test_store_test_failures.py
 delete mode 100644 tests/integration/store_test_failures_tests/tests/failing_test.sql
 delete mode 100644 tests/integration/store_test_failures_tests/tests/passing_test.sql

diff --git a/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py b/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
new file mode 100644
index 000000000..b04579f4b
--- /dev/null
+++ b/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
@@ -0,0 +1,9 @@
+import pytest
+
+from dbt.tests.adapter.store_test_failures_tests.test_store_test_failures import StoreTestFailuresBase
+
+
+class TestBigQueryStoreTestFailures(StoreTestFailuresBase):
+    def test_store_and_assert(self, project):
+        self.run_tests_store_one_failure(project)
+        self.run_tests_store_failures_and_assert(project)
diff --git a/tests/integration/store_test_failures_tests/models/fine_model.sql b/tests/integration/store_test_failures_tests/models/fine_model.sql
deleted file mode 100644
index 94b923a17..000000000
--- a/tests/integration/store_test_failures_tests/models/fine_model.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('people') }}
diff --git a/tests/integration/store_test_failures_tests/models/fine_model_but_with_a_no_good_very_long_name.sql b/tests/integration/store_test_failures_tests/models/fine_model_but_with_a_no_good_very_long_name.sql
deleted file mode 100644
index 97536ffaf..000000000
--- a/tests/integration/store_test_failures_tests/models/fine_model_but_with_a_no_good_very_long_name.sql
+++ /dev/null
@@ -1 +0,0 @@
-select 1 as quite_long_column_name
diff --git a/tests/integration/store_test_failures_tests/models/problematic_model.sql b/tests/integration/store_test_failures_tests/models/problematic_model.sql
deleted file mode 100644
index e780d6b00..000000000
--- a/tests/integration/store_test_failures_tests/models/problematic_model.sql
+++ /dev/null
@@ -1,11 +0,0 @@
-select * from {{ ref('people') }}
-
-union all
-
-select * from {{ ref('people') }}
-where id in (1,2)
-
-union all
-
-select null as id, first_name, last_name, email, gender, ip_address from {{ ref('people') }}
-where id in (3,4)
diff --git a/tests/integration/store_test_failures_tests/models/schema.yml b/tests/integration/store_test_failures_tests/models/schema.yml
deleted file mode 100644
index f01a9e350..000000000
--- a/tests/integration/store_test_failures_tests/models/schema.yml
+++ /dev/null
@@ -1,40 +0,0 @@
-version: 2
-
-models:
-
-  - name: fine_model
-    columns:
-      - name: id
-        tests:
-          - unique
-          - not_null
-
-  - name: problematic_model
-    columns:
-      - name: id
-        tests:
-          - unique:
-              store_failures: true
-          - not_null
-      - name: first_name
-        tests:
-          # test truncation of really long test name
-          - accepted_values:
-              values:
-                - Jack
-                - Kathryn
-                - Gerald
-                - Bonnie
-                - Harold
-                - Jacqueline
-                - Wanda
-                - Craig
-                # - Gary
-                # - Rose
-
-  - name: fine_model_but_with_a_no_good_very_long_name
-    columns:
-      - name: quite_long_column_name
-        tests:
-          # test truncation of really long test name with builtin
-          - unique
diff --git a/tests/integration/store_test_failures_tests/seeds/expected/expected_accepted_values.csv b/tests/integration/store_test_failures_tests/seeds/expected/expected_accepted_values.csv
deleted file mode 100644
index 02f28435b..000000000
--- a/tests/integration/store_test_failures_tests/seeds/expected/expected_accepted_values.csv
+++ /dev/null
@@ -1,3 +0,0 @@
-value_field,n_records
-Gary,1
-Rose,1
diff --git a/tests/integration/store_test_failures_tests/seeds/expected/expected_failing_test.csv b/tests/integration/store_test_failures_tests/seeds/expected/expected_failing_test.csv
deleted file mode 100644
index d9e7257f1..000000000
--- a/tests/integration/store_test_failures_tests/seeds/expected/expected_failing_test.csv
+++ /dev/null
@@ -1,11 +0,0 @@
-id,first_name,last_name,email,gender,ip_address
-1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
-2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
-3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
-4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
-5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
-6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220
-7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64
-8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13
-9,Gary,Day,gday8@nih.gov,Male,35.81.68.186
-10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100
diff --git a/tests/integration/store_test_failures_tests/seeds/expected/expected_not_null_problematic_model_id.csv b/tests/integration/store_test_failures_tests/seeds/expected/expected_not_null_problematic_model_id.csv
deleted file mode 100644
index 95fef8a25..000000000
--- a/tests/integration/store_test_failures_tests/seeds/expected/expected_not_null_problematic_model_id.csv
+++ /dev/null
@@ -1,3 +0,0 @@
-id,first_name,last_name,email,gender,ip_address
-,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
-,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
diff --git a/tests/integration/store_test_failures_tests/seeds/expected/expected_unique_problematic_model_id.csv b/tests/integration/store_test_failures_tests/seeds/expected/expected_unique_problematic_model_id.csv
deleted file mode 100644
index 431d54ef8..000000000
--- a/tests/integration/store_test_failures_tests/seeds/expected/expected_unique_problematic_model_id.csv
+++ /dev/null
@@ -1,3 +0,0 @@
-unique_field,n_records
-2,2
-1,2
\ No newline at end of file
diff --git a/tests/integration/store_test_failures_tests/seeds/people.csv b/tests/integration/store_test_failures_tests/seeds/people.csv
deleted file mode 100644
index d9e7257f1..000000000
--- a/tests/integration/store_test_failures_tests/seeds/people.csv
+++ /dev/null
@@ -1,11 +0,0 @@
-id,first_name,last_name,email,gender,ip_address
-1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
-2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
-3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
-4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
-5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
-6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220
-7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64
-8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13
-9,Gary,Day,gday8@nih.gov,Male,35.81.68.186
-10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100
diff --git a/tests/integration/store_test_failures_tests/test_store_test_failures.py b/tests/integration/store_test_failures_tests/test_store_test_failures.py
deleted file mode 100644
index e647c2de4..000000000
--- a/tests/integration/store_test_failures_tests/test_store_test_failures.py
+++ /dev/null
@@ -1,60 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-
-class TestStoreTestFailures(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "test_store_test_failures"
-
-    def tearDown(self):
-        test_audit_schema = self.unique_schema() + "_dbt_test__audit"
-        with self.adapter.connection_named('__test'):
-            self._drop_schema_named(self.default_database, test_audit_schema)
-
-        super().tearDown()
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        return {
-            "config-version": 2,
-            "test-paths": ["tests"],
-            "seeds": {
-                "quote_columns": False,
-                "test": {
-                    "expected": self.column_type_overrides()
-                },
-            },
-        }
-
-    def column_type_overrides(self):
-        return {}
-
-    def run_tests_store_failures_and_assert(self):
-        test_audit_schema = self.unique_schema() + "_dbt_test__audit"
-
-        self.run_dbt(["seed"])
-        self.run_dbt(["run"])
-        # make sure this works idempotently for all tests
-        self.run_dbt(["test", "--store-failures"], expect_pass=False)
-        results = self.run_dbt(["test", "--store-failures"], expect_pass=False)
-
-        # compare test results
-        actual = [(r.status, r.failures) for r in results]
-        expected = [('pass', 0), ('pass', 0), ('pass', 0), ('pass', 0),
-                    ('fail', 2), ('fail', 2), ('fail', 2), ('fail', 10)]
-        self.assertEqual(sorted(actual), sorted(expected))
-
-        # compare test results stored in database
-        self.assertTablesEqual("failing_test", "expected_failing_test", test_audit_schema)
-        self.assertTablesEqual("not_null_problematic_model_id", "expected_not_null_problematic_model_id", test_audit_schema)
-        self.assertTablesEqual("unique_problematic_model_id", "expected_unique_problematic_model_id", test_audit_schema)
-        self.assertTablesEqual("accepted_values_problematic_mo_c533ab4ca65c1a9dbf14f79ded49b628", "expected_accepted_values", test_audit_schema)
-
-class BigQueryTestStoreTestFailures(TestStoreTestFailures):
-    @use_profile('bigquery')
-    def test__bigquery__store_and_assert(self):
-        self.run_tests_store_failures_and_assert()
diff --git a/tests/integration/store_test_failures_tests/tests/failing_test.sql b/tests/integration/store_test_failures_tests/tests/failing_test.sql
deleted file mode 100644
index 1bb5ae5ba..000000000
--- a/tests/integration/store_test_failures_tests/tests/failing_test.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('fine_model') }}
diff --git a/tests/integration/store_test_failures_tests/tests/passing_test.sql b/tests/integration/store_test_failures_tests/tests/passing_test.sql
deleted file mode 100644
index 15c9a7a64..000000000
--- a/tests/integration/store_test_failures_tests/tests/passing_test.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-select * from {{ ref('fine_model') }}
-where false

From 1c2dd5731a3e039a31eb6194716e3ff90df7df1b Mon Sep 17 00:00:00 2001
From: Neelesh Salian <nssalian@users.noreply.github.com>
Date: Mon, 6 Feb 2023 17:56:48 -0800
Subject: [PATCH 546/860] [CT-1966]: convert upload_file test to a functional
 test (#517)

* convert upload file test

* Use utility for extracting columns
---
 .../upload_file_tests/data}/csv/source.csv    |   0
 .../data}/ndjson/source.ndjson                |   0
 .../data}/parquet/source.parquet              | Bin
 .../upload_file_tests/test_upload_file.py     | 101 ++++++++++++++++++
 .../upload_file_tests/macros/upload_file.sql  |   3 -
 .../upload_file_tests/test_upload_file.py     |  93 ----------------
 6 files changed, 101 insertions(+), 96 deletions(-)
 rename tests/{integration/upload_file_tests => functional/adapter/upload_file_tests/data}/csv/source.csv (100%)
 rename tests/{integration/upload_file_tests => functional/adapter/upload_file_tests/data}/ndjson/source.ndjson (100%)
 rename tests/{integration/upload_file_tests => functional/adapter/upload_file_tests/data}/parquet/source.parquet (100%)
 create mode 100644 tests/functional/adapter/upload_file_tests/test_upload_file.py
 delete mode 100644 tests/integration/upload_file_tests/macros/upload_file.sql
 delete mode 100644 tests/integration/upload_file_tests/test_upload_file.py

diff --git a/tests/integration/upload_file_tests/csv/source.csv b/tests/functional/adapter/upload_file_tests/data/csv/source.csv
similarity index 100%
rename from tests/integration/upload_file_tests/csv/source.csv
rename to tests/functional/adapter/upload_file_tests/data/csv/source.csv
diff --git a/tests/integration/upload_file_tests/ndjson/source.ndjson b/tests/functional/adapter/upload_file_tests/data/ndjson/source.ndjson
similarity index 100%
rename from tests/integration/upload_file_tests/ndjson/source.ndjson
rename to tests/functional/adapter/upload_file_tests/data/ndjson/source.ndjson
diff --git a/tests/integration/upload_file_tests/parquet/source.parquet b/tests/functional/adapter/upload_file_tests/data/parquet/source.parquet
similarity index 100%
rename from tests/integration/upload_file_tests/parquet/source.parquet
rename to tests/functional/adapter/upload_file_tests/data/parquet/source.parquet
diff --git a/tests/functional/adapter/upload_file_tests/test_upload_file.py b/tests/functional/adapter/upload_file_tests/test_upload_file.py
new file mode 100644
index 000000000..2393d4225
--- /dev/null
+++ b/tests/functional/adapter/upload_file_tests/test_upload_file.py
@@ -0,0 +1,101 @@
+import pytest
+from dbt.tests.util import (
+    get_relation_columns,
+    run_dbt,
+    run_sql_with_adapter
+)
+import datetime
+import yaml
+
+_UPLOAD_FILE_SQL = """
+{% macro upload_file(local_file_path, database, table_schema, table_name) %}
+  {% do adapter.upload_file(local_file_path, database, table_schema, table_name, kwargs=kwargs) %}
+{% endmacro %}
+""".lstrip()
+
+
+class TestUploadFile:
+    @pytest.fixture(scope="class")
+    def schema(self):
+        return "upload_file"
+
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {
+            "upload_file.sql": _UPLOAD_FILE_SQL,
+        }
+
+    @staticmethod
+    def perform_uploaded_table_checks(table_schema, table_name, project):
+        # Test the column names, and data types of the created table
+        col_result = get_relation_columns(project.adapter, f"{table_schema}.{table_name}")
+        assert [col_obj[0] for col_obj in col_result] == ['email', 'favorite_color', 'first_name', 'id', 'ip_address',
+                                                          'updated_at']
+        assert [col_obj[1] for col_obj in col_result] == ['STRING', 'STRING', 'STRING', 'INT64', 'STRING', 'TIMESTAMP']
+
+        # Test the values of the created table
+        value_query = f"""
+            select
+                count(*) row_count,
+                count(distinct id) as num_distinct_ids,
+                max(updated_at) as max_updated_at
+            from `{table_schema}.{table_name}`
+        """
+        value_results = run_sql_with_adapter(project.adapter, value_query)
+
+        # There should be 100 rows in this table
+        assert value_results[0][0] == 100
+        # There should be 100 distinct id values in this table
+        assert value_results[0][1] == 100
+        # Maximum updated_at value should be 2016-09-19 14:45:51
+        assert value_results[0][2] == datetime.datetime(2016, 9, 19, 14, 45, 51, tzinfo=datetime.timezone.utc)
+
+    def test_bigquery_upload_file_csv(self, project):
+        # Create a table from an uploaded CSV file
+        upload_args = yaml.safe_dump({
+            'local_file_path': f"{project.test_data_dir}/csv/source.csv",
+            'database': project.database,
+            'table_schema': project.test_schema,
+            'table_name': 'TestUploadFileCSV',
+            'skip_leading_rows': 1,
+            'autodetect': True,
+            'write_disposition': 'WRITE_TRUNCATE'
+        })
+        upload_result = run_dbt(['run-operation', 'upload_file', '--args', upload_args])
+        assert upload_result.success
+
+        # Check if the uploaded table contains expected values and schema
+        self.perform_uploaded_table_checks(project.test_schema, 'TestUploadFileCSV', project)
+
+    def test_bigquery_upload_file_ndjson(self, project):
+        # Create a table from an uploaded NDJSON file
+        upload_args = yaml.safe_dump({
+            'local_file_path': f"{project.test_data_dir}/ndjson/source.ndjson",
+            'database': project.database,
+            'table_schema': project.test_schema,
+            'table_name': 'TestUploadFileNDJSON',
+            'autodetect': True,
+            'source_format': 'NEWLINE_DELIMITED_JSON',
+            'write_disposition': 'WRITE_TRUNCATE'
+        })
+        upload_result = run_dbt(['run-operation', 'upload_file', '--args', upload_args])
+        assert upload_result.success
+
+        # Check if the uploaded table contains expected values and schema
+        self.perform_uploaded_table_checks(project.test_schema, 'TestUploadFileNDJSON', project)
+
+    def test_bigquery_upload_file_parquet(self, project):
+        # Create a table from an uploaded parquet file
+        upload_args = yaml.safe_dump({
+            'local_file_path': f"{project.test_data_dir}/parquet/source.parquet",
+            'database': project.database,
+            'table_schema': project.test_schema,
+            'table_name': 'TestUploadFileParquet',
+            'source_format': 'PARQUET',
+            'write_disposition': 'WRITE_TRUNCATE'
+        })
+        upload_result = run_dbt(['run-operation', 'upload_file', '--args', upload_args])
+        assert upload_result.success
+
+        # Check if the uploaded table contains expected values and schema
+        self.perform_uploaded_table_checks(project.test_schema, 'TestUploadFileParquet', project)
diff --git a/tests/integration/upload_file_tests/macros/upload_file.sql b/tests/integration/upload_file_tests/macros/upload_file.sql
deleted file mode 100644
index 5436170f3..000000000
--- a/tests/integration/upload_file_tests/macros/upload_file.sql
+++ /dev/null
@@ -1,3 +0,0 @@
-{% macro upload_file(local_file_path, database, table_schema, table_name) %}
-  {% do adapter.upload_file(local_file_path, database, table_schema, table_name, kwargs=kwargs) %}
-{% endmacro %}
\ No newline at end of file
diff --git a/tests/integration/upload_file_tests/test_upload_file.py b/tests/integration/upload_file_tests/test_upload_file.py
deleted file mode 100644
index a879fe2bf..000000000
--- a/tests/integration/upload_file_tests/test_upload_file.py
+++ /dev/null
@@ -1,93 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-import datetime
-import yaml
-
-
-class TestUploadFile(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "upload_file"
-
-    @property
-    def models(self):
-        return "models"
-        
-    def perform_uploaded_table_checks(self, table_schema, table_name):
-        # Test the data types of the created table
-        query = f"""
-            select
-                column_name,
-                data_type
-            from `{table_schema}.INFORMATION_SCHEMA.COLUMNS`
-            where
-                table_name = '{table_name}'
-            order by column_name
-        """
-        results = self.run_sql(query, fetch='all')
-        self.assertTrue([row[1] for row in results] == ['STRING', 'STRING', 'STRING', 'INT64', 'STRING', 'TIMESTAMP'], 'The table should consist of columns of types: STRING, STRING, STRING, INT64, STRING, TIMESTAMP')
-        
-        # Test the values of the created table
-        query = f"""
-            select
-                count(*) row_count,
-                count(distinct id) as num_distinct_ids,
-                max(updated_at) as max_updated_at
-            from `{table_schema}.{table_name}`
-        """
-        results = self.run_sql(query, fetch='all')
-        self.assertTrue(results[0][0] == 100, 'There should be 100 rows in this table')
-        self.assertTrue(results[0][1] == 100, 'There should be 100 distinct id values in this table')
-        self.assertTrue(results[0][2] == datetime.datetime(2016, 9, 19, 14, 45, 51, tzinfo=datetime.timezone.utc), 'Maximum updated_at value should be 2016-09-19 14:45:51')
-
-    @use_profile('bigquery')
-    def test_bigquery_upload_file_csv(self):
-        # Create a table from an uploaded CSV file
-        upload_args = yaml.safe_dump({
-            'local_file_path': './csv/source.csv',
-            'database': self.default_database,
-            'table_schema': self.unique_schema(),
-            'table_name': 'TestUploadFileCSV',
-            'skip_leading_rows': 1,
-            'autodetect': True,
-            'write_disposition': 'WRITE_TRUNCATE'
-        })
-        result = self.run_dbt(['run-operation', 'upload_file', '--args', upload_args])
-        self.assertTrue(result.success)
-
-        # Check uploaded table contains expected values and schema
-        self.perform_uploaded_table_checks(self.unique_schema(), 'TestUploadFileCSV')
-
-    @use_profile('bigquery')
-    def test_bigquery_upload_file_ndjson(self):
-        # Create a table from an uploaded NDJSON file
-        upload_args = yaml.safe_dump({
-            'local_file_path': './ndjson/source.ndjson',
-            'database': self.default_database,
-            'table_schema': self.unique_schema(),
-            'table_name': 'TestUploadFileNDJSON',
-            'autodetect': True,
-            'source_format': 'NEWLINE_DELIMITED_JSON',
-            'write_disposition': 'WRITE_TRUNCATE'
-        })
-        result = self.run_dbt(['run-operation', 'upload_file', '--args', upload_args])
-        self.assertTrue(result.success)
-
-        # Check uploaded table contains expected values and schema
-        self.perform_uploaded_table_checks(self.unique_schema(), 'TestUploadFileNDJSON')
-
-    @use_profile('bigquery')
-    def test_bigquery_upload_file_parquet(self):
-        # Create a table from an uploaded parquet file
-        upload_args = yaml.safe_dump({
-            'local_file_path': './parquet/source.parquet',
-            'database': self.default_database,
-            'table_schema': self.unique_schema(),
-            'table_name': 'TestUploadFileParquet',
-            'source_format': 'PARQUET',
-            'write_disposition': 'WRITE_TRUNCATE'
-        })
-        result = self.run_dbt(['run-operation', 'upload_file', '--args', upload_args])
-        self.assertTrue(result.success)
-
-        # Check uploaded table contains expected values and schema
-        self.perform_uploaded_table_checks(self.unique_schema(), 'TestUploadFileParquet')

From a3c64a2fbc97702e1c9565096ae6432692ff7580 Mon Sep 17 00:00:00 2001
From: Neelesh Salian <nssalian@users.noreply.github.com>
Date: Tue, 7 Feb 2023 14:11:42 -0800
Subject: [PATCH 547/860] [CT-2030]: Convert simple seed to functional test
 (#514)

* Convert simple seed to functional test

* Clean up

* Remove macros path

* Renamed to schema_yml

* Remove branch from dev-req

* Inherit from SeedConfigBase
---
 tests/functional/adapter/test_simple_seed.py  | 166 ++++++++++++++++++
 .../simple_seed_test/macros/schema_test.sql   |  22 ---
 .../simple_seed_test/models-bq/schema.yml     |  54 ------
 .../seeds-config/seed_configs.csv             |   3 -
 .../seeds-config/seed_disabled.csv            |  21 ---
 .../seeds-config/seed_enabled.csv             |  21 ---
 .../seeds-config/seed_tricky.csv              |   7 -
 .../simple_seed_test/test_seed_configs.py     |  90 ----------
 8 files changed, 166 insertions(+), 218 deletions(-)
 create mode 100644 tests/functional/adapter/test_simple_seed.py
 delete mode 100644 tests/integration/simple_seed_test/macros/schema_test.sql
 delete mode 100644 tests/integration/simple_seed_test/models-bq/schema.yml
 delete mode 100644 tests/integration/simple_seed_test/seeds-config/seed_configs.csv
 delete mode 100644 tests/integration/simple_seed_test/seeds-config/seed_disabled.csv
 delete mode 100644 tests/integration/simple_seed_test/seeds-config/seed_enabled.csv
 delete mode 100644 tests/integration/simple_seed_test/seeds-config/seed_tricky.csv
 delete mode 100644 tests/integration/simple_seed_test/test_seed_configs.py

diff --git a/tests/functional/adapter/test_simple_seed.py b/tests/functional/adapter/test_simple_seed.py
new file mode 100644
index 000000000..b803e66d4
--- /dev/null
+++ b/tests/functional/adapter/test_simple_seed.py
@@ -0,0 +1,166 @@
+import pytest
+from dbt.tests.adapter.simple_seed.fixtures import macros__schema_test
+from dbt.tests.adapter.simple_seed.seeds import (
+    seeds__enabled_in_config_csv,
+    seeds__tricky_csv
+)
+from dbt.tests.adapter.simple_seed.test_seed import SeedConfigBase
+from dbt.tests.adapter.utils.base_utils import run_dbt
+
+
+_SEED_CONFIGS_CSV = """
+seed_id,stuff
+1,a
+2,b
+""".lstrip()
+
+_SCHEMA_YML = """
+version: 2
+seeds:
+- name: seed_enabled
+  columns:
+  - name: birthday
+    tests:
+    - column_type:
+        type: STRING
+  - name: seed_id
+    tests:
+    - column_type:
+        type: FLOAT64
+
+- name: seed_tricky
+  columns:
+  - name: seed_id
+    tests:
+    - column_type:
+        type: INT64
+  - name: seed_id_str
+    tests:
+    - column_type:
+        type: STRING
+  - name: a_bool
+    tests:
+    - column_type:
+        type: BOOLEAN
+  - name: looks_like_a_bool
+    tests:
+    - column_type:
+        type: STRING
+  - name: a_date
+    tests:
+    - column_type:
+        type: DATETIME
+  - name: looks_like_a_date
+    tests:
+    - column_type:
+        type: STRING
+  - name: relative
+    tests:
+    - column_type:
+        type: STRING
+  - name: weekday
+    tests:
+    - column_type:
+        type: STRING
+
+- name: seed_configs
+  config:
+    hours_to_expiration: 2
+    labels:
+      contains_pii: 'yes'
+      contains_pie: 'no'
+""".lstrip()
+
+
+class TestSimpleSeedConfigs(SeedConfigBase):
+    @pytest.fixture(scope="class")
+    def schema(self):
+        return "simple_seed"
+
+    @pytest.fixture(scope="class")
+    def seeds(self):
+        return {
+            "seed_enabled.csv": seeds__enabled_in_config_csv,
+            "seed_tricky.csv": seeds__tricky_csv,
+            "seed_configs.csv": _SEED_CONFIGS_CSV
+        }
+
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {
+            "schema_test.sql": macros__schema_test,
+        }
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "models-bq.yml": _SCHEMA_YML
+        }
+
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            'config-version': 2,
+            'seeds': {
+                'test': {
+                    'enabled': False,
+                    'quote_columns': True,
+                    'seed_enabled': {
+                        'enabled': True,
+                        '+column_types': self.seed_enabled_types(),
+                    },
+                    'seed_tricky': {
+                        'enabled': True,
+                        '+column_types': self.seed_tricky_types(),
+                    },
+                    'seed_configs': {
+                        'enabled': True,
+                    },
+                },
+            },
+        }
+
+    @staticmethod
+    def seed_enabled_types():
+        return {
+            "seed_id": "FLOAT64",
+            "birthday": "STRING",
+        }
+
+    @staticmethod
+    def seed_tricky_types():
+        return {
+            'seed_id_str': 'STRING',
+            'looks_like_a_bool': 'STRING',
+            'looks_like_a_date': 'STRING',
+        }
+
+    @staticmethod
+    def table_labels():
+        return {
+            "contains_pii": "yes",
+            "contains_pie": "no"
+        }
+
+    def test__bigquery_simple_seed_with_column_override_bigquery(self, project):
+        seed_results = run_dbt(["seed"])
+        assert len(seed_results) == 3
+        test_results = run_dbt(["test"])
+        assert len(test_results) == 10
+
+    def test__bigquery_seed_table_with_labels_config_bigquery(self, project):
+        seed_results = run_dbt(["seed"])
+        assert len(seed_results) == 3
+        with project.adapter.connection_named('_test'):
+            client = project.adapter.connections\
+                .get_thread_connection().handle
+            table_id = "{}.{}.{}".format(
+                project.database,
+                project.test_schema,
+                'seed_configs'
+            )
+            bq_table = client.get_table(table_id)
+
+            assert bq_table.labels
+            assert bq_table.labels == self.table_labels()
+            assert bq_table.expires
diff --git a/tests/integration/simple_seed_test/macros/schema_test.sql b/tests/integration/simple_seed_test/macros/schema_test.sql
deleted file mode 100644
index 5c7f25964..000000000
--- a/tests/integration/simple_seed_test/macros/schema_test.sql
+++ /dev/null
@@ -1,22 +0,0 @@
-
-{% test column_type(model, column_name, type) %}
-
-    {% set cols = adapter.get_columns_in_relation(model) %}
-
-    {% set col_types = {} %}
-    {% for col in cols %}
-        {% do col_types.update({col.name: col.data_type}) %}
-    {% endfor %}
-
-    {% set validation_message = 'Got a column type of ' ~ col_types.get(column_name) ~ ', expected ' ~ type %}
-
-    {% set val = 0 if col_types.get(column_name) == type else 1 %}
-    {% if val == 1 and execute %}
-        {{ log(validation_message, info=True) }}
-    {% endif %}
-
-    select '{{ validation_message }}' as validation_error
-    from (select true) as nothing
-    where {{ val }} = 1
-
-{% endtest %}
diff --git a/tests/integration/simple_seed_test/models-bq/schema.yml b/tests/integration/simple_seed_test/models-bq/schema.yml
deleted file mode 100644
index 606794ce5..000000000
--- a/tests/integration/simple_seed_test/models-bq/schema.yml
+++ /dev/null
@@ -1,54 +0,0 @@
-version: 2
-seeds:
-- name: seed_enabled
-  columns:
-  - name: birthday
-    tests:
-    - column_type:
-        type: STRING
-  - name: id
-    tests:
-    - column_type:
-        type: FLOAT64
-
-- name: seed_tricky
-  columns:
-  - name: id
-    tests:
-    - column_type:
-        type: INT64
-  - name: id_str
-    tests:
-    - column_type:
-        type: STRING
-  - name: a_bool
-    tests:
-    - column_type:
-        type: BOOLEAN
-  - name: looks_like_a_bool
-    tests:
-    - column_type:
-        type: STRING
-  - name: a_date
-    tests:
-    - column_type:
-        type: DATETIME
-  - name: looks_like_a_date
-    tests:
-    - column_type:
-        type: STRING
-  - name: relative
-    tests:
-    - column_type:
-        type: STRING
-  - name: weekday
-    tests:
-    - column_type:
-        type: STRING
-
-- name: seed_configs
-  config:
-    hours_to_expiration: 2
-    labels:
-      contains_pii: 'yes'
-      contains_pie: 'no'
diff --git a/tests/integration/simple_seed_test/seeds-config/seed_configs.csv b/tests/integration/simple_seed_test/seeds-config/seed_configs.csv
deleted file mode 100644
index a0d796378..000000000
--- a/tests/integration/simple_seed_test/seeds-config/seed_configs.csv
+++ /dev/null
@@ -1,3 +0,0 @@
-id,stuff
-1,a
-2,b
diff --git a/tests/integration/simple_seed_test/seeds-config/seed_disabled.csv b/tests/integration/simple_seed_test/seeds-config/seed_disabled.csv
deleted file mode 100644
index 0227dd609..000000000
--- a/tests/integration/simple_seed_test/seeds-config/seed_disabled.csv
+++ /dev/null
@@ -1,21 +0,0 @@
-id,first_name,email,ip_address,birthday
-1,Larry,lking0@miitbeian.gov.cn,69.135.206.194,2008-09-12 19:08:31
-2,Larry,lperkins1@toplist.cz,64.210.133.162,1978-05-09 04:15:14
-3,Anna,amontgomery2@miitbeian.gov.cn,168.104.64.114,2011-10-16 04:07:57
-4,Sandra,sgeorge3@livejournal.com,229.235.252.98,1973-07-19 10:52:43
-5,Fred,fwoods4@google.cn,78.229.170.124,2012-09-30 16:38:29
-6,Stephen,shanson5@livejournal.com,182.227.157.105,1995-11-07 21:40:50
-7,William,wmartinez6@upenn.edu,135.139.249.50,1982-09-05 03:11:59
-8,Jessica,jlong7@hao123.com,203.62.178.210,1991-10-16 11:03:15
-9,Douglas,dwhite8@tamu.edu,178.187.247.1,1979-10-01 09:49:48
-10,Lisa,lcoleman9@nydailynews.com,168.234.128.249,2011-05-26 07:45:49
-11,Ralph,rfieldsa@home.pl,55.152.163.149,1972-11-18 19:06:11
-12,Louise,lnicholsb@samsung.com,141.116.153.154,2014-11-25 20:56:14
-13,Clarence,cduncanc@sfgate.com,81.171.31.133,2011-11-17 07:02:36
-14,Daniel,dfranklind@omniture.com,8.204.211.37,1980-09-13 00:09:04
-15,Katherine,klanee@auda.org.au,176.96.134.59,1997-08-22 19:36:56
-16,Billy,bwardf@wikia.com,214.108.78.85,2003-10-19 02:14:47
-17,Annie,agarzag@ocn.ne.jp,190.108.42.70,1988-10-28 15:12:35
-18,Shirley,scolemanh@fastcompany.com,109.251.164.84,1988-08-24 10:50:57
-19,Roger,rfrazieri@scribd.com,38.145.218.108,1985-12-31 15:17:15
-20,Lillian,lstanleyj@goodreads.com,47.57.236.17,1970-06-08 02:09:05
diff --git a/tests/integration/simple_seed_test/seeds-config/seed_enabled.csv b/tests/integration/simple_seed_test/seeds-config/seed_enabled.csv
deleted file mode 100644
index 0227dd609..000000000
--- a/tests/integration/simple_seed_test/seeds-config/seed_enabled.csv
+++ /dev/null
@@ -1,21 +0,0 @@
-id,first_name,email,ip_address,birthday
-1,Larry,lking0@miitbeian.gov.cn,69.135.206.194,2008-09-12 19:08:31
-2,Larry,lperkins1@toplist.cz,64.210.133.162,1978-05-09 04:15:14
-3,Anna,amontgomery2@miitbeian.gov.cn,168.104.64.114,2011-10-16 04:07:57
-4,Sandra,sgeorge3@livejournal.com,229.235.252.98,1973-07-19 10:52:43
-5,Fred,fwoods4@google.cn,78.229.170.124,2012-09-30 16:38:29
-6,Stephen,shanson5@livejournal.com,182.227.157.105,1995-11-07 21:40:50
-7,William,wmartinez6@upenn.edu,135.139.249.50,1982-09-05 03:11:59
-8,Jessica,jlong7@hao123.com,203.62.178.210,1991-10-16 11:03:15
-9,Douglas,dwhite8@tamu.edu,178.187.247.1,1979-10-01 09:49:48
-10,Lisa,lcoleman9@nydailynews.com,168.234.128.249,2011-05-26 07:45:49
-11,Ralph,rfieldsa@home.pl,55.152.163.149,1972-11-18 19:06:11
-12,Louise,lnicholsb@samsung.com,141.116.153.154,2014-11-25 20:56:14
-13,Clarence,cduncanc@sfgate.com,81.171.31.133,2011-11-17 07:02:36
-14,Daniel,dfranklind@omniture.com,8.204.211.37,1980-09-13 00:09:04
-15,Katherine,klanee@auda.org.au,176.96.134.59,1997-08-22 19:36:56
-16,Billy,bwardf@wikia.com,214.108.78.85,2003-10-19 02:14:47
-17,Annie,agarzag@ocn.ne.jp,190.108.42.70,1988-10-28 15:12:35
-18,Shirley,scolemanh@fastcompany.com,109.251.164.84,1988-08-24 10:50:57
-19,Roger,rfrazieri@scribd.com,38.145.218.108,1985-12-31 15:17:15
-20,Lillian,lstanleyj@goodreads.com,47.57.236.17,1970-06-08 02:09:05
diff --git a/tests/integration/simple_seed_test/seeds-config/seed_tricky.csv b/tests/integration/simple_seed_test/seeds-config/seed_tricky.csv
deleted file mode 100644
index 3e90a18e3..000000000
--- a/tests/integration/simple_seed_test/seeds-config/seed_tricky.csv
+++ /dev/null
@@ -1,7 +0,0 @@
-id,id_str,a_bool,looks_like_a_bool,a_date,looks_like_a_date,relative,weekday
-1,1,true,true,2019-01-01 12:32:30,2019-01-01 12:32:30,tomorrow,Saturday
-2,2,True,True,2019-01-01 12:32:31,2019-01-01 12:32:31,today,Sunday
-3,3,TRUE,TRUE,2019-01-01 12:32:32,2019-01-01 12:32:32,yesterday,Monday
-4,4,false,false,2019-01-01 01:32:32,2019-01-01 01:32:32,tomorrow,Saturday
-5,5,False,False,2019-01-01 01:32:32,2019-01-01 01:32:32,today,Sunday
-6,6,FALSE,FALSE,2019-01-01 01:32:32,2019-01-01 01:32:32,yesterday,Monday
diff --git a/tests/integration/simple_seed_test/test_seed_configs.py b/tests/integration/simple_seed_test/test_seed_configs.py
deleted file mode 100644
index 4dc9da9e1..000000000
--- a/tests/integration/simple_seed_test/test_seed_configs.py
+++ /dev/null
@@ -1,90 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-class TestSimpleSeedConfigs(DBTIntegrationTest):
-    run_once = False
-
-    @property
-    def schema(self):
-        return "simple_seed"
-
-    @property
-    def models(self):
-        return "models-bq"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds-config'],
-            'macro-paths': ['macros'],
-            'seeds': {
-                'test': {
-                    'enabled': False,
-                    'quote_columns': True,
-                    'seed_enabled': {
-                        'enabled': True,
-                        '+column_types': self.seed_enabled_types(),
-                    },
-                    'seed_tricky': {
-                        'enabled': True,
-                        '+column_types': self.seed_tricky_types(),
-                    },
-                    'seed_configs': {
-                        'enabled': True,
-                    },
-                },
-            },
-        }
-
-    @property
-    def profile_config(self):
-        return self.bigquery_profile()
-
-    def seed_enabled_types(self):
-        return {
-            "id": "FLOAT64",
-            "birthday": "STRING",
-        }
-
-    def seed_tricky_types(self):
-        return {
-            'id_str': 'STRING',
-            'looks_like_a_bool': 'STRING',
-            'looks_like_a_date': 'STRING',
-        }
-    
-    def seed_data(self):
-        if self.run_once:
-            return
-        results = self.run_dbt(["seed", "--show"])
-        self.assertEqual(len(results),  3)
-        self.run_once = True
-
-    @property
-    def table_labels(self):
-        return {
-            'contains_pii': 'yes',
-            'contains_pie': 'no'
-        }
-
-    @use_profile('bigquery')
-    def test__bigquery_simple_seed_with_column_override_bigquery(self):
-        self.seed_data()
-        results = self.run_dbt(["test"])
-        self.assertEqual(len(results),  10)
-
-    @use_profile('bigquery')
-    def test__bigquery_seed_table_with_labels_config_bigquery(self):
-        self.seed_data()
-        with self.get_connection() as conn:
-            client = conn.handle
-
-            table = client.get_table(
-                self.adapter.connections.get_bq_table(
-                    self.default_database, self.unique_schema(), 'seed_configs')
-            )
-
-            self.assertTrue(table.labels)
-            self.assertEquals(table.labels, self.table_labels)
-            self.assertTrue(table.expires)
-

From 4e72e92127bd13dbd567ec115baca40f784ead6e Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Tue, 7 Feb 2023 14:39:05 -0800
Subject: [PATCH 548/860] Remove the schema tests test. (#519)

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .../models-v2/bq-models/ephemeral_copy.sql    |  8 ---
 .../models-v2/bq-models/schema.yml            | 38 -------------
 .../models-v2/bq-models/table_summary.sql     |  9 ----
 .../schema_tests_test/models-v2/seed/seed.csv |  5 --
 .../schema_tests_test/test_schema_v2_tests.py | 53 -------------------
 5 files changed, 113 deletions(-)
 delete mode 100644 tests/integration/schema_tests_test/models-v2/bq-models/ephemeral_copy.sql
 delete mode 100644 tests/integration/schema_tests_test/models-v2/bq-models/schema.yml
 delete mode 100644 tests/integration/schema_tests_test/models-v2/bq-models/table_summary.sql
 delete mode 100644 tests/integration/schema_tests_test/models-v2/seed/seed.csv
 delete mode 100644 tests/integration/schema_tests_test/test_schema_v2_tests.py

diff --git a/tests/integration/schema_tests_test/models-v2/bq-models/ephemeral_copy.sql b/tests/integration/schema_tests_test/models-v2/bq-models/ephemeral_copy.sql
deleted file mode 100644
index 73c62165e..000000000
--- a/tests/integration/schema_tests_test/models-v2/bq-models/ephemeral_copy.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-
-{{
-    config(
-        materialized='ephemeral'
-    )
-}}
-
-select * from {{ this.schema }}.seed
diff --git a/tests/integration/schema_tests_test/models-v2/bq-models/schema.yml b/tests/integration/schema_tests_test/models-v2/bq-models/schema.yml
deleted file mode 100644
index 16d3e155a..000000000
--- a/tests/integration/schema_tests_test/models-v2/bq-models/schema.yml
+++ /dev/null
@@ -1,38 +0,0 @@
-version: 2
-
-models:
-  - name: ephemeral_copy
-    description: "An ephemeral copy of the table"
-    columns:
-      - name: id
-        description: "The ID"
-        tests:
-          - not_null
-          - unique
-      - name: favorite_color
-        tests:
-          - accepted_values: { values: ['blue', 'green'] }
-      - name: ephemeral_copy
-        description: "Testing a column with the same name as the model"
-        tests:
-          - not_null
-          - unique
-
-  # this whole model should pass and run
-  - name: table_summary
-    description: "The summary table"
-    tests: 
-     - unique:
-          column_name: "concat(favorite_color_copy, count)"
-    columns:
-      - name: favorite_color_copy
-        description: "The favorite color"
-        tests:
-          - not_null
-          - unique
-          - accepted_values: { values: ['blue', 'green'] }
-          - relationships: { field: favorite_color, to: ref('ephemeral_copy') }
-      - name: count
-        description: "The number of responses for this favorite color"
-        tests:
-          - not_null
diff --git a/tests/integration/schema_tests_test/models-v2/bq-models/table_summary.sql b/tests/integration/schema_tests_test/models-v2/bq-models/table_summary.sql
deleted file mode 100644
index 845ce3d58..000000000
--- a/tests/integration/schema_tests_test/models-v2/bq-models/table_summary.sql
+++ /dev/null
@@ -1,9 +0,0 @@
-{{
-    config(
-        materialized='table'
-    )
-}}
-
-select favorite_color as favorite_color_copy, count(*) as count
-from {{ ref('ephemeral_copy') }}
-group by 1
diff --git a/tests/integration/schema_tests_test/models-v2/seed/seed.csv b/tests/integration/schema_tests_test/models-v2/seed/seed.csv
deleted file mode 100644
index 77fa7af00..000000000
--- a/tests/integration/schema_tests_test/models-v2/seed/seed.csv
+++ /dev/null
@@ -1,5 +0,0 @@
-favorite_color,id,first_name,email,ip_address,updated_at,ephemeral_copy
-blue,1,Larry,null,69.135.206.194,2008-09-12 19:08:31,1000
-blue,2,Larry,null,64.210.133.162,1978-05-09 04:15:14,2000
-green,99,Paul,pjohnson2q@umn.edu,183.59.198.197,1991-11-14 12:33:55,3000
-green,100,Frank,fgreene2r@blogspot.com,150.143.68.121,2010-06-12 23:55:39,4000
diff --git a/tests/integration/schema_tests_test/test_schema_v2_tests.py b/tests/integration/schema_tests_test/test_schema_v2_tests.py
deleted file mode 100644
index 7b213db8a..000000000
--- a/tests/integration/schema_tests_test/test_schema_v2_tests.py
+++ /dev/null
@@ -1,53 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, FakeArgs, use_profile
-import os
-
-from dbt.task.test import TestTask
-
-class TestBQSchemaTests(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "schema_tests"
-
-    @property
-    def models(self):
-        return "models-v2/bq-models"
-
-    @staticmethod
-    def dir(path):
-        return os.path.normpath(
-            os.path.join('models-v2', path))
-
-    def run_schema_validations(self):
-        args = FakeArgs()
-
-        test_task = TestTask(args, self.config)
-        return test_task.run()
-
-    @use_profile('bigquery')
-    def test_schema_tests_bigquery(self):
-        self.use_default_project({'seed-paths': [self.dir('seed')]})
-        self.assertEqual(len(self.run_dbt(['seed'])), 1)
-        results = self.run_dbt()
-        self.assertEqual(len(results), 1)
-        test_results = self.run_schema_validations()
-        self.assertEqual(len(test_results), 11)
-
-        for result in test_results:
-            # assert that all deliberately failing tests actually fail
-            if 'failure' in result.node.name:
-                self.assertEqual(result.status, 'fail')
-                self.assertFalse(result.skipped)
-                self.assertTrue(
-                    result.failures > 0,
-                    'test {} did not fail'.format(result.node.name)
-                )
-            # assert that actual tests pass
-            else:
-                self.assertEqual(result.status, 'pass')
-                self.assertFalse(result.skipped)
-                self.assertEqual(
-                    result.failures, 0,
-                    'test {} failed'.format(result.node.name)
-                )
-
-        self.assertEqual(sum(x.failures for x in test_results), 0)

From db6b527cbc6fc06c9c96f7253d301b70185be37e Mon Sep 17 00:00:00 2001
From: dave-connors-3 <73915542+dave-connors-3@users.noreply.github.com>
Date: Wed, 8 Feb 2023 15:12:02 -0600
Subject: [PATCH 549/860] Feature/merge exclude columns (#227)

* add new seeds, extend incremental merge test class

* fix issue in seed

* pin core branch for integration tests

* changie entry

* pin core branch

* add adapter test

* pin core branch for tests

* remove integration tests

* remove commented line

* unpin core dev branch

---------

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .changes/unreleased/Features-20220826-115320.yaml         | 7 +++++++
 .../incremental/test_incremental_merge_exclude_columns.py | 8 ++++++++
 tests/integration/simple_copy_test/test_simple_copy.py    | 3 +--
 3 files changed, 16 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Features-20220826-115320.yaml
 create mode 100644 tests/functional/adapter/incremental/test_incremental_merge_exclude_columns.py

diff --git a/.changes/unreleased/Features-20220826-115320.yaml b/.changes/unreleased/Features-20220826-115320.yaml
new file mode 100644
index 000000000..27bca84fc
--- /dev/null
+++ b/.changes/unreleased/Features-20220826-115320.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: add option to specify excluded columns on merge
+time: 2022-08-26T11:53:20.194981-05:00
+custom:
+  Author: dave-connors-3
+  Issue: "5260"
+  PR: "227"
diff --git a/tests/functional/adapter/incremental/test_incremental_merge_exclude_columns.py b/tests/functional/adapter/incremental/test_incremental_merge_exclude_columns.py
new file mode 100644
index 000000000..c8c326004
--- /dev/null
+++ b/tests/functional/adapter/incremental/test_incremental_merge_exclude_columns.py
@@ -0,0 +1,8 @@
+import pytest
+
+from dbt.tests.util import run_dbt
+from dbt.tests.adapter.incremental.test_incremental_merge_exclude_columns import BaseMergeExcludeColumns
+
+class TestMergeExcludeColumns(BaseMergeExcludeColumns):
+    pass
+
diff --git a/tests/integration/simple_copy_test/test_simple_copy.py b/tests/integration/simple_copy_test/test_simple_copy.py
index 45a9838e7..47c7665c8 100644
--- a/tests/integration/simple_copy_test/test_simple_copy.py
+++ b/tests/integration/simple_copy_test/test_simple_copy.py
@@ -93,5 +93,4 @@ def test__bigquery__incremental_merge_columns(self):
             "seed-paths": ["seeds-merge-cols-update"]
         })
         self.seed_and_run()
-        self.assertTablesEqual("incremental_update_cols", "expected_result")
-
+        self.assertTablesEqual("incremental_update_cols", "expected_result")
\ No newline at end of file

From 1179eb2e128af4ef47e37e798ae53f3ecb5fd0b8 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Wed, 8 Feb 2023 15:23:30 -0800
Subject: [PATCH 550/860] add dataset update lock (#404)

* add dataset update lock

* make dataset lock global

* add changie, clean up

* add debug logging to experiment

* further experiment

* add unit test

* fix dataset_ref

* fix grant access idempotency

* add unit test and fix imports

* make mypy happy

* further mypy fixes

* add double assert

* use simpler dict comparison and test failure

* add role / entity match back to access entry equality check

* extra space nit

* simplify test assertion

* temp update dev reqs

* fix dev-requirements.txt

* add extra idempotency unit test
---
 .../unreleased/Fixes-20221213-102005.yaml     |   7 +
 dbt/adapters/bigquery/dataset.py              |  31 +
 dbt/adapters/bigquery/impl.py                 |  39 +-
 .../adapter/test_grant_access_to.py           |  65 ++
 tests/unit/test_bigquery_adapter.py           | 736 +++++++++---------
 tests/unit/test_dataset.py                    |  43 +
 6 files changed, 529 insertions(+), 392 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20221213-102005.yaml
 create mode 100644 dbt/adapters/bigquery/dataset.py
 create mode 100644 tests/functional/adapter/test_grant_access_to.py
 create mode 100644 tests/unit/test_dataset.py

diff --git a/.changes/unreleased/Fixes-20221213-102005.yaml b/.changes/unreleased/Fixes-20221213-102005.yaml
new file mode 100644
index 000000000..5e4c486d8
--- /dev/null
+++ b/.changes/unreleased/Fixes-20221213-102005.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: Make BQ access_grant updates thread safe
+time: 2022-12-13T10:20:05.714134-08:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "266"
+  PR: "404"
diff --git a/dbt/adapters/bigquery/dataset.py b/dbt/adapters/bigquery/dataset.py
new file mode 100644
index 000000000..ebffe1072
--- /dev/null
+++ b/dbt/adapters/bigquery/dataset.py
@@ -0,0 +1,31 @@
+from typing import List
+from google.cloud.bigquery import Dataset, AccessEntry
+
+from dbt.events import AdapterLogger
+
+logger = AdapterLogger("BigQuery")
+
+
+def add_access_entry_to_dataset(dataset: Dataset, access_entry: AccessEntry) -> Dataset:
+    """Idempotently adds an access entry to a dataset
+
+    Args:
+        dataset (Dataset): the dataset to be updated
+        access_entry (AccessEntry): the access entry to be added to the dataset
+
+    Returns:
+        Dataset
+    """
+    access_entries: List[AccessEntry] = dataset.access_entries
+    # we can't simply check if an access entry is in the list as the current equality check
+    # does not work because the locally created AccessEntry can have extra properties.
+    for existing_entry in access_entries:
+        role_match = existing_entry.role == access_entry.role
+        entity_type_match = existing_entry.entity_type == access_entry.entity_type
+        property_match = existing_entry._properties.items() <= access_entry._properties.items()
+        if role_match and entity_type_match and property_match:
+            logger.warning(f"Access entry {access_entry} " f"already exists in dataset")
+            return dataset
+    access_entries.append(access_entry)
+    dataset.access_entries = access_entries
+    return dataset
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 920b992a0..c41df44c3 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -1,4 +1,5 @@
 from dataclasses import dataclass
+import threading
 from typing import Dict, List, Optional, Any, Set, Union, Type
 from dbt.dataclass_schema import dbtClassMixin, ValidationError
 
@@ -11,6 +12,7 @@
     BaseAdapter,
     available,
     RelationType,
+    BaseRelation,
     SchemaSearchMap,
     AdapterConfig,
     PythonJobHelper,
@@ -19,6 +21,7 @@
 from dbt.adapters.cache import _make_ref_key_msg
 
 from dbt.adapters.bigquery.relation import BigQueryRelation
+from dbt.adapters.bigquery.dataset import add_access_entry_to_dataset
 from dbt.adapters.bigquery import BigQueryColumn
 from dbt.adapters.bigquery import BigQueryConnectionManager
 from dbt.adapters.bigquery.python_submissions import (
@@ -51,6 +54,7 @@
 WRITE_TRUNCATE = google.cloud.bigquery.job.WriteDisposition.WRITE_TRUNCATE
 
 CREATE_SCHEMA_MACRO_NAME = "create_schema"
+_dataset_lock = threading.Lock()
 
 
 def sql_escape(string):
@@ -287,14 +291,16 @@ def list_relations_without_caching(
         # This will 404 if the dataset does not exist. This behavior mirrors
         # the implementation of list_relations for other adapters
         try:
-            return [self._bq_table_to_relation(table) for table in all_tables]
+            return [self._bq_table_to_relation(table) for table in all_tables]  # type: ignore[misc]
         except google.api_core.exceptions.NotFound:
             return []
         except google.api_core.exceptions.Forbidden as exc:
             logger.debug("list_relations_without_caching error: {}".format(str(exc)))
             return []
 
-    def get_relation(self, database: str, schema: str, identifier: str) -> BigQueryRelation:
+    def get_relation(
+        self, database: str, schema: str, identifier: str
+    ) -> Optional[BigQueryRelation]:
         if self._schema_is_cached(database, schema):
             # if it's in the cache, use the parent's model of going through
             # the relations cache and picking out the relation
@@ -477,7 +483,7 @@ def poll_until_job_completes(cls, job, timeout):
             message = "\n".join(error["message"].strip() for error in job.errors)
             raise dbt.exceptions.DbtRuntimeError(message)
 
-    def _bq_table_to_relation(self, bq_table):
+    def _bq_table_to_relation(self, bq_table) -> Union[BigQueryRelation, None]:
         if bq_table is None:
             return None
 
@@ -597,7 +603,7 @@ def parse_partition_by(self, raw_partition_by: Any) -> Optional[PartitionConfig]
         """
         return PartitionConfig.parse(raw_partition_by)
 
-    def get_table_ref_from_relation(self, relation):
+    def get_table_ref_from_relation(self, relation: BaseRelation):
         return self.connections.table_ref(relation.database, relation.schema, relation.identifier)
 
     def _update_column_dict(self, bq_column_dict, dbt_columns, parent=""):
@@ -798,29 +804,20 @@ def get_view_options(self, config: Dict[str, Any], node: Dict[str, Any]) -> Dict
     @available.parse_none
     def grant_access_to(self, entity, entity_type, role, grant_target_dict):
         """
-        Given an entity, grants it access to a permissioned dataset.
+        Given an entity, grants it access to a dataset.
         """
-        conn = self.connections.get_thread_connection()
+        conn: BigQueryConnectionManager = self.connections.get_thread_connection()
         client = conn.handle
-
         GrantTarget.validate(grant_target_dict)
         grant_target = GrantTarget.from_dict(grant_target_dict)
-        dataset_ref = self.connections.dataset_ref(grant_target.project, grant_target.dataset)
-        dataset = client.get_dataset(dataset_ref)
-
         if entity_type == "view":
             entity = self.get_table_ref_from_relation(entity).to_api_repr()
-
-        access_entry = AccessEntry(role, entity_type, entity)
-        access_entries = dataset.access_entries
-
-        if access_entry in access_entries:
-            logger.debug(f"Access entry {access_entry} " f"already exists in dataset")
-            return
-
-        access_entries.append(AccessEntry(role, entity_type, entity))
-        dataset.access_entries = access_entries
-        client.update_dataset(dataset, ["access_entries"])
+        with _dataset_lock:
+            dataset_ref = self.connections.dataset_ref(grant_target.project, grant_target.dataset)
+            dataset = client.get_dataset(dataset_ref)
+            access_entry = AccessEntry(role, entity_type, entity)
+            dataset = add_access_entry_to_dataset(dataset, access_entry)
+            client.update_dataset(dataset, ["access_entries"])
 
     @available.parse_none
     def get_dataset_location(self, relation):
diff --git a/tests/functional/adapter/test_grant_access_to.py b/tests/functional/adapter/test_grant_access_to.py
new file mode 100644
index 000000000..76996479d
--- /dev/null
+++ b/tests/functional/adapter/test_grant_access_to.py
@@ -0,0 +1,65 @@
+from abc import abstractmethod
+import pytest
+import os
+from dbt.tests.util import run_dbt
+
+SELECT_1 = """
+{{ config(
+    materialized='view',
+    grant_access_to=[
+      {'project': 'dbt-test-env-alt', 'dataset': 'GrantAccessTest'},
+    ]
+) }}
+SELECT 1 as one
+"""
+
+SELECT_1_TABLE = """
+{{ config(
+    materialized='table',
+    grant_access_to=[
+      {'project': 'dbt-test-env-alt', 'dataset': 'GrantAccessTest'},
+    ]
+) }}
+SELECT 1 as one
+"""
+BAD_CONFIG_TABLE_NAME = "bad_view"
+BAD_CONFIG_TABLE = """
+{{ config(
+    materialized='view',
+    grant_access_to=[
+      {'project': 'dbt-test-env-alt', 'dataset': 'NonExistentDataset'},
+    ]
+) }}
+SELECT 1 as one
+"""
+
+BAD_CONFIG_CHILD_TABLE = "SELECT 1 as one FROM {{ref('" + BAD_CONFIG_TABLE_NAME + "')}}"
+
+
+class TestAccessGrantSucceeds:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {"select_1.sql": SELECT_1, "select_1_table.sql": SELECT_1_TABLE}
+
+    def test_grant_access_succeeds(self, project):
+        # Need to run twice to validate idempotency
+        results = run_dbt(["run"])
+        assert len(results) == 2
+        results = run_dbt(["run"])
+        assert len(results) == 2
+
+
+class TestAccessGrantFails:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "bad_config_table_child.sql": BAD_CONFIG_CHILD_TABLE,
+            f"{BAD_CONFIG_TABLE_NAME}.sql": BAD_CONFIG_TABLE,
+        }
+
+    def test_grant_access_fails_without_running_child_table(self, project):
+        # Need to run twice to validate idempotency
+        results = run_dbt(["run"], expect_pass=False)
+        assert results[0].status == "error"
+        assert results[1].status == "skipped"
+        assert results[0].message.startswith("404 GET https://bigquery.googleapis.com/")
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 29c7a9e3d..6ac77eee4 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -1,3 +1,4 @@
+from unicodedata import name
 import agate
 import decimal
 import json
@@ -27,6 +28,7 @@
 from dbt.context.providers import RuntimeConfigObject
 
 import google.cloud.bigquery
+from google.cloud.bigquery import AccessEntry
 
 from .utils import config_from_parts_or_dicts, inject_adapter, TestAdapterConversions
 
@@ -38,85 +40,84 @@ def _bq_conn():
 
 
 class BaseTestBigQueryAdapter(unittest.TestCase):
-
     def setUp(self):
         self.raw_profile = {
-            'outputs': {
-                'oauth': {
-                    'type': 'bigquery',
-                    'method': 'oauth',
-                    'project': 'dbt-unit-000000',
-                    'schema': 'dummy_schema',
-                    'threads': 1,
+            "outputs": {
+                "oauth": {
+                    "type": "bigquery",
+                    "method": "oauth",
+                    "project": "dbt-unit-000000",
+                    "schema": "dummy_schema",
+                    "threads": 1,
                 },
-                'service_account': {
-                    'type': 'bigquery',
-                    'method': 'service-account',
-                    'project': 'dbt-unit-000000',
-                    'schema': 'dummy_schema',
-                    'keyfile': '/tmp/dummy-service-account.json',
-                    'threads': 1,
+                "service_account": {
+                    "type": "bigquery",
+                    "method": "service-account",
+                    "project": "dbt-unit-000000",
+                    "schema": "dummy_schema",
+                    "keyfile": "/tmp/dummy-service-account.json",
+                    "threads": 1,
                 },
-                'loc': {
-                    'type': 'bigquery',
-                    'method': 'oauth',
-                    'project': 'dbt-unit-000000',
-                    'schema': 'dummy_schema',
-                    'threads': 1,
-                    'location': 'Luna Station',
-                    'priority': 'batch',
-                    'maximum_bytes_billed': 0,
+                "loc": {
+                    "type": "bigquery",
+                    "method": "oauth",
+                    "project": "dbt-unit-000000",
+                    "schema": "dummy_schema",
+                    "threads": 1,
+                    "location": "Luna Station",
+                    "priority": "batch",
+                    "maximum_bytes_billed": 0,
                 },
-                'impersonate': {
-                    'type': 'bigquery',
-                    'method': 'oauth',
-                    'project': 'dbt-unit-000000',
-                    'schema': 'dummy_schema',
-                    'threads': 1,
-                    'impersonate_service_account': 'dummyaccount@dbt.iam.gserviceaccount.com'
+                "impersonate": {
+                    "type": "bigquery",
+                    "method": "oauth",
+                    "project": "dbt-unit-000000",
+                    "schema": "dummy_schema",
+                    "threads": 1,
+                    "impersonate_service_account": "dummyaccount@dbt.iam.gserviceaccount.com",
                 },
-                'oauth-credentials-token': {
-                    'type': 'bigquery',
-                    'method': 'oauth-secrets',
-                    'token': 'abc',
-                    'project': 'dbt-unit-000000',
-                    'schema': 'dummy_schema',
-                    'threads': 1,
-                    'location': 'Luna Station',
-                    'priority': 'batch',
-                    'maximum_bytes_billed': 0,
+                "oauth-credentials-token": {
+                    "type": "bigquery",
+                    "method": "oauth-secrets",
+                    "token": "abc",
+                    "project": "dbt-unit-000000",
+                    "schema": "dummy_schema",
+                    "threads": 1,
+                    "location": "Luna Station",
+                    "priority": "batch",
+                    "maximum_bytes_billed": 0,
                 },
-                'oauth-credentials': {
-                    'type': 'bigquery',
-                    'method': 'oauth-secrets',
-                    'client_id': 'abc',
-                    'client_secret': 'def',
-                    'refresh_token': 'ghi',
-                    'token_uri': 'jkl',
-                    'project': 'dbt-unit-000000',
-                    'schema': 'dummy_schema',
-                    'threads': 1,
-                    'location': 'Luna Station',
-                    'priority': 'batch',
-                    'maximum_bytes_billed': 0,
+                "oauth-credentials": {
+                    "type": "bigquery",
+                    "method": "oauth-secrets",
+                    "client_id": "abc",
+                    "client_secret": "def",
+                    "refresh_token": "ghi",
+                    "token_uri": "jkl",
+                    "project": "dbt-unit-000000",
+                    "schema": "dummy_schema",
+                    "threads": 1,
+                    "location": "Luna Station",
+                    "priority": "batch",
+                    "maximum_bytes_billed": 0,
                 },
-                'oauth-no-project': {
-                    'type': 'bigquery',
-                    'method': 'oauth',
-                    'schema': 'dummy_schema',
-                    'threads': 1,
-                    'location': 'Solar Station',
+                "oauth-no-project": {
+                    "type": "bigquery",
+                    "method": "oauth",
+                    "schema": "dummy_schema",
+                    "threads": 1,
+                    "location": "Solar Station",
                 },
             },
-            'target': 'oauth',
+            "target": "oauth",
         }
 
         self.project_cfg = {
-            'name': 'X',
-            'version': '0.1',
-            'project-root': '/tmp/dbt/does-not-exist',
-            'profile': 'default',
-            'config-version': 2,
+            "name": "X",
+            "version": "0.1",
+            "project-root": "/tmp/dbt/does-not-exist",
+            "profile": "default",
+            "config-version": 2,
         }
         self.qh_patch = None
 
@@ -125,7 +126,7 @@ def tearDown(self):
             self.qh_patch.stop()
         super().tearDown()
 
-    def get_adapter(self, target):
+    def get_adapter(self, target) -> BigQueryAdapter:
         project = self.project_cfg.copy()
         profile = self.raw_profile.copy()
         profile['target'] = target
@@ -187,8 +188,8 @@ def test_acquire_connection_oauth_validations(self, mock_open_connection):
     def test_acquire_connection_service_account_validations(self, mock_open_connection):
         adapter = self.get_adapter('service_account')
         try:
-            connection = adapter.acquire_connection('dummy')
-            self.assertEqual(connection.type, 'bigquery')
+            connection = adapter.acquire_connection("dummy")
+            self.assertEqual(connection.type, "bigquery")
 
         except dbt.exceptions.DbtValidationError as e:
             self.fail('got DbtValidationError: {}'.format(str(e)))
@@ -200,12 +201,12 @@ def test_acquire_connection_service_account_validations(self, mock_open_connecti
         connection.handle
         mock_open_connection.assert_called_once()
 
-    @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn())
+    @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
     def test_acquire_connection_oauth_token_validations(self, mock_open_connection):
-        adapter = self.get_adapter('oauth-credentials-token')
+        adapter = self.get_adapter("oauth-credentials-token")
         try:
-            connection = adapter.acquire_connection('dummy')
-            self.assertEqual(connection.type, 'bigquery')
+            connection = adapter.acquire_connection("dummy")
+            self.assertEqual(connection.type, "bigquery")
 
         except dbt.exceptions.DbtValidationError as e:
             self.fail('got DbtValidationError: {}'.format(str(e)))
@@ -217,12 +218,12 @@ def test_acquire_connection_oauth_token_validations(self, mock_open_connection):
         connection.handle
         mock_open_connection.assert_called_once()
 
-    @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn())
+    @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
     def test_acquire_connection_oauth_credentials_validations(self, mock_open_connection):
-        adapter = self.get_adapter('oauth-credentials')
+        adapter = self.get_adapter("oauth-credentials")
         try:
-            connection = adapter.acquire_connection('dummy')
-            self.assertEqual(connection.type, 'bigquery')
+            connection = adapter.acquire_connection("dummy")
+            self.assertEqual(connection.type, "bigquery")
 
         except dbt.exceptions.DbtValidationError as e:
             self.fail('got DbtValidationError: {}'.format(str(e)))
@@ -234,12 +235,14 @@ def test_acquire_connection_oauth_credentials_validations(self, mock_open_connec
         connection.handle
         mock_open_connection.assert_called_once()
 
-    @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn())
-    def test_acquire_connection_impersonated_service_account_validations(self, mock_open_connection):
-        adapter = self.get_adapter('impersonate')
+    @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
+    def test_acquire_connection_impersonated_service_account_validations(
+        self, mock_open_connection
+    ):
+        adapter = self.get_adapter("impersonate")
         try:
-            connection = adapter.acquire_connection('dummy')
-            self.assertEqual(connection.type, 'bigquery')
+            connection = adapter.acquire_connection("dummy")
+            self.assertEqual(connection.type, "bigquery")
 
         except dbt.exceptions.DbtValidationError as e:
             self.fail('got DbtValidationError: {}'.format(str(e)))
@@ -251,13 +254,13 @@ def test_acquire_connection_impersonated_service_account_validations(self, mock_
         connection.handle
         mock_open_connection.assert_called_once()
 
-    @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn())
+    @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
     def test_acquire_connection_priority(self, mock_open_connection):
-        adapter = self.get_adapter('loc')
+        adapter = self.get_adapter("loc")
         try:
-            connection = adapter.acquire_connection('dummy')
-            self.assertEqual(connection.type, 'bigquery')
-            self.assertEqual(connection.credentials.priority, 'batch')
+            connection = adapter.acquire_connection("dummy")
+            self.assertEqual(connection.type, "bigquery")
+            self.assertEqual(connection.credentials.priority, "batch")
 
         except dbt.exceptions.DbtValidationError as e:
             self.fail('got DbtValidationError: {}'.format(str(e)))
@@ -266,12 +269,12 @@ def test_acquire_connection_priority(self, mock_open_connection):
         connection.handle
         mock_open_connection.assert_called_once()
 
-    @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn())
+    @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
     def test_acquire_connection_maximum_bytes_billed(self, mock_open_connection):
-        adapter = self.get_adapter('loc')
+        adapter = self.get_adapter("loc")
         try:
-            connection = adapter.acquire_connection('dummy')
-            self.assertEqual(connection.type, 'bigquery')
+            connection = adapter.acquire_connection("dummy")
+            self.assertEqual(connection.type, "bigquery")
             self.assertEqual(connection.credentials.maximum_bytes_billed, 0)
 
         except dbt.exceptions.DbtValidationError as e:
@@ -282,63 +285,66 @@ def test_acquire_connection_maximum_bytes_billed(self, mock_open_connection):
         mock_open_connection.assert_called_once()
 
     def test_cancel_open_connections_empty(self):
-        adapter = self.get_adapter('oauth')
+        adapter = self.get_adapter("oauth")
         self.assertEqual(adapter.cancel_open_connections(), None)
 
     def test_cancel_open_connections_master(self):
-        adapter = self.get_adapter('oauth')
+        adapter = self.get_adapter("oauth")
         adapter.connections.thread_connections[0] = object()
         self.assertEqual(adapter.cancel_open_connections(), None)
 
     def test_cancel_open_connections_single(self):
-        adapter = self.get_adapter('oauth')
-        adapter.connections.thread_connections.update({
-            0: object(),
-            1: object(),
-        })
+        adapter = self.get_adapter("oauth")
+        adapter.connections.thread_connections.update(
+            {
+                0: object(),
+                1: object(),
+            }
+        )
         # actually does nothing
         self.assertEqual(adapter.cancel_open_connections(), None)
 
-    @patch('dbt.adapters.bigquery.impl.google.auth.default')
-    @patch('dbt.adapters.bigquery.impl.google.cloud.bigquery')
+    @patch("dbt.adapters.bigquery.impl.google.auth.default")
+    @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
     def test_location_user_agent(self, mock_bq, mock_auth_default):
         creds = MagicMock()
         mock_auth_default.return_value = (creds, MagicMock())
-        adapter = self.get_adapter('loc')
+        adapter = self.get_adapter("loc")
 
-        connection = adapter.acquire_connection('dummy')
+        connection = adapter.acquire_connection("dummy")
         mock_client = mock_bq.Client
 
         mock_client.assert_not_called()
         connection.handle
-        mock_client.assert_called_once_with('dbt-unit-000000', creds,
-                                            location='Luna Station',
-                                            client_info=HasUserAgent())
+        mock_client.assert_called_once_with(
+            "dbt-unit-000000", creds, location="Luna Station", client_info=HasUserAgent()
+        )
 
 
 class HasUserAgent:
-    PAT = re.compile(r'dbt-\d+\.\d+\.\d+((a|b|rc)\d+)?')
+    PAT = re.compile(r"dbt-\d+\.\d+\.\d+((a|b|rc)\d+)?")
 
     def __eq__(self, other):
-        compare = getattr(other, 'user_agent', '')
+        compare = getattr(other, "user_agent", "")
         return bool(self.PAT.match(compare))
 
 
 class TestConnectionNamePassthrough(BaseTestBigQueryAdapter):
-
     def setUp(self):
         super().setUp()
-        self._conn_patch = patch.object(BigQueryAdapter, 'ConnectionManager')
+        self._conn_patch = patch.object(BigQueryAdapter, "ConnectionManager")
         self.conn_manager_cls = self._conn_patch.start()
 
-        self._relation_patch = patch.object(BigQueryAdapter, 'Relation')
+        self._relation_patch = patch.object(BigQueryAdapter, "Relation")
         self.relation_cls = self._relation_patch.start()
 
         self.mock_connection_manager = self.conn_manager_cls.return_value
-        self.conn_manager_cls.TYPE = 'bigquery'
-        self.relation_cls.get_default_quote_policy.side_effect = BigQueryRelation.get_default_quote_policy
+        self.conn_manager_cls.TYPE = "bigquery"
+        self.relation_cls.get_default_quote_policy.side_effect = (
+            BigQueryRelation.get_default_quote_policy
+        )
 
-        self.adapter = self.get_adapter('oauth')
+        self.adapter = self.get_adapter("oauth")
 
     def tearDown(self):
         super().tearDown()
@@ -346,23 +352,25 @@ def tearDown(self):
         self._relation_patch.stop()
 
     def test_get_relation(self):
-        self.adapter.get_relation('db', 'schema', 'my_model')
-        self.mock_connection_manager.get_bq_table.assert_called_once_with('db', 'schema', 'my_model')
+        self.adapter.get_relation("db", "schema", "my_model")
+        self.mock_connection_manager.get_bq_table.assert_called_once_with(
+            "db", "schema", "my_model"
+        )
 
-    @patch.object(BigQueryAdapter, 'check_schema_exists')
+    @patch.object(BigQueryAdapter, "check_schema_exists")
     def test_drop_schema(self, mock_check_schema):
         mock_check_schema.return_value = True
-        relation = BigQueryRelation.create(database='db', schema='schema')
+        relation = BigQueryRelation.create(database="db", schema="schema")
         self.adapter.drop_schema(relation)
-        self.mock_connection_manager.drop_dataset.assert_called_once_with('db', 'schema')
+        self.mock_connection_manager.drop_dataset.assert_called_once_with("db", "schema")
 
     def test_get_columns_in_relation(self):
         self.mock_connection_manager.get_bq_table.side_effect = ValueError
         self.adapter.get_columns_in_relation(
-            MagicMock(database='db', schema='schema', identifier='ident'),
+            MagicMock(database="db", schema="schema", identifier="ident"),
         )
         self.mock_connection_manager.get_bq_table.assert_called_once_with(
-            database='db', schema='schema', identifier='ident'
+            database="db", schema="schema", identifier="ident"
         )
 
 
@@ -372,75 +380,49 @@ def setUp(self):
 
     def test_view_temp_relation(self):
         kwargs = {
-            'type': None,
-            'path': {
-                'database': 'test-project',
-                'schema': 'test_schema',
-                'identifier': 'my_view'
-            },
-            'quote_policy': {
-                'identifier': False
-            }
+            "type": None,
+            "path": {"database": "test-project", "schema": "test_schema", "identifier": "my_view"},
+            "quote_policy": {"identifier": False},
         }
         BigQueryRelation.validate(kwargs)
 
     def test_view_relation(self):
         kwargs = {
-            'type': 'view',
-            'path': {
-                'database': 'test-project',
-                'schema': 'test_schema',
-                'identifier': 'my_view'
-            },
-            'quote_policy': {
-                'identifier': True,
-                'schema': True
-            }
+            "type": "view",
+            "path": {"database": "test-project", "schema": "test_schema", "identifier": "my_view"},
+            "quote_policy": {"identifier": True, "schema": True},
         }
         BigQueryRelation.validate(kwargs)
 
     def test_table_relation(self):
         kwargs = {
-            'type': 'table',
-            'path': {
-                'database': 'test-project',
-                'schema': 'test_schema',
-                'identifier': 'generic_table'
+            "type": "table",
+            "path": {
+                "database": "test-project",
+                "schema": "test_schema",
+                "identifier": "generic_table",
             },
-            'quote_policy': {
-                'identifier': True,
-                'schema': True
-            }
+            "quote_policy": {"identifier": True, "schema": True},
         }
         BigQueryRelation.validate(kwargs)
 
     def test_external_source_relation(self):
         kwargs = {
-            'type': 'external',
-            'path': {
-                'database': 'test-project',
-                'schema': 'test_schema',
-                'identifier': 'sheet'
-            },
-            'quote_policy': {
-                'identifier': True,
-                'schema': True
-            }
+            "type": "external",
+            "path": {"database": "test-project", "schema": "test_schema", "identifier": "sheet"},
+            "quote_policy": {"identifier": True, "schema": True},
         }
         BigQueryRelation.validate(kwargs)
 
     def test_invalid_relation(self):
         kwargs = {
-            'type': 'invalid-type',
-            'path': {
-                'database': 'test-project',
-                'schema': 'test_schema',
-                'identifier': 'my_invalid_id'
+            "type": "invalid-type",
+            "path": {
+                "database": "test-project",
+                "schema": "test_schema",
+                "identifier": "my_invalid_id",
             },
-            'quote_policy': {
-                'identifier': False,
-                'schema': True
-            }
+            "quote_policy": {"identifier": False, "schema": True},
         }
         with self.assertRaises(dbt.dataclass_schema.ValidationError):
             BigQueryRelation.validate(kwargs)
@@ -453,28 +435,22 @@ def setUp(self):
     def test_replace(self):
 
         kwargs = {
-            'type': None,
-            'path': {
-                'database': 'test-project',
-                'schema': 'test_schema',
-                'identifier': 'my_view'
-            },
+            "type": None,
+            "path": {"database": "test-project", "schema": "test_schema", "identifier": "my_view"},
             # test for #2188
-            'quote_policy': {
-                'database': False
+            "quote_policy": {"database": False},
+            "include_policy": {
+                "database": True,
+                "schema": True,
+                "identifier": True,
             },
-            'include_policy': {
-                'database': True,
-                'schema': True,
-                'identifier': True,
-            }
         }
         BigQueryRelation.validate(kwargs)
         relation = BigQueryRelation.from_dict(kwargs)
         info_schema = relation.information_schema()
 
-        tables_schema = info_schema.replace(information_schema_view='__TABLES__')
-        assert tables_schema.information_schema_view == '__TABLES__'
+        tables_schema = info_schema.replace(information_schema_view="__TABLES__")
+        assert tables_schema.information_schema_view == "__TABLES__"
         assert tables_schema.include_policy.schema is True
         assert tables_schema.include_policy.identifier is False
         assert tables_schema.include_policy.database is True
@@ -482,8 +458,8 @@ def test_replace(self):
         assert tables_schema.quote_policy.identifier is False
         assert tables_schema.quote_policy.database is False
 
-        schemata_schema = info_schema.replace(information_schema_view='SCHEMATA')
-        assert schemata_schema.information_schema_view == 'SCHEMATA'
+        schemata_schema = info_schema.replace(information_schema_view="SCHEMATA")
+        assert schemata_schema.information_schema_view == "SCHEMATA"
         assert schemata_schema.include_policy.schema is False
         assert schemata_schema.include_policy.identifier is True
         assert schemata_schema.include_policy.database is True
@@ -491,8 +467,8 @@ def test_replace(self):
         assert schemata_schema.quote_policy.identifier is False
         assert schemata_schema.quote_policy.database is False
 
-        other_schema = info_schema.replace(information_schema_view='SOMETHING_ELSE')
-        assert other_schema.information_schema_view == 'SOMETHING_ELSE'
+        other_schema = info_schema.replace(information_schema_view="SOMETHING_ELSE")
+        assert other_schema.information_schema_view == "SOMETHING_ELSE"
         assert other_schema.include_policy.schema is True
         assert other_schema.include_policy.identifier is True
         assert other_schema.include_policy.database is True
@@ -502,14 +478,12 @@ def test_replace(self):
 
 
 class TestBigQueryConnectionManager(unittest.TestCase):
-
     def setUp(self):
         credentials = Mock(BigQueryCredentials)
         profile = Mock(query_comment=None, credentials=credentials)
         self.connections = BigQueryConnectionManager(profile=profile)
 
-        self.mock_client = Mock(
-          dbt.adapters.bigquery.impl.google.cloud.bigquery.Client)
+        self.mock_client = Mock(dbt.adapters.bigquery.impl.google.cloud.bigquery.Client)
         self.mock_connection = MagicMock()
 
         self.mock_connection.handle = self.mock_client
@@ -518,8 +492,7 @@ def setUp(self):
         self.connections.get_job_retry_deadline_seconds = lambda x: None
         self.connections.get_job_retries = lambda x: 1
 
-    @patch(
-        'dbt.adapters.bigquery.connections._is_retryable', return_value=True)
+    @patch("dbt.adapters.bigquery.connections._is_retryable", return_value=True)
     def test_retry_and_handle(self, is_retryable):
         self.connections.DEFAULT_MAXIMUM_DELAY = 2.0
 
@@ -531,6 +504,7 @@ def dummy_handler(msg):
 
         class DummyException(Exception):
             """Count how many times this exception is raised"""
+
             count = 0
 
             def __init__(self):
@@ -541,12 +515,11 @@ def raiseDummyException():
 
         with self.assertRaises(DummyException):
             self.connections._retry_and_handle(
-                 "some sql", Mock(credentials=Mock(retries=8)),
-                 raiseDummyException)
+                "some sql", Mock(credentials=Mock(retries=8)), raiseDummyException
+            )
             self.assertEqual(DummyException.count, 9)
 
-    @patch(
-        'dbt.adapters.bigquery.connections._is_retryable', return_value=True)
+    @patch("dbt.adapters.bigquery.connections._is_retryable", return_value=True)
     def test_retry_connection_reset(self, is_retryable):
         self.connections.open = MagicMock()
         self.connections.close = MagicMock()
@@ -563,20 +536,20 @@ def raiseConnectionResetError():
 
         mock_conn = Mock(credentials=Mock(retries=1))
         with self.assertRaises(ConnectionResetError):
-            self.connections._retry_and_handle(
-              "some sql", mock_conn,
-              raiseConnectionResetError)
+            self.connections._retry_and_handle("some sql", mock_conn, raiseConnectionResetError)
         self.connections.close.assert_called_once_with(mock_conn)
         self.connections.open.assert_called_once_with(mock_conn)
 
     def test_is_retryable(self):
         _is_retryable = dbt.adapters.bigquery.connections._is_retryable
         exceptions = dbt.adapters.bigquery.impl.google.cloud.exceptions
-        internal_server_error = exceptions.InternalServerError('code broke')
-        bad_request_error = exceptions.BadRequest('code broke')
-        connection_error = ConnectionError('code broke')
-        client_error = exceptions.ClientError('bad code')
-        rate_limit_error = exceptions.Forbidden("code broke", errors=[{"reason": "rateLimitExceeded"}])
+        internal_server_error = exceptions.InternalServerError("code broke")
+        bad_request_error = exceptions.BadRequest("code broke")
+        connection_error = ConnectionError("code broke")
+        client_error = exceptions.ClientError("bad code")
+        rate_limit_error = exceptions.Forbidden(
+            "code broke", errors=[{"reason": "rateLimitExceeded"}]
+        )
 
         self.assertTrue(_is_retryable(internal_server_error))
         self.assertTrue(_is_retryable(bad_request_error))
@@ -587,34 +560,38 @@ def test_is_retryable(self):
     def test_drop_dataset(self):
 
         mock_table = Mock()
-        mock_table.reference = 'table1'
+        mock_table.reference = "table1"
         self.mock_client.list_tables.return_value = [mock_table]
 
-        self.connections.drop_dataset('project', 'dataset')
+        self.connections.drop_dataset("project", "dataset")
 
         self.mock_client.list_tables.assert_not_called()
         self.mock_client.delete_table.assert_not_called()
         self.mock_client.delete_dataset.assert_called_once()
 
-    @patch('dbt.adapters.bigquery.impl.google.cloud.bigquery')
+    @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
     def test_query_and_results(self, mock_bq):
         self.connections._query_and_results(
-            self.mock_client, 'sql', {'job_param_1': 'blah'},
+            self.mock_client,
+            "sql",
+            {"job_param_1": "blah"},
             job_creation_timeout=15,
-            job_execution_timeout=100)
+            job_execution_timeout=100,
+        )
 
         mock_bq.QueryJobConfig.assert_called_once()
         self.mock_client.query.assert_called_once_with(
-            query='sql', job_config=mock_bq.QueryJobConfig(), timeout=15)
+            query="sql", job_config=mock_bq.QueryJobConfig(), timeout=15
+        )
 
     def test_copy_bq_table_appends(self):
-        self._copy_table(
-            write_disposition=dbt.adapters.bigquery.impl.WRITE_APPEND)
+        self._copy_table(write_disposition=dbt.adapters.bigquery.impl.WRITE_APPEND)
         args, kwargs = self.mock_client.copy_table.call_args
         self.mock_client.copy_table.assert_called_once_with(
-            [self._table_ref('project', 'dataset', 'table1')],
-            self._table_ref('project', 'dataset', 'table2'),
-            job_config=ANY)
+            [self._table_ref("project", "dataset", "table1")],
+            self._table_ref("project", "dataset", "table2"),
+            job_config=ANY,
+        )
         args, kwargs = self.mock_client.copy_table.call_args
         self.assertEqual(
             kwargs['job_config'].write_disposition,
@@ -654,25 +631,24 @@ def _copy_table(self, write_disposition):
 
 
 class TestBigQueryAdapter(BaseTestBigQueryAdapter):
-
     def test_copy_table_materialization_table(self):
-        adapter = self.get_adapter('oauth')
+        adapter = self.get_adapter("oauth")
         adapter.connections = MagicMock()
-        adapter.copy_table('source', 'destination', 'table')
+        adapter.copy_table("source", "destination", "table")
         adapter.connections.copy_bq_table.assert_called_once_with(
-            'source', 'destination',
-            dbt.adapters.bigquery.impl.WRITE_TRUNCATE)
+            "source", "destination", dbt.adapters.bigquery.impl.WRITE_TRUNCATE
+        )
 
     def test_copy_table_materialization_incremental(self):
-        adapter = self.get_adapter('oauth')
+        adapter = self.get_adapter("oauth")
         adapter.connections = MagicMock()
-        adapter.copy_table('source', 'destination', 'incremental')
+        adapter.copy_table("source", "destination", "incremental")
         adapter.connections.copy_bq_table.assert_called_once_with(
-            'source', 'destination',
-            dbt.adapters.bigquery.impl.WRITE_APPEND)
+            "source", "destination", dbt.adapters.bigquery.impl.WRITE_APPEND
+        )
 
     def test_parse_partition_by(self):
-        adapter = self.get_adapter('oauth')
+        adapter = self.get_adapter("oauth")
 
         with self.assertRaises(dbt.exceptions.DbtValidationError):
             adapter.parse_partition_by("date(ts)")
@@ -681,163 +657,150 @@ def test_parse_partition_by(self):
             adapter.parse_partition_by("ts")
 
         self.assertEqual(
-            adapter.parse_partition_by({
-                "field": "ts",
-            }).to_dict(omit_none=True), {
+            adapter.parse_partition_by(
+                {
+                    "field": "ts",
+                }
+            ).to_dict(omit_none=True),
+            {
                 "field": "ts",
                 "data_type": "date",
                 "granularity": "day",
                 "time_ingestion_partitioning": False,
-                "copy_partitions": False
-            }
+                "copy_partitions": False,
+            },
         )
 
         self.assertEqual(
-            adapter.parse_partition_by({
-                "field": "ts",
-                "data_type": "date",
-            }).to_dict(omit_none=True), {
+            adapter.parse_partition_by(
+                {
+                    "field": "ts",
+                    "data_type": "date",
+                }
+            ).to_dict(omit_none=True),
+            {
                 "field": "ts",
                 "data_type": "date",
                 "granularity": "day",
                 "time_ingestion_partitioning": False,
-                "copy_partitions": False
-            }
+                "copy_partitions": False,
+            },
         )
 
         self.assertEqual(
-            adapter.parse_partition_by({
-                "field": "ts",
-                "data_type": "date",
-                "granularity": "MONTH"
-
-            }).to_dict(omit_none=True), {
+            adapter.parse_partition_by(
+                {"field": "ts", "data_type": "date", "granularity": "MONTH"}
+            ).to_dict(omit_none=True),
+            {
                 "field": "ts",
                 "data_type": "date",
                 "granularity": "MONTH",
                 "time_ingestion_partitioning": False,
-                "copy_partitions": False
-            }
+                "copy_partitions": False,
+            },
         )
 
         self.assertEqual(
-            adapter.parse_partition_by({
-                "field": "ts",
-                "data_type": "date",
-                "granularity": "YEAR"
-
-            }).to_dict(omit_none=True), {
+            adapter.parse_partition_by(
+                {"field": "ts", "data_type": "date", "granularity": "YEAR"}
+            ).to_dict(omit_none=True),
+            {
                 "field": "ts",
                 "data_type": "date",
                 "granularity": "YEAR",
                 "time_ingestion_partitioning": False,
-                "copy_partitions": False
-            }
+                "copy_partitions": False,
+            },
         )
 
         self.assertEqual(
-            adapter.parse_partition_by({
-                "field": "ts",
-                "data_type": "timestamp",
-                "granularity": "HOUR"
-
-            }).to_dict(omit_none=True), {
+            adapter.parse_partition_by(
+                {"field": "ts", "data_type": "timestamp", "granularity": "HOUR"}
+            ).to_dict(omit_none=True),
+            {
                 "field": "ts",
                 "data_type": "timestamp",
                 "granularity": "HOUR",
                 "time_ingestion_partitioning": False,
-                "copy_partitions": False
-            }
+                "copy_partitions": False,
+            },
         )
 
         self.assertEqual(
-            adapter.parse_partition_by({
-                "field": "ts",
-                "data_type": "timestamp",
-                "granularity": "MONTH"
-
-            }).to_dict(omit_none=True), {
+            adapter.parse_partition_by(
+                {"field": "ts", "data_type": "timestamp", "granularity": "MONTH"}
+            ).to_dict(omit_none=True), {
                 "field": "ts",
                 "data_type": "timestamp",
                 "granularity": "MONTH",
                 "time_ingestion_partitioning": False,
-                "copy_partitions": False
-            }
+                "copy_partitions": False,
+            },
         )
 
         self.assertEqual(
-            adapter.parse_partition_by({
-                "field": "ts",
-                "data_type": "timestamp",
-                "granularity": "YEAR"
-
-            }).to_dict(omit_none=True), {
+            adapter.parse_partition_by(
+                {"field": "ts", "data_type": "timestamp", "granularity": "YEAR"}
+            ).to_dict(omit_none=True),
+            {
                 "field": "ts",
                 "data_type": "timestamp",
                 "granularity": "YEAR",
                 "time_ingestion_partitioning": False,
-                "copy_partitions": False
-            }
+                "copy_partitions": False,
+            },
         )
 
         self.assertEqual(
-            adapter.parse_partition_by({
-                "field": "ts",
-                "data_type": "datetime",
-                "granularity": "HOUR"
-
-            }).to_dict(omit_none=True), {
+            adapter.parse_partition_by(
+                {"field": "ts", "data_type": "datetime", "granularity": "HOUR"}
+            ).to_dict(omit_none=True),
+            {
                 "field": "ts",
                 "data_type": "datetime",
                 "granularity": "HOUR",
                 "time_ingestion_partitioning": False,
-                "copy_partitions": False
-            }
+                "copy_partitions": False,
+            },
         )
 
         self.assertEqual(
-            adapter.parse_partition_by({
-                "field": "ts",
-                "data_type": "datetime",
-                "granularity": "MONTH"
-
-            }).to_dict(omit_none=True), {
+            adapter.parse_partition_by(
+                {"field": "ts", "data_type": "datetime", "granularity": "MONTH"}
+            ).to_dict(omit_none=True),
+            {
                 "field": "ts",
                 "data_type": "datetime",
                 "granularity": "MONTH",
                 "time_ingestion_partitioning": False,
-                "copy_partitions": False
-            }
+                "copy_partitions": False,
+            },
         )
 
         self.assertEqual(
-            adapter.parse_partition_by({
-                "field": "ts",
-                "data_type": "datetime",
-                "granularity": "YEAR"
-
-            }).to_dict(omit_none=True), {
+            adapter.parse_partition_by(
+                {"field": "ts", "data_type": "datetime", "granularity": "YEAR"}
+            ).to_dict(omit_none=True),
+            {
                 "field": "ts",
                 "data_type": "datetime",
                 "granularity": "YEAR",
                 "time_ingestion_partitioning": False,
-                "copy_partitions": False
-            }
+                "copy_partitions": False,
+            },
         )
 
         self.assertEqual(
-            adapter.parse_partition_by({
-                "field": "ts",
-                "time_ingestion_partitioning": True,
-                "copy_partitions": True
-
-            }).to_dict(omit_none=True), {
+            adapter.parse_partition_by(
+                {"field": "ts", "time_ingestion_partitioning": True, "copy_partitions": True}
+            ).to_dict(omit_none=True),
+            {
                 "field": "ts",
                 "data_type": "date",
                 "granularity": "day",
                 "time_ingestion_partitioning": True,
-                "copy_partitions": True
-            }
+                "copy_partitions": True,
+            },
         )
 
         # Invalid, should raise an error
@@ -846,38 +809,31 @@ def test_parse_partition_by(self):
 
         # passthrough
         self.assertEqual(
-            adapter.parse_partition_by({
-                "field": "id",
-                "data_type": "int64",
-                "range": {
-                    "start": 1,
-                    "end": 100,
-                    "interval": 20
+            adapter.parse_partition_by(
+                {
+                    "field": "id",
+                    "data_type": "int64",
+                    "range": {"start": 1, "end": 100, "interval": 20},
                 }
-            }).to_dict(omit_none=True
-                ), {
+            ).to_dict(omit_none=True),
+            {
                 "field": "id",
                 "data_type": "int64",
                 "granularity": "day",
-                "range": {
-                    "start": 1,
-                    "end": 100,
-                    "interval": 20
-                },
+                "range": {"start": 1, "end": 100, "interval": 20},
                 "time_ingestion_partitioning": False,
-                "copy_partitions": False
-            }
+                "copy_partitions": False,
+            },
         )
 
     def test_hours_to_expiration(self):
-        adapter = self.get_adapter('oauth')
-        mock_config = create_autospec(
-            RuntimeConfigObject)
-        config = {'hours_to_expiration': 4}
+        adapter = self.get_adapter("oauth")
+        mock_config = create_autospec(RuntimeConfigObject)
+        config = {"hours_to_expiration": 4}
         mock_config.get.side_effect = lambda name: config.get(name)
 
         expected = {
-            'expiration_timestamp': 'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 4 hour)',
+            "expiration_timestamp": "TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 4 hour)",
         }
         actual = adapter.get_table_options(mock_config, node={}, temporary=False)
         self.assertEqual(expected, actual)
@@ -890,8 +846,7 @@ def test_hours_to_expiration_temporary(self):
         mock_config.get.side_effect = lambda name: config.get(name)
 
         expected = {
-            'expiration_timestamp': (
-                'TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)'),
+            "expiration_timestamp": ("TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 12 hour)"),
         }
         actual = adapter.get_table_options(mock_config, node={}, temporary=True)
         self.assertEqual(expected, actual)
@@ -903,9 +858,7 @@ def test_table_kms_key_name(self):
         config = {'kms_key_name': 'some_key'}
         mock_config.get.side_effect = lambda name: config.get(name)
 
-        expected = {
-            'kms_key_name': "'some_key'"
-        }
+        expected = {"kms_key_name": "'some_key'"}
         actual = adapter.get_table_options(mock_config, node={}, temporary=False)
         self.assertEqual(expected, actual)
 
@@ -924,80 +877,80 @@ def test_view_kms_key_name(self):
 class TestBigQueryFilterCatalog(unittest.TestCase):
     def test__catalog_filter_table(self):
         manifest = MagicMock()
-        manifest.get_used_schemas.return_value = [['a', 'B'], ['a', '1234']]
-        column_names = ['table_name', 'table_database', 'table_schema', 'something']
+        manifest.get_used_schemas.return_value = [["a", "B"], ["a", "1234"]]
+        column_names = ["table_name", "table_database", "table_schema", "something"]
         rows = [
-            ['foo', 'a', 'b', '1234'],  # include
-            ['foo', 'a', '1234', '1234'],  # include, w/ table schema as str
-            ['foo', 'c', 'B', '1234'],  # skip
-            ['1234', 'A', 'B', '1234'],  # include, w/ table name as str
+            ["foo", "a", "b", "1234"],  # include
+            ["foo", "a", "1234", "1234"],  # include, w/ table schema as str
+            ["foo", "c", "B", "1234"],  # skip
+            ["1234", "A", "B", "1234"],  # include, w/ table name as str
         ]
-        table = agate.Table(
-            rows, column_names, agate_helper.DEFAULT_TYPE_TESTER
-        )
+        table = agate.Table(rows, column_names, agate_helper.DEFAULT_TYPE_TESTER)
 
         result = BigQueryAdapter._catalog_filter_table(table, manifest)
         assert len(result) == 3
         for row in result.rows:
-            assert isinstance(row['table_schema'], str)
-            assert isinstance(row['table_database'], str)
-            assert isinstance(row['table_name'], str)
-            assert isinstance(row['something'], decimal.Decimal)
+            assert isinstance(row["table_schema"], str)
+            assert isinstance(row["table_database"], str)
+            assert isinstance(row["table_name"], str)
+            assert isinstance(row["something"], decimal.Decimal)
 
 
 class TestBigQueryAdapterConversions(TestAdapterConversions):
     def test_convert_text_type(self):
         rows = [
-            ['', 'a1', 'stringval1'],
-            ['', 'a2', 'stringvalasdfasdfasdfa'],
-            ['', 'a3', 'stringval3'],
+            ["", "a1", "stringval1"],
+            ["", "a2", "stringvalasdfasdfasdfa"],
+            ["", "a3", "stringval3"],
         ]
         agate_table = self._make_table_of(rows, agate.Text)
-        expected = ['string', 'string', 'string']
+        expected = ["string", "string", "string"]
         for col_idx, expect in enumerate(expected):
             assert BigQueryAdapter.convert_text_type(agate_table, col_idx) == expect
 
     def test_convert_number_type(self):
         rows = [
-            ['', '23.98', '-1'],
-            ['', '12.78', '-2'],
-            ['', '79.41', '-3'],
+            ["", "23.98", "-1"],
+            ["", "12.78", "-2"],
+            ["", "79.41", "-3"],
         ]
         agate_table = self._make_table_of(rows, agate.Number)
-        expected = ['int64', 'float64', 'int64']
+        expected = ["int64", "float64", "int64"]
         for col_idx, expect in enumerate(expected):
             assert BigQueryAdapter.convert_number_type(agate_table, col_idx) == expect
 
     def test_convert_boolean_type(self):
         rows = [
-            ['', 'false', 'true'],
-            ['', 'false', 'false'],
-            ['', 'false', 'true'],
+            ["", "false", "true"],
+            ["", "false", "false"],
+            ["", "false", "true"],
         ]
         agate_table = self._make_table_of(rows, agate.Boolean)
-        expected = ['bool', 'bool', 'bool']
+        expected = ["bool", "bool", "bool"]
         for col_idx, expect in enumerate(expected):
             assert BigQueryAdapter.convert_boolean_type(agate_table, col_idx) == expect
 
     def test_convert_datetime_type(self):
         rows = [
-            ['', '20190101T01:01:01Z', '2019-01-01 01:01:01'],
-            ['', '20190102T01:01:01Z', '2019-01-01 01:01:01'],
-            ['', '20190103T01:01:01Z', '2019-01-01 01:01:01'],
+            ["", "20190101T01:01:01Z", "2019-01-01 01:01:01"],
+            ["", "20190102T01:01:01Z", "2019-01-01 01:01:01"],
+            ["", "20190103T01:01:01Z", "2019-01-01 01:01:01"],
         ]
-        agate_table = self._make_table_of(rows, [agate.DateTime, agate_helper.ISODateTime, agate.DateTime])
-        expected = ['datetime', 'datetime', 'datetime']
+        agate_table = self._make_table_of(
+            rows, [agate.DateTime, agate_helper.ISODateTime, agate.DateTime]
+        )
+        expected = ["datetime", "datetime", "datetime"]
         for col_idx, expect in enumerate(expected):
             assert BigQueryAdapter.convert_datetime_type(agate_table, col_idx) == expect
 
     def test_convert_date_type(self):
         rows = [
-            ['', '2019-01-01', '2019-01-04'],
-            ['', '2019-01-02', '2019-01-04'],
-            ['', '2019-01-03', '2019-01-04'],
+            ["", "2019-01-01", "2019-01-04"],
+            ["", "2019-01-02", "2019-01-04"],
+            ["", "2019-01-03", "2019-01-04"],
         ]
         agate_table = self._make_table_of(rows, agate.Date)
-        expected = ['date', 'date', 'date']
+        expected = ["date", "date", "date"]
         for col_idx, expect in enumerate(expected):
             assert BigQueryAdapter.convert_date_type(agate_table, col_idx) == expect
 
@@ -1005,16 +958,57 @@ def test_convert_time_type(self):
         # dbt's default type testers actually don't have a TimeDelta at all.
         agate.TimeDelta
         rows = [
-            ['', '120s', '10s'],
-            ['', '3m', '11s'],
-            ['', '1h', '12s'],
+            ["", "120s", "10s"],
+            ["", "3m", "11s"],
+            ["", "1h", "12s"],
         ]
         agate_table = self._make_table_of(rows, agate.TimeDelta)
-        expected = ['time', 'time', 'time']
+        expected = ["time", "time", "time"]
         for col_idx, expect in enumerate(expected):
             assert BigQueryAdapter.convert_time_type(agate_table, col_idx) == expect
 
 
+class TestBigQueryGrantAccessTo(BaseTestBigQueryAdapter):
+    entity = BigQueryRelation.from_dict(
+        {
+            "type": None,
+            "path": {"database": "test-project", "schema": "test_schema", "identifier": "my_view"},
+            "quote_policy": {"identifier": False},
+        }
+    )
+
+    def setUp(self):
+        super().setUp()
+        self.mock_dataset: MagicMock = MagicMock(name="GrantMockDataset")
+        self.mock_dataset.access_entries = [AccessEntry(None, "table", self.entity)]
+        self.mock_client: MagicMock = MagicMock(name="MockBQClient")
+        self.mock_client.get_dataset.return_value = self.mock_dataset
+        self.mock_connection = MagicMock(name="MockConn")
+        self.mock_connection.handle = self.mock_client
+        self.mock_connection_mgr = MagicMock(
+            name="GrantAccessMockMgr",
+        )
+        self.mock_connection_mgr.get_thread_connection.return_value = self.mock_connection
+        _adapter = self.get_adapter("oauth")
+        _adapter.connections = self.mock_connection_mgr
+        self.adapter = _adapter
+
+    def test_grant_access_to_calls_update_with_valid_access_entry(self):
+        a_different_entity = BigQueryRelation.from_dict(
+            {
+                "type": None,
+                "path": {"database": "another-test-project",
+                 "schema": "test_schema_2", "identifier": "my_view"},
+                "quote_policy": {"identifier": True},
+            }
+        )
+        grant_target_dict = {"dataset": "someOtherDataset", "project": "someProject"}
+        self.adapter.grant_access_to(
+            entity=a_different_entity, entity_type="view", role=None, grant_target_dict=grant_target_dict
+        )
+        self.mock_client.update_dataset.assert_called_once()
+
+
 @pytest.mark.parametrize(
     ["input", "output"],
     [
diff --git a/tests/unit/test_dataset.py b/tests/unit/test_dataset.py
new file mode 100644
index 000000000..53109e5cf
--- /dev/null
+++ b/tests/unit/test_dataset.py
@@ -0,0 +1,43 @@
+from dbt.adapters.bigquery.dataset import add_access_entry_to_dataset
+from dbt.adapters.bigquery import BigQueryRelation
+
+from google.cloud.bigquery import Dataset, AccessEntry, DatasetReference
+
+
+def test_add_access_entry_to_dataset_idempotently_adds_entries():
+    database = "someDb"
+    dataset = "someDataset"
+    entity = BigQueryRelation.from_dict(
+        {
+            "type": None,
+            "path": {"database": "test-project", "schema": "test_schema", "identifier": "my_table"},
+            "quote_policy": {"identifier": False},
+        }
+    ).to_dict()
+    dataset_ref = DatasetReference(project=database, dataset_id=dataset)
+    dataset = Dataset(dataset_ref)
+    access_entry = AccessEntry(None, "table", entity)
+    dataset = add_access_entry_to_dataset(dataset, access_entry)
+    assert access_entry in dataset.access_entries
+    dataset = add_access_entry_to_dataset(dataset, access_entry)
+    assert len(dataset.access_entries) == 1
+
+
+def test_add_access_entry_to_dataset_does_not_add_with_pre_existing_entries():
+    database = "someOtherDb"
+    dataset = "someOtherDataset"
+    entity_2 = BigQueryRelation.from_dict(
+        {
+            "type": None,
+            "path": {"database": "test-project", "schema": "test_schema", "identifier": "some_other_view"},
+            "quote_policy": {"identifier": False},
+        }
+    ).to_dict()
+    dataset_ref = DatasetReference(project=database, dataset_id=dataset)
+    dataset = Dataset(dataset_ref)
+    initial_entry = AccessEntry(None, "view", entity_2)
+    initial_entry._properties.pop("role")
+    dataset.access_entries = [initial_entry]
+    access_entry = AccessEntry(None, "view", entity_2)
+    dataset = add_access_entry_to_dataset(dataset, access_entry)
+    assert len(dataset.access_entries) == 1

From c5cf1ea2bab57e83e22dd4afa330fe309763f5dd Mon Sep 17 00:00:00 2001
From: Jeremy Cohen <jeremy@dbtlabs.com>
Date: Thu, 9 Feb 2023 22:48:30 +0100
Subject: [PATCH 551/860] Rm defer_state_test (#468)

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .../macros/infinite_macros.sql                | 13 ---
 .../defer_state_test/macros/macros.sql        |  3 -
 .../models/ephemeral_model.sql                |  2 -
 .../defer_state_test/models/exposures.yml     |  8 --
 .../defer_state_test/models/schema.yml        |  9 --
 .../defer_state_test/models/table_model.sql   |  5 -
 .../defer_state_test/models/view_model.sql    |  4 -
 .../defer_state_test/seeds/seed.csv           |  3 -
 .../defer_state_test/test_defer_state.py      | 91 -------------------
 9 files changed, 138 deletions(-)
 delete mode 100644 tests/integration/defer_state_test/macros/infinite_macros.sql
 delete mode 100644 tests/integration/defer_state_test/macros/macros.sql
 delete mode 100644 tests/integration/defer_state_test/models/ephemeral_model.sql
 delete mode 100644 tests/integration/defer_state_test/models/exposures.yml
 delete mode 100644 tests/integration/defer_state_test/models/schema.yml
 delete mode 100644 tests/integration/defer_state_test/models/table_model.sql
 delete mode 100644 tests/integration/defer_state_test/models/view_model.sql
 delete mode 100644 tests/integration/defer_state_test/seeds/seed.csv
 delete mode 100644 tests/integration/defer_state_test/test_defer_state.py

diff --git a/tests/integration/defer_state_test/macros/infinite_macros.sql b/tests/integration/defer_state_test/macros/infinite_macros.sql
deleted file mode 100644
index 81d2083d3..000000000
--- a/tests/integration/defer_state_test/macros/infinite_macros.sql
+++ /dev/null
@@ -1,13 +0,0 @@
-{# trigger infinite recursion if not handled #}
-
-{% macro my_infinitely_recursive_macro() %}
-  {{ return(adapter.dispatch('my_infinitely_recursive_macro')()) }}
-{% endmacro %}
-
-{% macro default__my_infinitely_recursive_macro() %}
-    {% if unmet_condition %}
-        {{ my_infinitely_recursive_macro() }}
-    {% else %}
-        {{ return('') }}
-    {% endif %}
-{% endmacro %}
diff --git a/tests/integration/defer_state_test/macros/macros.sql b/tests/integration/defer_state_test/macros/macros.sql
deleted file mode 100644
index 79519c1b6..000000000
--- a/tests/integration/defer_state_test/macros/macros.sql
+++ /dev/null
@@ -1,3 +0,0 @@
-{% macro my_macro() %}
-    {% do log('in a macro' ) %}
-{% endmacro %}
diff --git a/tests/integration/defer_state_test/models/ephemeral_model.sql b/tests/integration/defer_state_test/models/ephemeral_model.sql
deleted file mode 100644
index 2f976e3a9..000000000
--- a/tests/integration/defer_state_test/models/ephemeral_model.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-{{ config(materialized='ephemeral') }}
-select * from {{ ref('view_model') }}
diff --git a/tests/integration/defer_state_test/models/exposures.yml b/tests/integration/defer_state_test/models/exposures.yml
deleted file mode 100644
index 489dec3c3..000000000
--- a/tests/integration/defer_state_test/models/exposures.yml
+++ /dev/null
@@ -1,8 +0,0 @@
-version: 2
-exposures:
-  - name: my_exposure
-    type: application
-    depends_on:
-      - ref('view_model')
-    owner:
-      email: test@example.com
diff --git a/tests/integration/defer_state_test/models/schema.yml b/tests/integration/defer_state_test/models/schema.yml
deleted file mode 100644
index 1ec506d3d..000000000
--- a/tests/integration/defer_state_test/models/schema.yml
+++ /dev/null
@@ -1,9 +0,0 @@
-version: 2
-models:
-  - name: view_model
-    columns:
-      - name: id
-        tests:
-          - unique
-          - not_null
-      - name: name
diff --git a/tests/integration/defer_state_test/models/table_model.sql b/tests/integration/defer_state_test/models/table_model.sql
deleted file mode 100644
index 65909318b..000000000
--- a/tests/integration/defer_state_test/models/table_model.sql
+++ /dev/null
@@ -1,5 +0,0 @@
-{{ config(materialized='table') }}
-select * from {{ ref('ephemeral_model') }}
-
--- establish a macro dependency to trigger state:modified.macros
--- depends on: {{ my_macro() }}
\ No newline at end of file
diff --git a/tests/integration/defer_state_test/models/view_model.sql b/tests/integration/defer_state_test/models/view_model.sql
deleted file mode 100644
index 72cb07a5e..000000000
--- a/tests/integration/defer_state_test/models/view_model.sql
+++ /dev/null
@@ -1,4 +0,0 @@
-select * from {{ ref('seed') }}
-
--- establish a macro dependency that trips infinite recursion if not handled
--- depends on: {{ my_infinitely_recursive_macro() }}
\ No newline at end of file
diff --git a/tests/integration/defer_state_test/seeds/seed.csv b/tests/integration/defer_state_test/seeds/seed.csv
deleted file mode 100644
index 1a728c8ab..000000000
--- a/tests/integration/defer_state_test/seeds/seed.csv
+++ /dev/null
@@ -1,3 +0,0 @@
-id,name
-1,Alice
-2,Bob
diff --git a/tests/integration/defer_state_test/test_defer_state.py b/tests/integration/defer_state_test/test_defer_state.py
deleted file mode 100644
index ab319d0b1..000000000
--- a/tests/integration/defer_state_test/test_defer_state.py
+++ /dev/null
@@ -1,91 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-import copy
-import json
-import os
-import shutil
-
-import pytest
-
-
-class TestDeferState(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "defer_state"
-
-    @property
-    def models(self):
-        return "models"
-
-    def setUp(self):
-        self.other_schema = None
-        super().setUp()
-        self._created_schemas.add(self.other_schema)
-
-    def tearDown(self):
-        with self.adapter.connection_named('__test'):
-            self._drop_schema_named(self.default_database, self.other_schema)
-
-        super().tearDown()
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seeds': {
-                'test': {
-                    'quote_columns': False,
-                }
-            }
-        }
-
-    def get_profile(self, adapter_type):
-        if self.other_schema is None:
-            self.other_schema = self.unique_schema() + '_other'
-        profile = super().get_profile(adapter_type)
-        default_name = profile['test']['target']
-        profile['test']['outputs']['otherschema'] = copy.deepcopy(profile['test']['outputs'][default_name])
-        profile['test']['outputs']['otherschema']['schema'] = self.other_schema
-        return profile
-
-    def copy_state(self):
-        assert not os.path.exists('state')
-        os.makedirs('state')
-        shutil.copyfile('target/manifest.json', 'state/manifest.json')
-
-    def run_and_defer(self):
-        results = self.run_dbt(['seed'])
-        assert len(results) == 1
-        assert not any(r.node.deferred for r in results)
-        results = self.run_dbt(['run'])
-        assert len(results) == 2
-        assert not any(r.node.deferred for r in results)
-        results = self.run_dbt(['test'])
-        assert len(results) == 2
-
-        # copy files over from the happy times when we had a good target
-        self.copy_state()
-
-        # test tests first, because run will change things
-        # no state, wrong schema, failure.
-        self.run_dbt(['test', '--target', 'otherschema'], expect_pass=False)
-
-        # no state, run also fails
-        self.run_dbt(['run', '--target', 'otherschema'], expect_pass=False)
-
-        # defer test, it succeeds
-        results = self.run_dbt(['test', '-m', 'view_model+', '--state', 'state', '--defer', '--target', 'otherschema'])
-
-        # with state it should work though
-        results = self.run_dbt(['run', '-m', 'view_model', '--state', 'state', '--defer', '--target', 'otherschema'])
-        assert self.other_schema not in results[0].node.compiled_code
-        assert self.unique_schema() in results[0].node.compiled_code
-
-        with open('target/manifest.json') as fp:
-            data = json.load(fp)
-        assert data['nodes']['seed.test.seed']['deferred']
-
-        assert len(results) == 1
-
-    @use_profile('bigquery')
-    def test_bigquery_state_changetarget(self):
-        self.run_and_defer()

From a7af39aef82a16895bd35d4edc97e4dba6d32a7e Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 9 Feb 2023 16:40:11 -0800
Subject: [PATCH 552/860] remove invocation of integration tests in tox (#498)

* remove invocation of integration tests in tox

* add changie
---
 .changes/unreleased/Under the Hood-20230130-170118.yaml | 7 +++++++
 CONTRIBUTING.md                                         | 4 ++--
 tox.ini                                                 | 1 -
 3 files changed, 9 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20230130-170118.yaml

diff --git a/.changes/unreleased/Under the Hood-20230130-170118.yaml b/.changes/unreleased/Under the Hood-20230130-170118.yaml
new file mode 100644
index 000000000..39e0ee8f8
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20230130-170118.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: remove tox call to integration tests
+time: 2023-01-30T17:01:18.971636-08:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "459"
+  PR: "498"
diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md
index 9b765784e..e0c905b39 100644
--- a/CONTRIBUTING.md
+++ b/CONTRIBUTING.md
@@ -73,8 +73,8 @@ There are a few methods for running tests locally.
 Finally, you can also run a specific test or group of tests using `pytest` directly. With a Python virtualenv active and dev dependencies installed you can do things like:
 
 ```sh
-# run specific bigquery integration tests
-python -m pytest -m profile_bigquery tests/integration/simple_copy_test
+# run specific bigquery functional tests
+python -m pytest -m profile_bigquery tests/functional/adapter/test_aliases.py::TestSameTestSameAliasDifferentDatabasesBigQuery
 # run all unit tests in a file
 python -m pytest tests/unit/test_bigquery_adapter.py
 # run a specific unit test
diff --git a/tox.ini b/tox.ini
index a04781985..2a28f0d57 100644
--- a/tox.ini
+++ b/tox.ini
@@ -23,7 +23,6 @@ passenv =
     DATAPROC_*
     GCS_BUCKET
 commands =
-  bigquery: {envpython} -m pytest {posargs} -m profile_bigquery tests/integration
   bigquery: {envpython} -m pytest {posargs} -vv tests/functional --profile service_account
 deps =
   -rdev-requirements.txt

From 34b0a77891f5f470389f45674e31bcf0478792fa Mon Sep 17 00:00:00 2001
From: Michelle Ark <MichelleArk@users.noreply.github.com>
Date: Fri, 10 Feb 2023 11:54:30 -0500
Subject: [PATCH 553/860] parse vars in unit test utils (#518)

* test against feature/click-cli core branch

* fix unit tests

* remove tests/integration tests

* done testing against feature branch

* Testing click cli branch

* fix unittest

* Done testing click branch, working properly

---------

Co-authored-by: Stu Kilgore <stuart.kilgore@gmail.com>
Co-authored-by: Chenyu Li <chenyu.li@dbtlabs.com>
---
 tests/unit/utils.py | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)

diff --git a/tests/unit/utils.py b/tests/unit/utils.py
index a4a382c5d..982680449 100644
--- a/tests/unit/utils.py
+++ b/tests/unit/utils.py
@@ -46,6 +46,12 @@ def profile_from_dict(profile, profile_name, cli_vars='{}'):
         cli_vars = parse_cli_vars(cli_vars)
 
     renderer = ProfileRenderer(cli_vars)
+
+    # in order to call dbt's internal profile rendering, we need to set the
+    # flags global. This is a bit of a hack, but it's the best way to do it.
+    from dbt.flags import set_from_args
+    from argparse import Namespace
+    set_from_args(Namespace(), None)
     return Profile.from_raw_profile_info(
         profile,
         profile_name,
@@ -54,7 +60,6 @@ def profile_from_dict(profile, profile_name, cli_vars='{}'):
 
 
 def project_from_dict(project, profile, packages=None, selectors=None, cli_vars='{}'):
-    from dbt.context.target import generate_target_context
     from dbt.config import Project
     from dbt.config.renderer import DbtProjectYamlRenderer
     from dbt.config.utils import parse_cli_vars
@@ -78,6 +83,9 @@ def project_from_dict(project, profile, packages=None, selectors=None, cli_vars=
 def config_from_parts_or_dicts(project, profile, packages=None, selectors=None, cli_vars='{}'):
     from dbt.config import Project, Profile, RuntimeConfig
     from copy import deepcopy
+    from dbt.config.utils import parse_cli_vars
+    if not isinstance(cli_vars, dict):
+        cli_vars = parse_cli_vars(cli_vars)
 
     if isinstance(project, Project):
         profile_name = project.profile_name

From 44c6cc671b73b4334c3502b2beca1e29a323ac92 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Mon, 13 Feb 2023 14:10:38 -0600
Subject: [PATCH 554/860] Sync Changelog team to GitHub team (#508)

* update to dynamically determine core team members

* tweak team logic a to make it more intuitive

* temporarily point to branch

* fix comments so they are not overwritten

* point workflow back to main
---
 .bumpversion.cfg |  8 ++------
 .changie.yaml    | 13 ++++++++++---
 2 files changed, 12 insertions(+), 9 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 9fe96617b..f2d5f57d4 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,17 +1,13 @@
 [bumpversion]
 current_version = 1.5.0a1
-# `parse` allows parsing the version into the parts we need to check.  There are some
-# unnamed groups and that's okay because they do not need to be audited.  If any part
-# of the version passed and does not match the regex, it will fail.
-# expected matches: `1.5.0`, `1.5.0a1`, `1.5.0a1.dev123457+nightly`
-# excepted failures: `1`, `1.5`, `1.5.2-a1`, `text1.5.0`
+
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
 	(((?P<prekind>a|b|rc) # optional pre-release type
 	?(?P<num>[\d]+?)) # optional pre-release version number
 	\.?(?P<nightly>[a-z0-9]+\+[a-z]+)? # optional nightly release indicator
-	)?
+	)? # expected matches: `1.5.0`, `1.5.0a1`, `1.5.0a1.dev123457+nightly`, expected failures: `1`, `1.5`, `1.5.2-a1`, `text1.5.0`
 serialize =
 	{major}.{minor}.{patch}{prekind}{num}.{nightly}
 	{major}.{minor}.{patch}{prekind}{num}
diff --git a/.changie.yaml b/.changie.yaml
index 37f9191de..7ca7c3797 100644
--- a/.changie.yaml
+++ b/.changie.yaml
@@ -4,6 +4,7 @@ headerPath: header.tpl.md
 versionHeaderPath: ""
 changelogPath: CHANGELOG.md
 versionExt: md
+envPrefix: "CHANGIE_"
 versionFormat: '## dbt-bigquery {{.Version}} - {{.Time.Format "January 02, 2006"}}'
 kindFormat: '### {{.Kind}}'
 changeFormat: |-
@@ -78,15 +79,21 @@ custom:
 
 footerFormat: |
   {{- $contributorDict := dict }}
-  {{- /* any names added to this list should be all lowercase for later matching purposes */}}
-  {{- $core_team := list "michelleark" "peterallenwebb" "emmyoop" "nathaniel-may" "gshank" "leahwicz" "chenyulinx" "stu-k" "iknox-fa" "versusfacit" "mcknight-42" "jtcohen6" "aranke" "dependabot[bot]" "snyk-bot" "colin-rogers-dbt" }}
+  {{- /* ensure all names in this list are all lowercase for later matching purposes */}}
+  {{- $core_team := splitList " " .Env.CORE_TEAM }}
+  {{- /* ensure we always skip snyk and dependabot in addition to the core team */}}
+  {{- $maintainers := list "dependabot[bot]" "snyk-bot"}}
+  {{- range $team_member := $core_team }}
+    {{- $team_member_lower := lower $team_member }}
+    {{- $maintainers = append $maintainers $team_member_lower }}
+  {{- end }}
   {{- range $change := .Changes }}
     {{- $authorList := splitList " " $change.Custom.Author }}
     {{- /* loop through all authors for a single changelog */}}
     {{- range $author := $authorList }}
       {{- $authorLower := lower $author }}
       {{- /* we only want to include non-core team contributors */}}
-      {{- if not (has $authorLower $core_team)}}
+      {{- if not (has $authorLower $maintainers)}}
         {{- $changeList := splitList " " $change.Custom.Author }}
           {{- $IssueList := list }}
           {{- $changeLink := $change.Kind }}

From 919a6caa05059d5acd9cb942bb9ed13792209ed5 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Tue, 14 Feb 2023 09:21:58 -0800
Subject: [PATCH 555/860] specify 1.1 in batch runtime config (#532)

* specify 1.1 in batch runtime config

* add changie

* fix quoting
---
 .changes/unreleased/Fixes-20230213-203317.yaml | 6 ++++++
 dbt/adapters/bigquery/python_submissions.py    | 3 ++-
 2 files changed, 8 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20230213-203317.yaml

diff --git a/.changes/unreleased/Fixes-20230213-203317.yaml b/.changes/unreleased/Fixes-20230213-203317.yaml
new file mode 100644
index 000000000..ae64c4e2d
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230213-203317.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Pin dataproc serverless spark runtime to `1.1`
+time: 2023-02-13T20:33:17.839861-08:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "531"
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 43b8201d1..f82b72d7c 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -121,7 +121,8 @@ def _get_job_client(self) -> dataproc_v1.BatchControllerClient:
 
     def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         # create the Dataproc Serverless job config
-        batch = dataproc_v1.Batch()
+        # need to pin dataproc version to 1.1 as it now defaults to 2.0
+        batch = dataproc_v1.Batch({"runtime_config": dataproc_v1.RuntimeConfig(version="1.1")})
         batch.pyspark_batch.main_python_file_uri = self.gcs_location
         # how to keep this up to date?
         # we should probably also open this up to be configurable

From 145bb59983bd5760046daac2d4766c78b84032eb Mon Sep 17 00:00:00 2001
From: Neelesh Salian <nssalian@users.noreply.github.com>
Date: Wed, 15 Feb 2023 08:30:14 -0800
Subject: [PATCH 556/860] [CT-2092], [CT-2093]: test conversion for
 bigquery_test integration tests (#533)

* test conversion for bigquery remainder tests

* Renamed fixtures file
---
 .../adapter/bigquery_test/fixtures.py         | 368 ++++++++++++++++++
 .../incremental_strategy_fixtures.py          | 357 +++++++++++++++++
 .../functional/adapter/bigquery_test/seeds.py |  49 +++
 .../test_incremental_strategies.py            |  66 ++++
 .../test_simple_bigquery_view.py              |  92 +++++
 .../bigquery_test/adapter-models/schema.yml   |  39 --
 .../bigquery_test/adapter-models/source.sql   |  41 --
 .../test_flattened_get_columns_in_table.sql   |  22 --
 .../test_get_columns_in_table.sql             |  10 -
 .../incremental_merge_range.sql               |  46 ---
 .../incremental_merge_time.sql                |  42 --
 .../incremental_overwrite_date.sql            |  41 --
 .../incremental_overwrite_day.sql             |  41 --
 ...tal_overwrite_day_with_copy_partitions.sql |  41 --
 .../incremental_overwrite_partitions.sql      |  42 --
 .../incremental_overwrite_range.sql           |  46 ---
 .../incremental_overwrite_time.sql            |  42 --
 .../bigquery_test/macros/test_creation.sql    |  18 -
 .../macros/test_int_inference.sql             |  36 --
 .../macros/test_project_for_job_id.sql        |   7 -
 .../bigquery_test/macros/wrapped_macros.sql   |  43 --
 .../bigquery_test/models/clustered_model.sql  |  10 -
 .../bigquery_test/models/fUnKyCaSe.sql        |   1 -
 .../bigquery_test/models/labeled_model.sql    |   8 -
 .../models/multi_clustered_model.sql          |  10 -
 .../models/partitioned_model.sql              |   9 -
 .../bigquery_test/models/schema.yml           |  60 ---
 .../bigquery_test/models/sql_header_model.sql |  14 -
 .../models/sql_header_model_incr.sql          |  17 -
 ...sql_header_model_incr_insert_overwrite.sql |  33 --
 ...der_model_incr_insert_overwrite_static.sql |  32 --
 .../bigquery_test/models/table_model.sql      |   8 -
 .../bigquery_test/models/view_model.sql       |  14 -
 .../bigquery_test/seeds/data_seed.csv         |   5 -
 .../incremental_overwrite_date_expected.csv   |   5 -
 .../incremental_overwrite_day_expected.csv    |   5 -
 .../incremental_overwrite_range_expected.csv  |   5 -
 .../incremental_overwrite_time_expected.csv   |   5 -
 .../bigquery_test/seeds/merge_expected.csv    |   7 -
 .../test_incremental_strategies.py            |  31 --
 .../test_simple_bigquery_view.py              |  71 ----
 41 files changed, 932 insertions(+), 907 deletions(-)
 create mode 100644 tests/functional/adapter/bigquery_test/fixtures.py
 create mode 100644 tests/functional/adapter/bigquery_test/incremental_strategy_fixtures.py
 create mode 100644 tests/functional/adapter/bigquery_test/seeds.py
 create mode 100644 tests/functional/adapter/bigquery_test/test_incremental_strategies.py
 create mode 100644 tests/functional/adapter/bigquery_test/test_simple_bigquery_view.py
 delete mode 100644 tests/integration/bigquery_test/adapter-models/schema.yml
 delete mode 100644 tests/integration/bigquery_test/adapter-models/source.sql
 delete mode 100644 tests/integration/bigquery_test/adapter-models/test_flattened_get_columns_in_table.sql
 delete mode 100644 tests/integration/bigquery_test/adapter-models/test_get_columns_in_table.sql
 delete mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_range.sql
 delete mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_time.sql
 delete mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_date.sql
 delete mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day.sql
 delete mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day_with_copy_partitions.sql
 delete mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_partitions.sql
 delete mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_range.sql
 delete mode 100644 tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_time.sql
 delete mode 100644 tests/integration/bigquery_test/macros/test_creation.sql
 delete mode 100644 tests/integration/bigquery_test/macros/test_int_inference.sql
 delete mode 100644 tests/integration/bigquery_test/macros/test_project_for_job_id.sql
 delete mode 100644 tests/integration/bigquery_test/macros/wrapped_macros.sql
 delete mode 100644 tests/integration/bigquery_test/models/clustered_model.sql
 delete mode 100644 tests/integration/bigquery_test/models/fUnKyCaSe.sql
 delete mode 100644 tests/integration/bigquery_test/models/labeled_model.sql
 delete mode 100644 tests/integration/bigquery_test/models/multi_clustered_model.sql
 delete mode 100644 tests/integration/bigquery_test/models/partitioned_model.sql
 delete mode 100644 tests/integration/bigquery_test/models/schema.yml
 delete mode 100644 tests/integration/bigquery_test/models/sql_header_model.sql
 delete mode 100644 tests/integration/bigquery_test/models/sql_header_model_incr.sql
 delete mode 100644 tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite.sql
 delete mode 100644 tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite_static.sql
 delete mode 100644 tests/integration/bigquery_test/models/table_model.sql
 delete mode 100644 tests/integration/bigquery_test/models/view_model.sql
 delete mode 100644 tests/integration/bigquery_test/seeds/data_seed.csv
 delete mode 100644 tests/integration/bigquery_test/seeds/incremental_overwrite_date_expected.csv
 delete mode 100644 tests/integration/bigquery_test/seeds/incremental_overwrite_day_expected.csv
 delete mode 100644 tests/integration/bigquery_test/seeds/incremental_overwrite_range_expected.csv
 delete mode 100644 tests/integration/bigquery_test/seeds/incremental_overwrite_time_expected.csv
 delete mode 100644 tests/integration/bigquery_test/seeds/merge_expected.csv
 delete mode 100644 tests/integration/bigquery_test/test_incremental_strategies.py
 delete mode 100644 tests/integration/bigquery_test/test_simple_bigquery_view.py

diff --git a/tests/functional/adapter/bigquery_test/fixtures.py b/tests/functional/adapter/bigquery_test/fixtures.py
new file mode 100644
index 000000000..af43b5f0f
--- /dev/null
+++ b/tests/functional/adapter/bigquery_test/fixtures.py
@@ -0,0 +1,368 @@
+#
+# Models
+#
+
+clustered_model_sql = """
+{{
+	config(
+		materialized = "table",
+		partition_by = {"field": "updated_at", "data_type": "date"},
+		cluster_by = "dupe",
+	)
+}}
+
+select * from {{ ref('view_model') }}
+""".lstrip()
+
+funky_case_sql = """
+select 1 as id
+""".lstrip()
+
+labeled_model_sql = """
+{{
+	config(
+		materialized = "table",
+		labels = {'town': 'fish', 'analytics': 'yes'}
+	)
+}}
+
+select * from {{ ref('view_model') }}
+""".lstrip()
+
+multi_clustered_model_sql = """
+{{
+	config(
+		materialized = "table",
+		partition_by = {"field": "updated_at", "data_type": "date"},
+		cluster_by = ["dupe","id"],
+	)
+}}
+
+select * from {{ ref('view_model') }}
+""".lstrip()
+
+
+partitioned_model_sql = """
+{{
+	config(
+		materialized = "table",
+		partition_by = {'field': 'updated_at', 'data_type': 'date'},
+	)
+}}
+
+select * from {{ ref('view_model') }}
+""".lstrip()
+
+schema_yml = """
+version: 2
+models:
+- name: view_model
+  description: |
+    View model description "with double quotes"
+    and with 'single  quotes' as welll as other;
+    '''abc123'''
+    reserved -- characters
+    --
+    /* comment */
+  columns:
+  - name: dupe
+    tests:
+    - unique
+  - name: id
+    tests:
+    - not_null
+    - unique
+  - name: updated_at
+    tests:
+    - not_null
+  tests:
+  - was_materialized:
+      name: view_model
+      type: view
+- name: table_model
+  description: |
+    View model description "with double quotes"
+    and with 'single  quotes' as welll as other;
+    '''abc123'''
+    reserved -- characters
+    --
+    /* comment */
+  columns:
+  - name: id
+    tests:
+    - not_null
+  tests:
+  - was_materialized:
+      name: table_model
+      type: table
+- name: fUnKyCaSe
+  columns:
+    - name: id
+      tests:
+        - not_null
+        - unique
+  tests:
+    - was_materialized:
+        name: fUnKyCaSe
+        type: view
+
+
+sources:
+  - name: raw
+    project: "{{ target.database }}"
+    dataset: "{{ target.schema }}"
+    tables:
+      - name: seed
+        identifier: data_seed
+""".lstrip()
+
+sql_header_model_sql = """
+{{ config(materialized="table") }}
+
+{# This will fail if it is not extracted correctly #}
+{% call set_sql_header(config) %}
+  	CREATE TEMPORARY FUNCTION a_to_b(str STRING)
+	RETURNS STRING AS (
+	  CASE
+	  WHEN LOWER(str) = 'a' THEN 'b'
+	  ELSE str
+	  END
+	);
+{% endcall %}
+
+select a_to_b(dupe) as dupe from {{ ref('view_model') }}
+""".lstrip()
+
+sql_header_model_incr_sql = """
+{{ config(materialized="incremental") }}
+
+{# This will fail if it is not extracted correctly #}
+{% call set_sql_header(config) %}
+    DECLARE int_var INT64 DEFAULT 42;
+
+  	CREATE TEMPORARY FUNCTION a_to_b(str STRING)
+	RETURNS STRING AS (
+	  CASE
+	  WHEN LOWER(str) = 'a' THEN 'b'
+	  ELSE str
+	  END
+	);
+{% endcall %}
+
+select a_to_b(dupe) as dupe from {{ ref('view_model') }}
+""".lstrip()
+
+sql_header_model_incr_insert_overwrite_sql = """
+{#
+    Ensure that the insert overwrite incremental strategy
+    works correctly when a UDF is used in a sql_header. The
+    failure mode here is that dbt might inject the UDF header
+    twice: once for the `create table` and then again for the
+    merge statement.
+#}
+
+{{ config(
+    materialized="incremental",
+    incremental_strategy='insert_overwrite',
+    partition_by={"field": "dt", "data_type": "date"}
+) }}
+
+{# This will fail if it is not extracted correctly #}
+{% call set_sql_header(config) %}
+    DECLARE int_var INT64 DEFAULT 42;
+
+  	CREATE TEMPORARY FUNCTION a_to_b(str STRING)
+	RETURNS STRING AS (
+	  CASE
+	  WHEN LOWER(str) = 'a' THEN 'b'
+	  ELSE str
+	  END
+	);
+{% endcall %}
+
+select
+    current_date() as dt,
+    a_to_b(dupe) as dupe
+
+from {{ ref('view_model') }}
+""".lstrip()
+
+sql_header_model_incr_insert_overwrite_static_sql = """
+{#
+    Ensure that the insert overwrite incremental strategy
+    works correctly when a UDF is used in a sql_header. The
+    failure mode here is that dbt might inject the UDF header
+    twice: once for the `create table` and then again for the
+    merge statement.
+#}
+
+{{ config(
+    materialized="incremental",
+    incremental_strategy='insert_overwrite',
+    partition_by={"field": "dt", "data_type": "date"},
+    partitions=["'2020-01-1'"]
+) }}
+
+{# This will fail if it is not extracted correctly #}
+{% call set_sql_header(config) %}
+  	CREATE TEMPORARY FUNCTION a_to_b(str STRING)
+	RETURNS STRING AS (
+	  CASE
+	  WHEN LOWER(str) = 'a' THEN 'b'
+	  ELSE str
+	  END
+	);
+{% endcall %}
+
+select
+    cast('2020-01-01' as date) as dt,
+    a_to_b(dupe) as dupe
+
+from {{ ref('view_model') }}
+""".lstrip()
+
+tabel_model_sql = """
+{{
+  config(
+    materialized = "table",
+    persist_docs={ "relation": true, "columns": true, "schema": true }
+  )
+}}
+
+select * from {{ ref('view_model') }}
+""".lstrip()
+
+view_model_sql = """
+{{
+  config(
+    materialized = "view",
+    persist_docs={ "relation": true, "columns": true, "schema": true }
+  )
+}}
+
+
+select
+    id,
+    current_date as updated_at,
+    dupe
+
+from {{ source('raw', 'seed') }}
+""".lstrip()
+
+#
+# Macros
+#
+
+test_creation_sql = """
+{% test was_materialized(model, name, type) %}
+
+    {#-- don't run this query in the parsing step #}
+    {%- if model -%}
+        {%- set table = adapter.get_relation(database=model.database, schema=model.schema,
+                                             identifier=model.name) -%}
+    {%- else -%}
+        {%- set table = {} -%}
+    {%- endif -%}
+
+    {% if table %}
+      select '{{ table.type }} does not match expected value {{ type }}'
+      from (select true)
+      where '{{ table.type }}' != '{{ type }}'
+    {% endif %}
+
+{% endtest %}
+""".lstrip()
+
+test_int_inference_sql = """
+{% macro assert_eq(value, expected, msg) %}
+    {% if value != expected %}
+        {% do exceptions.raise_compiler_error(msg ~ value) %}
+    {% endif %}
+{% endmacro %}
+
+
+{% macro test_int_inference() %}
+
+    {% set sql %}
+        select
+            0 as int_0,
+            1 as int_1,
+            2 as int_2
+    {% endset %}
+
+    {% set result = run_query(sql) %}
+    {% do assert_eq((result | length), 1, 'expected 1 result, got ') %}
+    {% set actual_0 = result[0]['int_0'] %}
+    {% set actual_1 = result[0]['int_1'] %}
+    {% set actual_2 = result[0]['int_2'] %}
+
+    {% do assert_eq(actual_0, 0, 'expected expected actual_0 to be 0, it was ') %}
+    {% do assert_eq((actual_0 | string), '0', 'expected string form of actual_0 to be 0, it was ') %}
+    {% do assert_eq((actual_0 * 2), 0, 'expected actual_0 * 2 to be 0, it was ') %} {# not 00 #}
+
+    {% do assert_eq(actual_1, 1, 'expected actual_1 to be 1, it was ') %}
+    {% do assert_eq((actual_1 | string), '1', 'expected string form of actual_1 to be 1, it was ') %}
+    {% do assert_eq((actual_1 * 2), 2, 'expected actual_1 * 2 to be 2, it was ') %} {# not 11 #}
+
+    {% do assert_eq(actual_2, 2, 'expected actual_2 to be 2, it was ') %}
+    {% do assert_eq((actual_2 | string), '2', 'expected string form of actual_2 to be 2, it was ') %}
+    {% do assert_eq((actual_2 * 2), 4, 'expected actual_2 * 2 to be 4, it was ') %}  {# not 22 #}
+
+{% endmacro %}
+""".lstrip()
+
+test_project_for_job_id_sql = """
+{% test project_for_job_id(model, region, unique_schema_id, project_id) %}
+select 1
+from `region-us`.INFORMATION_SCHEMA.JOBS_BY_PROJECT
+where date(creation_time) = current_date
+  and job_project = {{project_id}}
+  and destination_table.dataset_id = {{unique_schema_id}}
+{% endtest %}
+""".lstrip()
+
+wrapped_macros_sql = """
+{% macro my_create_schema(db_name, schema_name) %}
+    {% if not execute %}
+        {% do return(None) %}
+    {% endif %}
+    {% set relation = api.Relation.create(database=db_name, schema=schema_name).without_identifier() %}
+    {% do create_schema(relation) %}
+{% endmacro %}
+
+{% macro my_drop_schema(db_name, schema_name) %}
+    {% if not execute %}
+        {% do return(None) %}
+    {% endif %}
+    {% set relation = api.Relation.create(database=db_name, schema=schema_name).without_identifier() %}
+    {% do drop_schema(relation) %}
+{% endmacro %}
+
+
+{% macro my_create_table_as(db_name, schema_name, table_name) %}
+    {% if not execute %}
+        {% do return(None) %}
+    {% endif %}
+    {% set relation = api.Relation.create(database=db_name, schema=schema_name, identifier=table_name) %}
+    {% do run_query(create_table_as(false, relation, 'select 1 as id')) %}
+{% endmacro %}
+
+
+{% macro ensure_one_relation_in(db_name, schema_name) %}
+    {% if not execute %}
+        {% do return(None) %}
+    {% endif %}
+    {% set relation = api.Relation.create(database=db_name, schema=schema_name).without_identifier() %}
+    {% set results = list_relations_without_caching(relation) %}
+    {% set rlen = (results | length) %}
+    {% if rlen != 1 %}
+        {% do exceptions.raise_compiler_error('Incorect number of results (expected 1): ' ~ rlen) %}
+    {% endif %}
+    {% set result = results[0] %}
+    {% set columns = get_columns_in_relation(result) %}
+    {% set clen = (columns | length) %}
+    {% if clen != 1 %}
+        {% do exceptions.raise_compiler_error('Incorrect number of columns (expected 1): ' ~ clen) %}
+    {% endif %}
+{% endmacro %}
+""".lstrip()
\ No newline at end of file
diff --git a/tests/functional/adapter/bigquery_test/incremental_strategy_fixtures.py b/tests/functional/adapter/bigquery_test/incremental_strategy_fixtures.py
new file mode 100644
index 000000000..7baee6c3d
--- /dev/null
+++ b/tests/functional/adapter/bigquery_test/incremental_strategy_fixtures.py
@@ -0,0 +1,357 @@
+merge_range_sql = """
+{{
+    config(
+        materialized="incremental",
+        unique_key="id",
+        cluster_by="id",
+        partition_by={
+            "field": "id",
+            "data_type": "int64",
+            "range": {
+                "start": 1,
+                "end": 10,
+                "interval": 1
+            }
+        }
+    )
+}}
+
+
+with data as (
+    
+    {% if not is_incremental() %}
+    
+        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 4 as id, cast('2020-01-01' as datetime) as date_time
+    
+    {% else %}
+    
+        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 4 as id, cast('2020-01-02' as datetime) as date_time union all
+        select 5 as id, cast('2020-01-02' as datetime) as date_time union all
+        select 6 as id, cast('2020-01-02' as datetime) as date_time
+    
+    {% endif %}
+
+)
+
+select * from data
+
+{% if is_incremental() %}
+where id >= (select max(id) from {{ this }})
+{% endif %}
+""".lstrip()
+
+merge_time_sql = """
+{{
+    config(
+        materialized="incremental",
+        unique_key="id",
+        cluster_by="id",
+        partition_by={
+            "field": "date_time",
+            "data_type": "datetime"
+        }
+    )
+}}
+
+
+
+with data as (
+    
+    {% if not is_incremental() %}
+    
+        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 4 as id, cast('2020-01-01' as datetime) as date_time
+    
+    {% else %}
+    
+        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 4 as id, cast('2020-01-02' as datetime) as date_time union all
+        select 5 as id, cast('2020-01-02' as datetime) as date_time union all
+        select 6 as id, cast('2020-01-02' as datetime) as date_time
+    
+    {% endif %}
+
+)
+
+select * from data
+
+{% if is_incremental() %}
+where date_time > (select max(date_time) from {{ this }})
+{% endif %}
+""".lstrip()
+
+overwrite_date_sql = """
+{{
+    config(
+        materialized="incremental",
+        incremental_strategy='insert_overwrite',
+        cluster_by="id",
+        partition_by={
+            "field": "date_day",
+            "data_type": "date"
+        }
+    )
+}}
+
+
+with data as (
+    
+    {% if not is_incremental() %}
+    
+        select 1 as id, cast('2020-01-01' as date) as date_day union all
+        select 2 as id, cast('2020-01-01' as date) as date_day union all
+        select 3 as id, cast('2020-01-01' as date) as date_day union all
+        select 4 as id, cast('2020-01-01' as date) as date_day
+
+    {% else %}
+    
+        -- we want to overwrite the 4 records in the 2020-01-01 partition
+        -- with the 2 records below, but add two more in the 2020-01-02 partition
+        select 10 as id, cast('2020-01-01' as date) as date_day union all
+        select 20 as id, cast('2020-01-01' as date) as date_day union all
+        select 30 as id, cast('2020-01-02' as date) as date_day union all
+        select 40 as id, cast('2020-01-02' as date) as date_day
+    
+    {% endif %}
+
+)
+
+select * from data
+
+{% if is_incremental() %}
+where date_day >= _dbt_max_partition
+{% endif %}
+""".lstrip()
+
+overwrite_day_sql = """
+{{
+    config(
+        materialized="incremental",
+        incremental_strategy='insert_overwrite',
+        cluster_by="id",
+        partition_by={
+            "field": "date_time",
+            "data_type": "datetime"
+        }
+    )
+}}
+
+
+with data as (
+    
+    {% if not is_incremental() %}
+    
+        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 4 as id, cast('2020-01-01' as datetime) as date_time
+
+    {% else %}
+        
+        -- we want to overwrite the 4 records in the 2020-01-01 partition
+        -- with the 2 records below, but add two more in the 2020-01-02 partition
+        select 10 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 20 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 30 as id, cast('2020-01-02' as datetime) as date_time union all
+        select 40 as id, cast('2020-01-02' as datetime) as date_time
+    
+    {% endif %}
+
+)
+
+select * from data
+
+{% if is_incremental() %}
+where date_time >= _dbt_max_partition
+{% endif %}
+""".lstrip()
+
+overwrite_day_with_copy_partitions_sql = """
+{{
+    config(
+        materialized="incremental",
+        incremental_strategy='insert_overwrite',
+        cluster_by="id",
+        partition_by={
+            "field": "date_time",
+            "data_type": "datetime",
+            "copy_partitions": true
+        }
+    )
+}}
+
+
+with data as (
+
+    {% if not is_incremental() %}
+
+        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 4 as id, cast('2020-01-01' as datetime) as date_time
+
+    {% else %}
+
+        -- we want to overwrite the 4 records in the 2020-01-01 partition
+        -- with the 2 records below, but add two more in the 2020-01-02 partition
+        select 10 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 20 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 30 as id, cast('2020-01-02' as datetime) as date_time union all
+        select 40 as id, cast('2020-01-02' as datetime) as date_time
+
+    {% endif %}
+
+)
+
+select * from data
+
+{% if is_incremental() %}
+where date_time >= _dbt_max_partition
+{% endif %}
+""".lstrip()
+
+overwrite_partitions_sql = """
+{{
+    config(
+        materialized="incremental",
+        incremental_strategy='insert_overwrite',
+        cluster_by="id",
+        partitions=["'2020-01-01'","'2020-01-02'"],
+        partition_by={
+            "field": "date_day",
+            "data_type": "date"
+        }
+    )
+}}
+
+
+with data as (
+    
+    {% if not is_incremental() %}
+    
+        select 1 as id, cast('2020-01-01' as date) as date_day union all
+        select 2 as id, cast('2020-01-01' as date) as date_day union all
+        select 3 as id, cast('2020-01-01' as date) as date_day union all
+        select 4 as id, cast('2020-01-01' as date) as date_day
+
+    {% else %}
+        
+        -- we want to overwrite the 4 records in the 2020-01-01 partition
+        -- with the 2 records below, but add two more in the 2020-01-02 partition
+        select 10 as id, cast('2020-01-01' as date) as date_day union all
+        select 20 as id, cast('2020-01-01' as date) as date_day union all
+        select 30 as id, cast('2020-01-02' as date) as date_day union all
+        select 40 as id, cast('2020-01-02' as date) as date_day
+    
+    {% endif %}
+
+)
+
+select * from data
+
+{% if is_incremental() %}
+where date_day in ({{ config.get("partitions") | join(",") }})
+{% endif %}
+""".lstrip()
+
+overwrite_range_sql = """
+{{
+    config(
+        materialized="incremental",
+        incremental_strategy='insert_overwrite',
+        cluster_by="id",
+        partition_by={
+            "field": "date_int",
+            "data_type": "int64",
+            "range": {
+                "start": 20200101,
+                "end": 20200110,
+                "interval": 1
+            }
+        }
+    )
+}}
+
+
+with data as (
+    
+    {% if not is_incremental() %}
+    
+        select 1 as id, 20200101 as date_int union all
+        select 2 as id, 20200101 as date_int union all
+        select 3 as id, 20200101 as date_int union all
+        select 4 as id, 20200101 as date_int
+
+    {% else %}
+        
+        -- we want to overwrite the 4 records in the 20200101 partition
+        -- with the 2 records below, but add two more in the 20200102 partition
+        select 10 as id, 20200101 as date_int union all
+        select 20 as id, 20200101 as date_int union all
+        select 30 as id, 20200102 as date_int union all
+        select 40 as id, 20200102 as date_int
+    
+    {% endif %}
+
+)
+
+select * from data
+
+{% if is_incremental() %}
+where date_int >= _dbt_max_partition
+{% endif %}
+""".lstrip()
+
+overwrite_time_sql = """
+{{
+    config(
+        materialized="incremental",
+        incremental_strategy='insert_overwrite',
+        cluster_by="id",
+        partition_by={
+            "field": "date_hour",
+            "data_type": "datetime",
+            "granularity": "hour"
+        }
+    )
+}}
+
+
+with data as (
+    
+    {% if not is_incremental() %}
+    
+        select 1 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 2 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 3 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 4 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour
+
+    {% else %}
+    
+        -- we want to overwrite the 4 records in the 2020-01-01 01:00:00 partition
+        -- with the 2 records below, but add two more in the 2020-01-00 02:00:00 partition
+        select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+        select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour union all
+        select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+    
+    {% endif %}
+
+)
+
+select * from data
+
+{% if is_incremental() %}
+where date_hour >= _dbt_max_partition
+{% endif %}
+""".lstrip()
diff --git a/tests/functional/adapter/bigquery_test/seeds.py b/tests/functional/adapter/bigquery_test/seeds.py
new file mode 100644
index 000000000..f7e7bb9d3
--- /dev/null
+++ b/tests/functional/adapter/bigquery_test/seeds.py
@@ -0,0 +1,49 @@
+seed_data_csv = """
+id,dupe
+1,a
+2,a
+3,a
+4,a
+""".lstrip()
+
+seed_incremental_overwrite_date_expected_csv = """
+id,date_day
+10,2020-01-01
+20,2020-01-01
+30,2020-01-02
+40,2020-01-02
+""".lstrip()
+
+seed_incremental_overwrite_day_expected_csv = """
+id,date_time
+10,2020-01-01 00:00:00
+20,2020-01-01 00:00:00
+30,2020-01-02 00:00:00
+40,2020-01-02 00:00:00
+""".lstrip()
+
+seed_incremental_overwrite_range_expected_csv = """
+id,date_int
+10,20200101
+20,20200101
+30,20200102
+40,20200102
+""".lstrip()
+
+seed_incremental_overwrite_time_expected_csv = """
+id,date_hour
+10,2020-01-01 01:00:00
+20,2020-01-01 01:00:00
+30,2020-01-01 02:00:00
+40,2020-01-01 02:00:00
+""".lstrip()
+
+seed_merge_expected_csv = """
+id,date_time
+1,2020-01-01 00:00:00
+2,2020-01-01 00:00:00
+3,2020-01-01 00:00:00
+4,2020-01-02 00:00:00
+5,2020-01-02 00:00:00
+6,2020-01-02 00:00:00
+""".lstrip()
diff --git a/tests/functional/adapter/bigquery_test/test_incremental_strategies.py b/tests/functional/adapter/bigquery_test/test_incremental_strategies.py
new file mode 100644
index 000000000..76b7ac054
--- /dev/null
+++ b/tests/functional/adapter/bigquery_test/test_incremental_strategies.py
@@ -0,0 +1,66 @@
+import pytest
+from dbt.tests.util import (
+    check_relations_equal,
+    run_dbt
+)
+from dbt.tests.adapter.simple_seed.test_seed import SeedConfigBase
+from tests.functional.adapter.bigquery_test.seeds import *
+from tests.functional.adapter.bigquery_test.incremental_strategy_fixtures import *
+
+
+class TestBigQueryScripting(SeedConfigBase):
+    @pytest.fixture(scope="class")
+    def schema(self):
+        return "bigquery_test"
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "incremental_merge_range.sql": merge_range_sql,
+            "incremental_merge_time.sql": merge_time_sql,
+            "incremental_overwrite_date.sql": overwrite_date_sql,
+            "incremental_overwrite_day.sql": overwrite_day_sql,
+            "incremental_overwrite_day_with_copy_partitions.sql": overwrite_day_with_copy_partitions_sql,
+            "incremental_overwrite_partitions.sql": overwrite_partitions_sql,
+            "incremental_overwrite_range.sql": overwrite_range_sql,
+            "incremental_overwrite_time.sql": overwrite_time_sql,
+        }
+
+    @pytest.fixture(scope="class")
+    def seeds(self):
+        return {
+            "data_seed.csv": seed_data_csv,
+            "merge_expected.csv": seed_merge_expected_csv,
+            "incremental_overwrite_time_expected.csv": seed_incremental_overwrite_time_expected_csv,
+            "incremental_overwrite_date_expected.csv": seed_incremental_overwrite_date_expected_csv,
+            "incremental_overwrite_day_expected.csv": seed_incremental_overwrite_day_expected_csv,
+            "incremental_overwrite_range_expected.csv": seed_incremental_overwrite_range_expected_csv
+        }
+
+    def test__bigquery_assert_incrementals(self, project):
+        results = run_dbt()
+        assert len(results) == 8
+
+        results = run_dbt()
+        assert len(results) == 8
+
+        seed_results = run_dbt(['seed'])
+
+        db_with_schema = f"{project.database}.{project.test_schema}"
+        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_merge_range",
+                                                f"{db_with_schema}.merge_expected"])
+        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_merge_time",
+                                                f"{db_with_schema}.merge_expected"])
+        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_overwrite_time",
+                                                f"{db_with_schema}.incremental_overwrite_time_expected"])
+        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_overwrite_date",
+                                                f"{db_with_schema}.incremental_overwrite_date_expected"])
+        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_overwrite_partitions",
+                                                f"{db_with_schema}.incremental_overwrite_date_expected"])
+        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_overwrite_day",
+                                                f"{db_with_schema}.incremental_overwrite_day_expected"])
+        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_overwrite_range",
+                                                f"{db_with_schema}.incremental_overwrite_range_expected"])
+        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_overwrite_day_with_copy_partitions",
+                                                f"{db_with_schema}.incremental_overwrite_day_expected"])
+
diff --git a/tests/functional/adapter/bigquery_test/test_simple_bigquery_view.py b/tests/functional/adapter/bigquery_test/test_simple_bigquery_view.py
new file mode 100644
index 000000000..fd2a9f0f6
--- /dev/null
+++ b/tests/functional/adapter/bigquery_test/test_simple_bigquery_view.py
@@ -0,0 +1,92 @@
+import pytest
+import random
+import time
+from dbt.tests.util import run_dbt
+from dbt.tests.adapter.simple_seed.test_seed import SeedConfigBase
+from tests.functional.adapter.bigquery_test.seeds import *
+from tests.functional.adapter.bigquery_test.fixtures import *
+
+
+class BaseBigQueryRun(SeedConfigBase):
+    @pytest.fixture(scope="class")
+    def schema(self):
+        return "bigquery_test"
+
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {
+            "test_creation.sql": test_creation_sql,
+            "test_int_inference.sql": test_int_inference_sql,
+            "test_project_for_job_id.sql": test_project_for_job_id_sql,
+            "wrapped_macros.sql": wrapped_macros_sql
+        }
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "clustered_model.sql": clustered_model_sql,
+            "fUnKyCaSe.sql": funky_case_sql,
+            "labeled_model.sql": labeled_model_sql,
+            "multi_clustered_model.sql": multi_clustered_model_sql,
+            "partitioned_model.sql": partitioned_model_sql,
+            "sql_header_model.sql": sql_header_model_sql,
+            "sql_header_model_incr.sql": sql_header_model_incr_sql,
+            "sql_header_model_incr_insert_overwrite.sql": sql_header_model_incr_insert_overwrite_sql,
+            "sql_header_model_incr_insert_overwrite_static.sql": sql_header_model_incr_insert_overwrite_static_sql,
+            "table_model.sql": tabel_model_sql,
+            "view_model.sql": view_model_sql,
+            "schema.yml": schema_yml
+        }
+
+    @pytest.fixture(scope="class")
+    def seeds(self):
+        return {
+            "data_seed.csv": seed_data_csv,
+            "merge_expected.csv": seed_merge_expected_csv,
+            "incremental_overwrite_time_expected.csv": seed_incremental_overwrite_time_expected_csv,
+            "incremental_overwrite_date_expected.csv": seed_incremental_overwrite_date_expected_csv,
+            "incremental_overwrite_day_expected.csv": seed_incremental_overwrite_day_expected_csv,
+            "incremental_overwrite_range_expected.csv": seed_incremental_overwrite_range_expected_csv
+        }
+
+    def assert_nondupes_pass(self, project):
+        # The 'dupe' model should fail, but all others should pass
+        test_results = run_dbt(['test'], expect_pass=False)
+
+        for test_result in test_results:
+            if 'dupe' in test_result.node.name:
+                assert test_result.status == 'fail'
+                assert not test_result.skipped
+                assert test_result.failures > 0
+
+            # assert that actual tests pass
+            else:
+                assert test_result.status == 'pass'
+                assert not test_result.skipped
+                assert test_result.failures == 0
+
+
+class TestSimpleBigQueryRun(BaseBigQueryRun):
+    def test__bigquery_simple_run(self, project):
+        # make sure seed works twice. Full-refresh is a no-op
+        run_dbt(['seed'])
+        run_dbt(['seed', '--full-refresh'])
+
+        results = run_dbt()
+        # Bump expected number of results when adding new model
+        assert len(results) == 11
+        self.assert_nondupes_pass(project)
+
+
+class TestUnderscoreBigQueryRun(BaseBigQueryRun):
+    prefix = "_test{}{:04}".format(int(time.time()), random.randint(0, 9999))
+
+    def test_bigquery_run_twice(self, project):
+        run_dbt(['seed'])
+        results = run_dbt()
+        assert len(results) == 11
+
+        results = run_dbt()
+        assert len(results) == 11
+
+        self.assert_nondupes_pass(project)
diff --git a/tests/integration/bigquery_test/adapter-models/schema.yml b/tests/integration/bigquery_test/adapter-models/schema.yml
deleted file mode 100644
index 97a54b3c7..000000000
--- a/tests/integration/bigquery_test/adapter-models/schema.yml
+++ /dev/null
@@ -1,39 +0,0 @@
-version: 2
-models:
-- name: test_get_columns_in_table
-  columns:
-  - name: field_1
-    tests:
-    - not_null
-  - name: field_2
-    tests:
-    - not_null
-  - name: field_3
-    tests:
-    - not_null
-  - name: nested_field
-    tests:
-    - not_null
-  - name: repeated_column
-    tests:
-    - not_null
-- name: test_flattened_get_columns_in_table
-  columns:
-  - name: field_1
-    tests:
-    - not_null
-  - name: field_2
-    tests:
-    - not_null
-  - name: field_3
-    tests:
-    - not_null
-  - name: field_4
-    tests:
-    - not_null
-  - name: field_5
-    tests:
-    - not_null
-  - name: repeated_column
-    tests:
-    - not_null
diff --git a/tests/integration/bigquery_test/adapter-models/source.sql b/tests/integration/bigquery_test/adapter-models/source.sql
deleted file mode 100644
index 2c419f25d..000000000
--- a/tests/integration/bigquery_test/adapter-models/source.sql
+++ /dev/null
@@ -1,41 +0,0 @@
-
-with nested_base as (
-  select
-    struct(
-      'a' as field_a,
-      'b' as field_b
-     ) as repeated_nested
-
-   union all
-
-   select
-     struct(
-      'a' as field_a,
-      'b' as field_b
-     ) as repeated_nested
-),
-
-nested as (
-
-  select
-    array_agg(repeated_nested) as repeated_column
-
-  from nested_base
-
-),
-
-base as (
-
-  select
-      1 as field_1,
-      2 as field_2,
-      3 as field_3,
-
-      struct(
-          4 as field_4,
-          5 as field_5
-      ) as nested_field
-)
-
-select *
-from base, nested
diff --git a/tests/integration/bigquery_test/adapter-models/test_flattened_get_columns_in_table.sql b/tests/integration/bigquery_test/adapter-models/test_flattened_get_columns_in_table.sql
deleted file mode 100644
index 3572ad6dc..000000000
--- a/tests/integration/bigquery_test/adapter-models/test_flattened_get_columns_in_table.sql
+++ /dev/null
@@ -1,22 +0,0 @@
-{% set source = ref('source') %}
-{% set cols = adapter.get_columns_in_relation(source) %}
-
-{{ log('source') }}
-{{ log(source) }}
-
-{% set flattened = [] %}
-{% for col in cols %}
-    {% if col.mode == 'REPEATED' %}
-        {% set _ = flattened.append(col) %}
-    {% else %}
-        {% set _ = flattened.extend(col.flatten()) %}
-    {% endif %}
-{% endfor %}
-
-select
-    {% for col in flattened %}
-        {{ col.name }}
-        {% if not loop.last %}, {% endif %}
-    {% endfor %}
-
-from {{ source }}
diff --git a/tests/integration/bigquery_test/adapter-models/test_get_columns_in_table.sql b/tests/integration/bigquery_test/adapter-models/test_get_columns_in_table.sql
deleted file mode 100644
index 8135dbe9a..000000000
--- a/tests/integration/bigquery_test/adapter-models/test_get_columns_in_table.sql
+++ /dev/null
@@ -1,10 +0,0 @@
-{% set source = ref('source') %}
-{% set cols = adapter.get_columns_in_relation(source) %}
-
-select
-    {% for col in cols %}
-        {{ col.name }}
-        {% if not loop.last %}, {% endif %}
-    {% endfor %}
-
-from {{ source }}
diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_range.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_range.sql
deleted file mode 100644
index 581007c01..000000000
--- a/tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_range.sql
+++ /dev/null
@@ -1,46 +0,0 @@
-
-{{
-    config(
-        materialized="incremental",
-        unique_key="id",
-        cluster_by="id",
-        partition_by={
-            "field": "id",
-            "data_type": "int64",
-            "range": {
-                "start": 1,
-                "end": 10,
-                "interval": 1
-            }
-        }
-    )
-}}
-
-
-with data as (
-    
-    {% if not is_incremental() %}
-    
-        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 4 as id, cast('2020-01-01' as datetime) as date_time
-    
-    {% else %}
-    
-        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 4 as id, cast('2020-01-02' as datetime) as date_time union all
-        select 5 as id, cast('2020-01-02' as datetime) as date_time union all
-        select 6 as id, cast('2020-01-02' as datetime) as date_time
-    
-    {% endif %}
-
-)
-
-select * from data
-
-{% if is_incremental() %}
-where id >= (select max(id) from {{ this }})
-{% endif %}
diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_time.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_time.sql
deleted file mode 100644
index 7629a9904..000000000
--- a/tests/integration/bigquery_test/incremental-strategy-models/incremental_merge_time.sql
+++ /dev/null
@@ -1,42 +0,0 @@
-
-{{
-    config(
-        materialized="incremental",
-        unique_key="id",
-        cluster_by="id",
-        partition_by={
-            "field": "date_time",
-            "data_type": "datetime"
-        }
-    )
-}}
-
-
-
-with data as (
-    
-    {% if not is_incremental() %}
-    
-        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 4 as id, cast('2020-01-01' as datetime) as date_time
-    
-    {% else %}
-    
-        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 4 as id, cast('2020-01-02' as datetime) as date_time union all
-        select 5 as id, cast('2020-01-02' as datetime) as date_time union all
-        select 6 as id, cast('2020-01-02' as datetime) as date_time
-    
-    {% endif %}
-
-)
-
-select * from data
-
-{% if is_incremental() %}
-where date_time > (select max(date_time) from {{ this }})
-{% endif %}
diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_date.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_date.sql
deleted file mode 100644
index df67599ef..000000000
--- a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_date.sql
+++ /dev/null
@@ -1,41 +0,0 @@
-
-{{
-    config(
-        materialized="incremental",
-        incremental_strategy='insert_overwrite',
-        cluster_by="id",
-        partition_by={
-            "field": "date_day",
-            "data_type": "date"
-        }
-    )
-}}
-
-
-with data as (
-    
-    {% if not is_incremental() %}
-    
-        select 1 as id, cast('2020-01-01' as date) as date_day union all
-        select 2 as id, cast('2020-01-01' as date) as date_day union all
-        select 3 as id, cast('2020-01-01' as date) as date_day union all
-        select 4 as id, cast('2020-01-01' as date) as date_day
-
-    {% else %}
-    
-        -- we want to overwrite the 4 records in the 2020-01-01 partition
-        -- with the 2 records below, but add two more in the 2020-01-02 partition
-        select 10 as id, cast('2020-01-01' as date) as date_day union all
-        select 20 as id, cast('2020-01-01' as date) as date_day union all
-        select 30 as id, cast('2020-01-02' as date) as date_day union all
-        select 40 as id, cast('2020-01-02' as date) as date_day
-    
-    {% endif %}
-
-)
-
-select * from data
-
-{% if is_incremental() %}
-where date_day >= _dbt_max_partition
-{% endif %}
diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day.sql
deleted file mode 100644
index 3434a5b47..000000000
--- a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day.sql
+++ /dev/null
@@ -1,41 +0,0 @@
-
-{{
-    config(
-        materialized="incremental",
-        incremental_strategy='insert_overwrite',
-        cluster_by="id",
-        partition_by={
-            "field": "date_time",
-            "data_type": "datetime"
-        }
-    )
-}}
-
-
-with data as (
-    
-    {% if not is_incremental() %}
-    
-        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 4 as id, cast('2020-01-01' as datetime) as date_time
-
-    {% else %}
-        
-        -- we want to overwrite the 4 records in the 2020-01-01 partition
-        -- with the 2 records below, but add two more in the 2020-01-02 partition
-        select 10 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 20 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 30 as id, cast('2020-01-02' as datetime) as date_time union all
-        select 40 as id, cast('2020-01-02' as datetime) as date_time
-    
-    {% endif %}
-
-)
-
-select * from data
-
-{% if is_incremental() %}
-where date_time >= _dbt_max_partition
-{% endif %}
diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day_with_copy_partitions.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day_with_copy_partitions.sql
deleted file mode 100644
index 77e7c0f52..000000000
--- a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_day_with_copy_partitions.sql
+++ /dev/null
@@ -1,41 +0,0 @@
-{{
-    config(
-        materialized="incremental",
-        incremental_strategy='insert_overwrite',
-        cluster_by="id",
-        partition_by={
-            "field": "date_time",
-            "data_type": "datetime",
-            "copy_partitions": true
-        }
-    )
-}}
-
-
-with data as (
-
-    {% if not is_incremental() %}
-
-        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 4 as id, cast('2020-01-01' as datetime) as date_time
-
-    {% else %}
-
-        -- we want to overwrite the 4 records in the 2020-01-01 partition
-        -- with the 2 records below, but add two more in the 2020-01-02 partition
-        select 10 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 20 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 30 as id, cast('2020-01-02' as datetime) as date_time union all
-        select 40 as id, cast('2020-01-02' as datetime) as date_time
-
-    {% endif %}
-
-)
-
-select * from data
-
-{% if is_incremental() %}
-where date_time >= _dbt_max_partition
-{% endif %}
diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_partitions.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_partitions.sql
deleted file mode 100644
index ef6cf8883..000000000
--- a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_partitions.sql
+++ /dev/null
@@ -1,42 +0,0 @@
-
-{{
-    config(
-        materialized="incremental",
-        incremental_strategy='insert_overwrite',
-        cluster_by="id",
-        partitions=["'2020-01-01'","'2020-01-02'"],
-        partition_by={
-            "field": "date_day",
-            "data_type": "date"
-        }
-    )
-}}
-
-
-with data as (
-    
-    {% if not is_incremental() %}
-    
-        select 1 as id, cast('2020-01-01' as date) as date_day union all
-        select 2 as id, cast('2020-01-01' as date) as date_day union all
-        select 3 as id, cast('2020-01-01' as date) as date_day union all
-        select 4 as id, cast('2020-01-01' as date) as date_day
-
-    {% else %}
-        
-        -- we want to overwrite the 4 records in the 2020-01-01 partition
-        -- with the 2 records below, but add two more in the 2020-01-02 partition
-        select 10 as id, cast('2020-01-01' as date) as date_day union all
-        select 20 as id, cast('2020-01-01' as date) as date_day union all
-        select 30 as id, cast('2020-01-02' as date) as date_day union all
-        select 40 as id, cast('2020-01-02' as date) as date_day
-    
-    {% endif %}
-
-)
-
-select * from data
-
-{% if is_incremental() %}
-where date_day in ({{ config.get("partitions") | join(",") }})
-{% endif %}
diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_range.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_range.sql
deleted file mode 100644
index 44d3ffc86..000000000
--- a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_range.sql
+++ /dev/null
@@ -1,46 +0,0 @@
-
-{{
-    config(
-        materialized="incremental",
-        incremental_strategy='insert_overwrite',
-        cluster_by="id",
-        partition_by={
-            "field": "date_int",
-            "data_type": "int64",
-            "range": {
-                "start": 20200101,
-                "end": 20200110,
-                "interval": 1
-            }
-        }
-    )
-}}
-
-
-with data as (
-    
-    {% if not is_incremental() %}
-    
-        select 1 as id, 20200101 as date_int union all
-        select 2 as id, 20200101 as date_int union all
-        select 3 as id, 20200101 as date_int union all
-        select 4 as id, 20200101 as date_int
-
-    {% else %}
-        
-        -- we want to overwrite the 4 records in the 20200101 partition
-        -- with the 2 records below, but add two more in the 20200102 partition
-        select 10 as id, 20200101 as date_int union all
-        select 20 as id, 20200101 as date_int union all
-        select 30 as id, 20200102 as date_int union all
-        select 40 as id, 20200102 as date_int
-    
-    {% endif %}
-
-)
-
-select * from data
-
-{% if is_incremental() %}
-where date_int >= _dbt_max_partition
-{% endif %}
diff --git a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_time.sql b/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_time.sql
deleted file mode 100644
index 77f901894..000000000
--- a/tests/integration/bigquery_test/incremental-strategy-models/incremental_overwrite_time.sql
+++ /dev/null
@@ -1,42 +0,0 @@
-
-{{
-    config(
-        materialized="incremental",
-        incremental_strategy='insert_overwrite',
-        cluster_by="id",
-        partition_by={
-            "field": "date_hour",
-            "data_type": "datetime",
-            "granularity": "hour"
-        }
-    )
-}}
-
-
-with data as (
-    
-    {% if not is_incremental() %}
-    
-        select 1 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 2 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 3 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 4 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour
-
-    {% else %}
-    
-        -- we want to overwrite the 4 records in the 2020-01-01 01:00:00 partition
-        -- with the 2 records below, but add two more in the 2020-01-00 02:00:00 partition
-        select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour union all
-        select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
-    
-    {% endif %}
-
-)
-
-select * from data
-
-{% if is_incremental() %}
-where date_hour >= _dbt_max_partition
-{% endif %}
diff --git a/tests/integration/bigquery_test/macros/test_creation.sql b/tests/integration/bigquery_test/macros/test_creation.sql
deleted file mode 100644
index f0f238919..000000000
--- a/tests/integration/bigquery_test/macros/test_creation.sql
+++ /dev/null
@@ -1,18 +0,0 @@
-
-{% test was_materialized(model, name, type) %}
-
-    {#-- don't run this query in the parsing step #}
-    {%- if model -%}
-        {%- set table = adapter.get_relation(database=model.database, schema=model.schema,
-                                             identifier=model.name) -%}
-    {%- else -%}
-        {%- set table = {} -%}
-    {%- endif -%}
-
-    {% if table %}
-      select '{{ table.type }} does not match expected value {{ type }}'
-      from (select true)
-      where '{{ table.type }}' != '{{ type }}'
-    {% endif %}
-
-{% endtest %}
diff --git a/tests/integration/bigquery_test/macros/test_int_inference.sql b/tests/integration/bigquery_test/macros/test_int_inference.sql
deleted file mode 100644
index a1ab1c8a9..000000000
--- a/tests/integration/bigquery_test/macros/test_int_inference.sql
+++ /dev/null
@@ -1,36 +0,0 @@
-
-{% macro assert_eq(value, expected, msg) %}
-    {% if value != expected %}
-        {% do exceptions.raise_compiler_error(msg ~ value) %}
-    {% endif %}
-{% endmacro %}
-
-
-{% macro test_int_inference() %}
-
-    {% set sql %}
-        select
-            0 as int_0,
-            1 as int_1,
-            2 as int_2
-    {% endset %}
-
-    {% set result = run_query(sql) %}
-    {% do assert_eq((result | length), 1, 'expected 1 result, got ') %}
-    {% set actual_0 = result[0]['int_0'] %}
-    {% set actual_1 = result[0]['int_1'] %}
-    {% set actual_2 = result[0]['int_2'] %}
-
-    {% do assert_eq(actual_0, 0, 'expected expected actual_0 to be 0, it was ') %}
-    {% do assert_eq((actual_0 | string), '0', 'expected string form of actual_0 to be 0, it was ') %}
-    {% do assert_eq((actual_0 * 2), 0, 'expected actual_0 * 2 to be 0, it was ') %} {# not 00 #}
-
-    {% do assert_eq(actual_1, 1, 'expected actual_1 to be 1, it was ') %}
-    {% do assert_eq((actual_1 | string), '1', 'expected string form of actual_1 to be 1, it was ') %}
-    {% do assert_eq((actual_1 * 2), 2, 'expected actual_1 * 2 to be 2, it was ') %} {# not 11 #}
-
-    {% do assert_eq(actual_2, 2, 'expected actual_2 to be 2, it was ') %}
-    {% do assert_eq((actual_2 | string), '2', 'expected string form of actual_2 to be 2, it was ') %}
-    {% do assert_eq((actual_2 * 2), 4, 'expected actual_2 * 2 to be 4, it was ') %}  {# not 22 #}
-
-{% endmacro %}
diff --git a/tests/integration/bigquery_test/macros/test_project_for_job_id.sql b/tests/integration/bigquery_test/macros/test_project_for_job_id.sql
deleted file mode 100644
index 327f4e2e2..000000000
--- a/tests/integration/bigquery_test/macros/test_project_for_job_id.sql
+++ /dev/null
@@ -1,7 +0,0 @@
-{% test project_for_job_id(model, region, unique_schema_id, project_id) %}
-select 1
-from `region-us`.INFORMATION_SCHEMA.JOBS_BY_PROJECT
-where date(creation_time) = current_date
-  and job_project = {{project_id}}
-  and destination_table.dataset_id = {{unique_schema_id}}
-{% endtest %}
diff --git a/tests/integration/bigquery_test/macros/wrapped_macros.sql b/tests/integration/bigquery_test/macros/wrapped_macros.sql
deleted file mode 100644
index fc66f5a95..000000000
--- a/tests/integration/bigquery_test/macros/wrapped_macros.sql
+++ /dev/null
@@ -1,43 +0,0 @@
-{% macro my_create_schema(db_name, schema_name) %}
-    {% if not execute %}
-        {% do return(None) %}
-    {% endif %}
-    {% set relation = api.Relation.create(database=db_name, schema=schema_name).without_identifier() %}
-    {% do create_schema(relation) %}
-{% endmacro %}
-
-{% macro my_drop_schema(db_name, schema_name) %}
-    {% if not execute %}
-        {% do return(None) %}
-    {% endif %}
-    {% set relation = api.Relation.create(database=db_name, schema=schema_name).without_identifier() %}
-    {% do drop_schema(relation) %}
-{% endmacro %}
-
-
-{% macro my_create_table_as(db_name, schema_name, table_name) %}
-    {% if not execute %}
-        {% do return(None) %}
-    {% endif %}
-    {% set relation = api.Relation.create(database=db_name, schema=schema_name, identifier=table_name) %}
-    {% do run_query(create_table_as(false, relation, 'select 1 as id')) %}
-{% endmacro %}
-
-
-{% macro ensure_one_relation_in(db_name, schema_name) %}
-    {% if not execute %}
-        {% do return(None) %}
-    {% endif %}
-    {% set relation = api.Relation.create(database=db_name, schema=schema_name).without_identifier() %}
-    {% set results = list_relations_without_caching(relation) %}
-    {% set rlen = (results | length) %}
-    {% if rlen != 1 %}
-        {% do exceptions.raise_compiler_error('Incorect number of results (expected 1): ' ~ rlen) %}
-    {% endif %}
-    {% set result = results[0] %}
-    {% set columns = get_columns_in_relation(result) %}
-    {% set clen = (columns | length) %}
-    {% if clen != 1 %}
-        {% do exceptions.raise_compiler_error('Incorrect number of columns (expected 1): ' ~ clen) %}
-    {% endif %}
-{% endmacro %}
diff --git a/tests/integration/bigquery_test/models/clustered_model.sql b/tests/integration/bigquery_test/models/clustered_model.sql
deleted file mode 100644
index 1e0987ccc..000000000
--- a/tests/integration/bigquery_test/models/clustered_model.sql
+++ /dev/null
@@ -1,10 +0,0 @@
-
-{{
-	config(
-		materialized = "table",
-		partition_by = {"field": "updated_at", "data_type": "date"},
-		cluster_by = "dupe",
-	)
-}}
-
-select * from {{ ref('view_model') }}
diff --git a/tests/integration/bigquery_test/models/fUnKyCaSe.sql b/tests/integration/bigquery_test/models/fUnKyCaSe.sql
deleted file mode 100644
index 43258a714..000000000
--- a/tests/integration/bigquery_test/models/fUnKyCaSe.sql
+++ /dev/null
@@ -1 +0,0 @@
-select 1 as id
diff --git a/tests/integration/bigquery_test/models/labeled_model.sql b/tests/integration/bigquery_test/models/labeled_model.sql
deleted file mode 100644
index b8847e819..000000000
--- a/tests/integration/bigquery_test/models/labeled_model.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-{{
-	config(
-		materialized = "table",
-		labels = {'town': 'fish', 'analytics': 'yes'}
-	)
-}}
-
-select * from {{ ref('view_model') }}
diff --git a/tests/integration/bigquery_test/models/multi_clustered_model.sql b/tests/integration/bigquery_test/models/multi_clustered_model.sql
deleted file mode 100644
index c2093d6d6..000000000
--- a/tests/integration/bigquery_test/models/multi_clustered_model.sql
+++ /dev/null
@@ -1,10 +0,0 @@
-
-{{
-	config(
-		materialized = "table",
-		partition_by = {"field": "updated_at", "data_type": "date"},
-		cluster_by = ["dupe","id"],
-	)
-}}
-
-select * from {{ ref('view_model') }}
diff --git a/tests/integration/bigquery_test/models/partitioned_model.sql b/tests/integration/bigquery_test/models/partitioned_model.sql
deleted file mode 100644
index 0f30a2185..000000000
--- a/tests/integration/bigquery_test/models/partitioned_model.sql
+++ /dev/null
@@ -1,9 +0,0 @@
-
-{{
-	config(
-		materialized = "table",
-		partition_by = {'field': 'updated_at', 'data_type': 'date'},
-	)
-}}
-
-select * from {{ ref('view_model') }}
diff --git a/tests/integration/bigquery_test/models/schema.yml b/tests/integration/bigquery_test/models/schema.yml
deleted file mode 100644
index 68625ffc6..000000000
--- a/tests/integration/bigquery_test/models/schema.yml
+++ /dev/null
@@ -1,60 +0,0 @@
-version: 2
-models:
-- name: view_model
-  description: |
-    View model description "with double quotes"
-    and with 'single  quotes' as welll as other;
-    '''abc123'''
-    reserved -- characters
-    --
-    /* comment */
-  columns:
-  - name: dupe
-    tests:
-    - unique
-  - name: id
-    tests:
-    - not_null
-    - unique
-  - name: updated_at
-    tests:
-    - not_null
-  tests:
-  - was_materialized:
-      name: view_model
-      type: view
-- name: table_model
-  description: |
-    View model description "with double quotes"
-    and with 'single  quotes' as welll as other;
-    '''abc123'''
-    reserved -- characters
-    --
-    /* comment */
-  columns:
-  - name: id
-    tests:
-    - not_null
-  tests:
-  - was_materialized:
-      name: table_model
-      type: table
-- name: fUnKyCaSe
-  columns:
-    - name: id
-      tests:
-        - not_null
-        - unique
-  tests:
-    - was_materialized:
-        name: fUnKyCaSe
-        type: view
-
-
-sources:
-  - name: raw
-    project: "{{ target.database }}"
-    dataset: "{{ target.schema }}"
-    tables:
-      - name: seed
-        identifier: data_seed
diff --git a/tests/integration/bigquery_test/models/sql_header_model.sql b/tests/integration/bigquery_test/models/sql_header_model.sql
deleted file mode 100644
index e49d82c4b..000000000
--- a/tests/integration/bigquery_test/models/sql_header_model.sql
+++ /dev/null
@@ -1,14 +0,0 @@
-{{ config(materialized="table") }}
-
-{# This will fail if it is not extracted correctly #}
-{% call set_sql_header(config) %}
-  	CREATE TEMPORARY FUNCTION a_to_b(str STRING)
-	RETURNS STRING AS (
-	  CASE
-	  WHEN LOWER(str) = 'a' THEN 'b'
-	  ELSE str
-	  END
-	);
-{% endcall %}
-
-select a_to_b(dupe) as dupe from {{ ref('view_model') }}
diff --git a/tests/integration/bigquery_test/models/sql_header_model_incr.sql b/tests/integration/bigquery_test/models/sql_header_model_incr.sql
deleted file mode 100644
index 2b9597686..000000000
--- a/tests/integration/bigquery_test/models/sql_header_model_incr.sql
+++ /dev/null
@@ -1,17 +0,0 @@
-
-{{ config(materialized="incremental") }}
-
-{# This will fail if it is not extracted correctly #}
-{% call set_sql_header(config) %}
-    DECLARE int_var INT64 DEFAULT 42;
-
-  	CREATE TEMPORARY FUNCTION a_to_b(str STRING)
-	RETURNS STRING AS (
-	  CASE
-	  WHEN LOWER(str) = 'a' THEN 'b'
-	  ELSE str
-	  END
-	);
-{% endcall %}
-
-select a_to_b(dupe) as dupe from {{ ref('view_model') }}
diff --git a/tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite.sql b/tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite.sql
deleted file mode 100644
index 0cafb8489..000000000
--- a/tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite.sql
+++ /dev/null
@@ -1,33 +0,0 @@
-
-{#
-    Ensure that the insert overwrite incremental strategy
-    works correctly when a UDF is used in a sql_header. The
-    failure mode here is that dbt might inject the UDF header
-    twice: once for the `create table` and then again for the
-    merge statement.
-#}
-
-{{ config(
-    materialized="incremental",
-    incremental_strategy='insert_overwrite',
-    partition_by={"field": "dt", "data_type": "date"}
-) }}
-
-{# This will fail if it is not extracted correctly #}
-{% call set_sql_header(config) %}
-    DECLARE int_var INT64 DEFAULT 42;
-
-  	CREATE TEMPORARY FUNCTION a_to_b(str STRING)
-	RETURNS STRING AS (
-	  CASE
-	  WHEN LOWER(str) = 'a' THEN 'b'
-	  ELSE str
-	  END
-	);
-{% endcall %}
-
-select
-    current_date() as dt,
-    a_to_b(dupe) as dupe
-
-from {{ ref('view_model') }}
diff --git a/tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite_static.sql b/tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite_static.sql
deleted file mode 100644
index 4d760a0fd..000000000
--- a/tests/integration/bigquery_test/models/sql_header_model_incr_insert_overwrite_static.sql
+++ /dev/null
@@ -1,32 +0,0 @@
-
-{#
-    Ensure that the insert overwrite incremental strategy
-    works correctly when a UDF is used in a sql_header. The
-    failure mode here is that dbt might inject the UDF header
-    twice: once for the `create table` and then again for the
-    merge statement.
-#}
-
-{{ config(
-    materialized="incremental",
-    incremental_strategy='insert_overwrite',
-    partition_by={"field": "dt", "data_type": "date"},
-    partitions=["'2020-01-1'"]
-) }}
-
-{# This will fail if it is not extracted correctly #}
-{% call set_sql_header(config) %}
-  	CREATE TEMPORARY FUNCTION a_to_b(str STRING)
-	RETURNS STRING AS (
-	  CASE
-	  WHEN LOWER(str) = 'a' THEN 'b'
-	  ELSE str
-	  END
-	);
-{% endcall %}
-
-select
-    cast('2020-01-01' as date) as dt,
-    a_to_b(dupe) as dupe
-
-from {{ ref('view_model') }}
diff --git a/tests/integration/bigquery_test/models/table_model.sql b/tests/integration/bigquery_test/models/table_model.sql
deleted file mode 100644
index c748eb687..000000000
--- a/tests/integration/bigquery_test/models/table_model.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-{{
-  config(
-    materialized = "table",
-    persist_docs={ "relation": true, "columns": true, "schema": true }
-  )
-}}
-
-select * from {{ ref('view_model') }}
diff --git a/tests/integration/bigquery_test/models/view_model.sql b/tests/integration/bigquery_test/models/view_model.sql
deleted file mode 100644
index bacdaf874..000000000
--- a/tests/integration/bigquery_test/models/view_model.sql
+++ /dev/null
@@ -1,14 +0,0 @@
-{{
-  config(
-    materialized = "view",
-    persist_docs={ "relation": true, "columns": true, "schema": true }
-  )
-}}
-
-
-select
-    id,
-    current_date as updated_at,
-    dupe
-
-from {{ source('raw', 'seed') }}
diff --git a/tests/integration/bigquery_test/seeds/data_seed.csv b/tests/integration/bigquery_test/seeds/data_seed.csv
deleted file mode 100644
index afd0a31ef..000000000
--- a/tests/integration/bigquery_test/seeds/data_seed.csv
+++ /dev/null
@@ -1,5 +0,0 @@
-id,dupe
-1,a
-2,a
-3,a
-4,a
diff --git a/tests/integration/bigquery_test/seeds/incremental_overwrite_date_expected.csv b/tests/integration/bigquery_test/seeds/incremental_overwrite_date_expected.csv
deleted file mode 100644
index 7454b880b..000000000
--- a/tests/integration/bigquery_test/seeds/incremental_overwrite_date_expected.csv
+++ /dev/null
@@ -1,5 +0,0 @@
-id,date_day
-10,2020-01-01
-20,2020-01-01
-30,2020-01-02
-40,2020-01-02
diff --git a/tests/integration/bigquery_test/seeds/incremental_overwrite_day_expected.csv b/tests/integration/bigquery_test/seeds/incremental_overwrite_day_expected.csv
deleted file mode 100644
index e9db40a62..000000000
--- a/tests/integration/bigquery_test/seeds/incremental_overwrite_day_expected.csv
+++ /dev/null
@@ -1,5 +0,0 @@
-id,date_time
-10,2020-01-01 00:00:00
-20,2020-01-01 00:00:00
-30,2020-01-02 00:00:00
-40,2020-01-02 00:00:00
diff --git a/tests/integration/bigquery_test/seeds/incremental_overwrite_range_expected.csv b/tests/integration/bigquery_test/seeds/incremental_overwrite_range_expected.csv
deleted file mode 100644
index 203a4a9d0..000000000
--- a/tests/integration/bigquery_test/seeds/incremental_overwrite_range_expected.csv
+++ /dev/null
@@ -1,5 +0,0 @@
-id,date_int
-10,20200101
-20,20200101
-30,20200102
-40,20200102
diff --git a/tests/integration/bigquery_test/seeds/incremental_overwrite_time_expected.csv b/tests/integration/bigquery_test/seeds/incremental_overwrite_time_expected.csv
deleted file mode 100644
index 50559dd51..000000000
--- a/tests/integration/bigquery_test/seeds/incremental_overwrite_time_expected.csv
+++ /dev/null
@@ -1,5 +0,0 @@
-id,date_hour
-10,2020-01-01 01:00:00
-20,2020-01-01 01:00:00
-30,2020-01-01 02:00:00
-40,2020-01-01 02:00:00
diff --git a/tests/integration/bigquery_test/seeds/merge_expected.csv b/tests/integration/bigquery_test/seeds/merge_expected.csv
deleted file mode 100644
index 0aa05500c..000000000
--- a/tests/integration/bigquery_test/seeds/merge_expected.csv
+++ /dev/null
@@ -1,7 +0,0 @@
-id,date_time
-1,2020-01-01 00:00:00
-2,2020-01-01 00:00:00
-3,2020-01-01 00:00:00
-4,2020-01-02 00:00:00
-5,2020-01-02 00:00:00
-6,2020-01-02 00:00:00
\ No newline at end of file
diff --git a/tests/integration/bigquery_test/test_incremental_strategies.py b/tests/integration/bigquery_test/test_incremental_strategies.py
deleted file mode 100644
index 920460eaa..000000000
--- a/tests/integration/bigquery_test/test_incremental_strategies.py
+++ /dev/null
@@ -1,31 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, FakeArgs, use_profile
-
-class TestBigQueryScripting(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "incremental-strategy-models"
-
-    @use_profile('bigquery')
-    def test__bigquery_assert_incrementals(self):
-        results = self.run_dbt()
-        self.assertEqual(len(results), 8)
-
-        results = self.run_dbt()
-        self.assertEqual(len(results), 8)
-
-        results = self.run_dbt(['seed'])
-
-        self.assertTablesEqual('incremental_merge_range', 'merge_expected')
-        self.assertTablesEqual('incremental_merge_time', 'merge_expected')
-        self.assertTablesEqual('incremental_overwrite_time', 'incremental_overwrite_time_expected')
-        self.assertTablesEqual('incremental_overwrite_date', 'incremental_overwrite_date_expected')
-        self.assertTablesEqual('incremental_overwrite_partitions', 'incremental_overwrite_date_expected')
-        self.assertTablesEqual('incremental_overwrite_day', 'incremental_overwrite_day_expected')
-        self.assertTablesEqual('incremental_overwrite_range', 'incremental_overwrite_range_expected')
-        self.assertTablesEqual('incremental_overwrite_day_with_copy_partitions', 'incremental_overwrite_day_expected')
-        
\ No newline at end of file
diff --git a/tests/integration/bigquery_test/test_simple_bigquery_view.py b/tests/integration/bigquery_test/test_simple_bigquery_view.py
deleted file mode 100644
index 0418dc386..000000000
--- a/tests/integration/bigquery_test/test_simple_bigquery_view.py
+++ /dev/null
@@ -1,71 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-import random
-import time
-
-
-class TestBaseBigQueryRun(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "bigquery_test"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds'],
-            'macro-paths': ['macros'],
-            'seeds': {
-                'quote_columns': False,
-            },
-        }
-
-    @property
-    def profile_config(self):
-        return self.bigquery_profile()
-
-    def assert_nondupes_pass(self):
-        # The 'dupe' model should fail, but all others should pass
-        test_results = self.run_dbt(['test'], expect_pass=False)
-
-        for result in test_results:
-            if 'dupe' in result.node.name:
-                self.assertEqual(result.status, 'fail')
-                self.assertFalse(result.skipped)
-                self.assertTrue(result.failures > 0)
-
-            # assert that actual tests pass
-            else:
-                self.assertEqual(result.status, 'pass')
-                self.assertFalse(result.skipped)
-                self.assertEqual(result.failures, 0)
-
-
-class TestSimpleBigQueryRun(TestBaseBigQueryRun):
-
-    @use_profile('bigquery')
-    def test__bigquery_simple_run(self):
-        # make sure seed works twice. Full-refresh is a no-op
-        self.run_dbt(['seed'])
-        self.run_dbt(['seed', '--full-refresh'])
-        results = self.run_dbt()
-        # Bump expected number of results when adding new model
-        self.assertEqual(len(results), 11)
-        self.assert_nondupes_pass()
-
-
-class TestUnderscoreBigQueryRun(TestBaseBigQueryRun):
-    prefix = "_test{}{:04}".format(int(time.time()), random.randint(0, 9999))
-
-    @use_profile('bigquery')
-    def test_bigquery_run_twice(self):
-        self.run_dbt(['seed'])
-        results = self.run_dbt()
-        self.assertEqual(len(results), 11)
-        results = self.run_dbt()
-        self.assertEqual(len(results), 11)
-        self.assert_nondupes_pass()

From 3817c07142f28375e30bccdfc8f2810587d261b4 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Wed, 15 Feb 2023 18:31:19 -0800
Subject: [PATCH 557/860] Python Test Clean Up (#540)

* specify 1.1 in batch runtime config

* add changie

* fix quoting

* only skip broken tests and make tests work with Oauth

* Add ToDo note
---
 tests/conftest.py                             |  3 +++
 tests/functional/adapter/test_python_model.py | 19 ++++++++++++++-----
 tox.ini                                       |  2 +-
 3 files changed, 18 insertions(+), 6 deletions(-)

diff --git a/tests/conftest.py b/tests/conftest.py
index 6ca033f8d..5c30f49fd 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -30,6 +30,9 @@ def oauth_target():
         'method': 'oauth',
         'threads': 1,
         'job_retries': 2,
+        'dataproc_region': os.getenv("DATAPROC_REGION"),
+        'dataproc_cluster_name': os.getenv("DATAPROC_CLUSTER_NAME"),
+        'gcs_bucket': os.getenv("GCS_BUCKET")
     }
 
 
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
index 03ad871e2..a4b4d90c7 100644
--- a/tests/functional/adapter/test_python_model.py
+++ b/tests/functional/adapter/test_python_model.py
@@ -3,13 +3,17 @@
 from dbt.tests.util import run_dbt, write_file
 import dbt.tests.adapter.python_model.test_python_model as dbt_tests
 
-@pytest.skip("cluster unstable", allow_module_level=True)
-class TestPythonIncrementalMatsDataproc(dbt_tests.BasePythonIncrementalTests):
+# ToDo: Fix and schedule these tests:
+# https://github.com/dbt-labs/dbt-bigquery/issues/306
+class TestPythonModelDataproc(dbt_tests.BasePythonModelTests):
     pass
 
-class TestPythonModelDataproc(dbt_tests.BasePythonModelTests):
+
+@pytest.mark.skip(reason="Currently Broken")
+class TestPythonIncrementalMatsDataproc(dbt_tests.BasePythonIncrementalTests):
     pass
 
+
 models__simple_python_model = """
 import pandas
 
@@ -20,6 +24,7 @@ def model(dbt, spark):
     data = [[1,2]] * 10
     return spark.createDataFrame(data, schema=['test', 'test2'])
 """
+
 models__simple_python_model_v2 = """
 import pandas
 
@@ -31,13 +36,17 @@ def model(dbt, spark):
     return spark.createDataFrame(data, schema=['test1', 'test3'])
 """
 
+
+@pytest.mark.skip(reason="Currently Broken")
 class TestChangingSchemaDataproc:
+
     @pytest.fixture(scope="class")
     def models(self):
         return {
             "simple_python_model.py": models__simple_python_model
-            }
-    def test_changing_schema(self,project, logs_dir):
+        }
+
+    def test_changing_schema(self, project, logs_dir):
         run_dbt(["run"])
         write_file(models__simple_python_model_v2, project.project_root + '/models', "simple_python_model.py")
         run_dbt(["run"])
diff --git a/tox.ini b/tox.ini
index 2a28f0d57..1721428ee 100644
--- a/tox.ini
+++ b/tox.ini
@@ -23,7 +23,7 @@ passenv =
     DATAPROC_*
     GCS_BUCKET
 commands =
-  bigquery: {envpython} -m pytest {posargs} -vv tests/functional --profile service_account
+  bigquery: {envpython} -m pytest {posargs} -vv tests/functional -k "not TestPython" --profile service_account
 deps =
   -rdev-requirements.txt
   -e.

From f26fadd46e1bb66b490207f1e65a16b51ec45635 Mon Sep 17 00:00:00 2001
From: Victoria Perez Mola <4315804+Victoriapm@users.noreply.github.com>
Date: Fri, 17 Feb 2023 12:28:51 +0100
Subject: [PATCH 558/860] dbt Constraints / model contracts (#426)

* Create bigquery adapter macro for constraints

* fix order

* update checks validation and constraints list

* add test

* add changie

* update issue number

* fix trailing space

* fix test name

* update BQ data types

* Update test_bigquery_constraints.py

* Update test_bigquery_constraints.py

* fix quotes

Co-authored-by: Sung Won Chung <sungwonchung3@gmail.com>

* Update test_bigquery_constraints.py

* fix constratints config

* fix config

* Update test_bigquery_constraints.py

* update constraints config

* Switch to new functional tests

* Fix failing tests

* Small cleanup

* Reset to dbt-core main

* Resolve PR comment

* PR feedback

---------

Co-authored-by: Sung Won Chung <sungwonchung3@gmail.com>
Co-authored-by: Jeremy Cohen <jeremy@dbtlabs.com>
Co-authored-by: Michelle Ark <michelle.ark@dbtlabs.com>
---
 .../unreleased/Features-20221220-193731.yaml  |  7 +++
 dbt/include/bigquery/macros/adapters.sql      |  5 ++
 .../macros/utils/get_columns_spec_ddl.sql     | 16 +++++
 dev-requirements.txt                          |  1 -
 tests/functional/adapter/test_constraints.py  | 59 +++++++++++++++++++
 5 files changed, 87 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Features-20221220-193731.yaml
 create mode 100644 dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
 create mode 100644 tests/functional/adapter/test_constraints.py

diff --git a/.changes/unreleased/Features-20221220-193731.yaml b/.changes/unreleased/Features-20221220-193731.yaml
new file mode 100644
index 000000000..dbd16ae0f
--- /dev/null
+++ b/.changes/unreleased/Features-20221220-193731.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: 'dbt-constraints support for BigQuery as per dbt-core issue #1358'
+time: 2022-12-20T19:37:31.982821+01:00
+custom:
+  Author: victoriapm
+  Issue: "444"
+  PR: "426"
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index 07cf3c3e5..d80adc70c 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -52,8 +52,13 @@
     {{ sql_header if sql_header is not none }}
 
     create or replace table {{ relation }}
+      {% if config.get('constraints_enabled', False) %}
+        {{ get_assert_columns_equivalent(sql) }}
+        {{ get_columns_spec_ddl() }}
+      {% endif %}
     {{ partition_by(partition_config) }}
     {{ cluster_by(raw_cluster_by) }}
+
     {{ bigquery_table_options(config, model, temporary) }}
     as (
       {{ compiled_code }}
diff --git a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
new file mode 100644
index 000000000..4f2720b7e
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
@@ -0,0 +1,16 @@
+{% macro bigquery__get_columns_spec_ddl() %}
+  {# loop through user_provided_columns to create DDL with data types and constraints #}
+    {%- set ns = namespace(at_least_one_check=False) -%}
+    {%- set user_provided_columns = model['columns'] -%}
+    (
+    {% for i in user_provided_columns -%}
+      {%- set col = user_provided_columns[i] -%}
+      {% set constraints = col['constraints'] -%}
+      {%- set ns.at_least_one_check = ns.at_least_one_check or col['constraints_check'] %}
+      {{ col['name'] }} {{ col['data_type'] }} {% for x in constraints %} {{ x or "" }} {% endfor %} {{ "," if not loop.last }}
+    {%- endfor %}
+  )
+  {%- if ns.at_least_one_check -%}
+      {{exceptions.warn("We noticed you have `constraints_check` configs, these are NOT compatible with BigQuery and will be ignored")}}
+  {%- endif %}
+{% endmacro %}
diff --git a/dev-requirements.txt b/dev-requirements.txt
index de146ae27..f489a04b3 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -2,7 +2,6 @@
 # TODO: how to automate switching from develop to version branches?
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
-
 black~=22.8.0
 bumpversion~=0.6.0
 flake8
diff --git a/tests/functional/adapter/test_constraints.py b/tests/functional/adapter/test_constraints.py
new file mode 100644
index 000000000..b9a271b8c
--- /dev/null
+++ b/tests/functional/adapter/test_constraints.py
@@ -0,0 +1,59 @@
+import pytest
+from dbt.tests.util import relation_from_name
+from dbt.tests.adapter.constraints.test_constraints import (
+    BaseConstraintsColumnsEqual,
+    BaseConstraintsRuntimeEnforcement
+)
+from dbt.tests.adapter.constraints.fixtures import (
+    my_model_sql,
+    my_model_wrong_order_sql,
+    my_model_wrong_name_sql,
+    model_schema_yml,
+)
+
+_expected_sql_bigquery = """
+create or replace table {0} (
+    id integer  not null    ,
+    color string  ,
+    date_day date
+)
+OPTIONS()
+as (
+    select
+        1 as id,
+        'blue' as color,
+        cast('2019-01-01' as date) as date_day
+);
+"""
+
+# Different on BigQuery:
+# - does not support a data type named 'text' (TODO handle this via type translation/aliasing!)
+# - raises an explicit error, if you try to set a primary key constraint, because it's not enforced
+constraints_yml = model_schema_yml.replace("text", "string").replace("primary key", "")
+
+class TestBigQueryConstraintsColumnsEqual(BaseConstraintsColumnsEqual):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model_wrong_order.sql": my_model_wrong_order_sql,
+            "my_model_wrong_name.sql": my_model_wrong_name_sql,
+            "constraints_schema.yml": constraints_yml,
+        }
+
+
+class TestBigQueryConstraintsRuntimeEnforcement(BaseConstraintsRuntimeEnforcement):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model.sql": my_model_sql,
+            "constraints_schema.yml": constraints_yml,
+        }
+    
+    @pytest.fixture(scope="class")
+    def expected_sql(self, project):
+        relation = relation_from_name(project.adapter, "my_model")
+        return _expected_sql_bigquery.format(relation)
+
+    @pytest.fixture(scope="class")
+    def expected_error_messages(self):
+        return ["Required field id cannot be null"]

From 4d86b6755e6f81a97ddf4a07c1c3edce948803ab Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Mon, 20 Feb 2023 11:35:21 -0800
Subject: [PATCH 559/860] Convert hook integration tests to functional (#525)

* delete hook integration tests

* fix missing "host" in profile

* fix test fixtures

* add test_run_hooks

* update test_model_hooks.py
---
 .../adapter/hooks/data/seed_model.sql}        |  11 +-
 .../adapter/hooks/data/seed_run.sql}          |   8 +-
 .../adapter/hooks/test_model_hooks.py         |  32 ++++
 .../adapter/hooks/test_run_hooks.py           |  38 +++++
 .../hook_tests/macros/before-and-after-bq.sql |  30 ----
 tests/integration/hook_tests/macros/hook.sql  |   3 -
 tests/integration/hook_tests/models/hooks.sql |   2 -
 .../hook_tests/seed-models-bq/schema.yml      |   7 -
 .../hook_tests/seeds/example_seed.csv         |   4 -
 .../hook_tests/test_model_hooks_bq.py         | 150 ------------------
 .../hook_tests/test_run_hooks_bq.py           | 102 ------------
 11 files changed, 77 insertions(+), 310 deletions(-)
 rename tests/{integration/hook_tests/seed_model_bigquery.sql => functional/adapter/hooks/data/seed_model.sql} (62%)
 rename tests/{integration/hook_tests/seed_run_bigquery.sql => functional/adapter/hooks/data/seed_run.sql} (81%)
 create mode 100644 tests/functional/adapter/hooks/test_model_hooks.py
 create mode 100644 tests/functional/adapter/hooks/test_run_hooks.py
 delete mode 100644 tests/integration/hook_tests/macros/before-and-after-bq.sql
 delete mode 100644 tests/integration/hook_tests/macros/hook.sql
 delete mode 100644 tests/integration/hook_tests/models/hooks.sql
 delete mode 100644 tests/integration/hook_tests/seed-models-bq/schema.yml
 delete mode 100644 tests/integration/hook_tests/seeds/example_seed.csv
 delete mode 100644 tests/integration/hook_tests/test_model_hooks_bq.py
 delete mode 100644 tests/integration/hook_tests/test_run_hooks_bq.py

diff --git a/tests/integration/hook_tests/seed_model_bigquery.sql b/tests/functional/adapter/hooks/data/seed_model.sql
similarity index 62%
rename from tests/integration/hook_tests/seed_model_bigquery.sql
rename to tests/functional/adapter/hooks/data/seed_model.sql
index 7093a47e8..9b5130953 100644
--- a/tests/integration/hook_tests/seed_model_bigquery.sql
+++ b/tests/functional/adapter/hooks/data/seed_model.sql
@@ -1,9 +1,7 @@
+drop table if exists `{schema}.on_model_hook`;
 
-drop table if exists {schema}.on_model_hook;
-
-create table {schema}.on_model_hook (
-    state            STRING, -- start|end
-
+create table `{schema}.on_model_hook` (
+    test_state       STRING, -- start|end
     target_dbname    STRING,
     target_host      STRING,
     target_name      STRING,
@@ -11,8 +9,7 @@ create table {schema}.on_model_hook (
     target_type      STRING,
     target_user      STRING,
     target_pass      STRING,
-    target_threads   INT64,
-
+    target_threads   INTEGER,
     run_started_at   STRING,
     invocation_id    STRING
 );
diff --git a/tests/integration/hook_tests/seed_run_bigquery.sql b/tests/functional/adapter/hooks/data/seed_run.sql
similarity index 81%
rename from tests/integration/hook_tests/seed_run_bigquery.sql
rename to tests/functional/adapter/hooks/data/seed_run.sql
index d9d5212ef..4699b0132 100644
--- a/tests/integration/hook_tests/seed_run_bigquery.sql
+++ b/tests/functional/adapter/hooks/data/seed_run.sql
@@ -2,8 +2,7 @@
 drop table if exists {schema}.on_run_hook;
 
 create table {schema}.on_run_hook (
-    state            STRING, -- start|end
-
+    test_state       STRING, -- start|end
     target_dbname    STRING,
     target_host      STRING,
     target_name      STRING,
@@ -11,8 +10,7 @@ create table {schema}.on_run_hook (
     target_type      STRING,
     target_user      STRING,
     target_pass      STRING,
-    target_threads   INT64,
-
+    target_threads   INTEGER,
     run_started_at   STRING,
     invocation_id    STRING
-);
\ No newline at end of file
+);
diff --git a/tests/functional/adapter/hooks/test_model_hooks.py b/tests/functional/adapter/hooks/test_model_hooks.py
new file mode 100644
index 000000000..ea722dbea
--- /dev/null
+++ b/tests/functional/adapter/hooks/test_model_hooks.py
@@ -0,0 +1,32 @@
+from dbt.tests.adapter.hooks import test_model_hooks as core_base
+import pytest
+
+class TestBigQueryPrePostModelHooks(core_base.TestPrePostModelHooks):
+    def check_hooks(self, state, project, host, count=1):
+        self.get_ctx_vars(state, count=count, project=project)
+
+
+class TestBigQueryPrePostModelHooksUnderscores(core_base.TestPrePostModelHooksUnderscores):
+    def check_hooks(self, state, project, host, count=1):
+        self.get_ctx_vars(state, count=count, project=project)
+
+class TestBigQueryHookRefs(core_base.TestHookRefs):
+    def check_hooks(self, state, project, host, count=1):
+        self.get_ctx_vars(state, count=count, project=project)
+
+class TestBigQueryPrePostModelHooksOnSeeds(core_base.TestPrePostModelHooksOnSeeds):
+    def check_hooks(self, state, project, host, count=1):
+        self.get_ctx_vars(state, count=count, project=project)
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            "seed-paths": ["seeds"],
+            "models": {},
+            "seeds": {
+                "+post-hook": [
+                    "alter table {{ this }} add column new_col int",
+                    "update {{ this }} set new_col = 1 where 1=1",
+                ],
+                "quote_columns": True,
+            },
+        }
diff --git a/tests/functional/adapter/hooks/test_run_hooks.py b/tests/functional/adapter/hooks/test_run_hooks.py
new file mode 100644
index 000000000..18a1f2bca
--- /dev/null
+++ b/tests/functional/adapter/hooks/test_run_hooks.py
@@ -0,0 +1,38 @@
+import pytest
+from dbt.tests.adapter.hooks import test_run_hooks as core_base
+
+
+class TestPrePostRunHooksBigQuery(core_base.TestPrePostRunHooks):
+    def check_hooks(self, state, project, host):
+        self.get_ctx_vars(state, project)
+
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            # The create and drop table statements here validate that these hooks run
+            # in the same order that they are defined. Drop before create is an error.
+            # Also check that the table does not exist below.
+            "on-run-start": [
+                "{{ custom_run_hook('start', target, run_started_at, invocation_id) }}",
+                "create table {{ target.schema }}.start_hook_order_test ( id int )",
+                "drop table {{ target.schema }}.start_hook_order_test",
+                "{{ log(env_var('TERM_TEST'), info=True) }}",
+            ],
+            "on-run-end": [
+                "{{ custom_run_hook('end', target, run_started_at, invocation_id) }}",
+                "create table {{ target.schema }}.end_hook_order_test ( id int )",
+                "drop table {{ target.schema }}.end_hook_order_test",
+                "create table {{ target.schema }}.schemas ( schema string )",
+                "insert into {{ target.schema }}.schemas (schema) values {% for schema in schemas %}( '{{ schema }}' ){% if not loop.last %},{% endif %}{% endfor %}",
+                "create table {{ target.schema }}.db_schemas ( db string, schema string )",
+                "insert into {{ target.schema }}.db_schemas (db, schema) values {% for db, schema in database_schemas %}('{{ db }}', '{{ schema }}' ){% if not loop.last %},{% endif %}{% endfor %}",
+            ],
+            "seeds": {
+                "quote_columns": False,
+            },
+        }
+
+
+class TestAfterRunHooksBigQuery(core_base.TestAfterRunHooks):
+    def check_hooks(self, state, project, host):
+        self.get_ctx_vars(state, project)
\ No newline at end of file
diff --git a/tests/integration/hook_tests/macros/before-and-after-bq.sql b/tests/integration/hook_tests/macros/before-and-after-bq.sql
deleted file mode 100644
index 030b869b7..000000000
--- a/tests/integration/hook_tests/macros/before-and-after-bq.sql
+++ /dev/null
@@ -1,30 +0,0 @@
-
-{% macro custom_run_hook_bq(state, target, run_started_at, invocation_id) %}
-
-   insert into {{ target.schema }}.on_run_hook (
-        state,
-        target_dbname,
-        target_host,
-        target_name,
-        target_schema,
-        target_type,
-        target_user,
-        target_pass,
-        target_threads,
-        run_started_at,
-        invocation_id
-   ) VALUES (
-    '{{ state }}',
-    '{{ target.database }}',
-    '', {# bigquery has no host value #}
-    '{{ target.name }}',
-    '{{ target.schema }}',
-    '{{ target.type }}',
-    '', {# bigquery has no user value #}
-    '{{ target.get("pass", "") }}',
-    {{ target.threads }},
-    '{{ run_started_at }}',
-    '{{ invocation_id }}'
-   )
-
-{% endmacro %}
diff --git a/tests/integration/hook_tests/macros/hook.sql b/tests/integration/hook_tests/macros/hook.sql
deleted file mode 100644
index 733278473..000000000
--- a/tests/integration/hook_tests/macros/hook.sql
+++ /dev/null
@@ -1,3 +0,0 @@
-{% macro hook() %}
-  select 1
-{% endmacro %}
diff --git a/tests/integration/hook_tests/models/hooks.sql b/tests/integration/hook_tests/models/hooks.sql
deleted file mode 100644
index 2cd691ea7..000000000
--- a/tests/integration/hook_tests/models/hooks.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-
-select 1 as id
diff --git a/tests/integration/hook_tests/seed-models-bq/schema.yml b/tests/integration/hook_tests/seed-models-bq/schema.yml
deleted file mode 100644
index 21bbf202f..000000000
--- a/tests/integration/hook_tests/seed-models-bq/schema.yml
+++ /dev/null
@@ -1,7 +0,0 @@
-version: 2
-seeds:
-- name: example_seed
-  columns:
-  - name: a
-    tests:
-    - not_null
diff --git a/tests/integration/hook_tests/seeds/example_seed.csv b/tests/integration/hook_tests/seeds/example_seed.csv
deleted file mode 100644
index 29e4b3171..000000000
--- a/tests/integration/hook_tests/seeds/example_seed.csv
+++ /dev/null
@@ -1,4 +0,0 @@
-a,b,c
-1,2,3
-4,5,6
-7,8,9
diff --git a/tests/integration/hook_tests/test_model_hooks_bq.py b/tests/integration/hook_tests/test_model_hooks_bq.py
deleted file mode 100644
index 8b5c8d5bd..000000000
--- a/tests/integration/hook_tests/test_model_hooks_bq.py
+++ /dev/null
@@ -1,150 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-MODEL_PRE_HOOK = """
-   insert into {{this.schema}}.on_model_hook (
-        state,
-        target_name,
-        target_schema,
-        target_type,
-        target_threads,
-        run_started_at,
-        invocation_id
-   ) VALUES (
-    'start',
-    '{{ target.name }}',
-    '{{ target.schema }}',
-    '{{ target.type }}',
-    {{ target.threads }},
-    '{{ run_started_at }}',
-    '{{ invocation_id }}'
-   )
-"""
-
-
-MODEL_POST_HOOK = """
-   insert into {{this.schema}}.on_model_hook (
-        state,
-        target_name,
-        target_schema,
-        target_type,
-        target_threads,
-        run_started_at,
-        invocation_id
-   ) VALUES (
-    'end',
-    '{{ target.name }}',
-    '{{ target.schema }}',
-    '{{ target.type }}',
-    {{ target.threads }},
-    '{{ run_started_at }}',
-    '{{ invocation_id }}'
-   )
-"""
-
-class TestBigqueryPrePostModelHooks(DBTIntegrationTest):
-    def setUp(self):
-        DBTIntegrationTest.setUp(self)
-        self.run_sql_file("seed_model_bigquery.sql")
-
-        self.fields = [
-            'state',
-            'target_name',
-            'target_schema',
-            'target_threads',
-            'target_type',
-            'run_started_at',
-            'invocation_id'
-        ]
-
-    @property
-    def schema(self):
-        return "model_hooks"
-
-    @property
-    def profile_config(self):
-        profile = self.bigquery_profile()
-        profile['test']['outputs']['default2']['threads'] = 3
-        return profile
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'macro-paths': ['macros'],
-            'models': {
-                'test': {
-                    'pre-hook': [MODEL_PRE_HOOK],
-                    'post-hook':[MODEL_POST_HOOK],
-                }
-            }
-        }
-
-    @property
-    def models(self):
-        return "models"
-
-    def get_ctx_vars(self, state):
-        field_list = ", ".join(self.fields)
-        query = "select {field_list} from `{schema}.on_model_hook` where state = '{state}'".format(field_list=field_list, schema=self.unique_schema(), state=state)
-
-        vals = self.run_sql(query, fetch='all')
-        self.assertFalse(len(vals) == 0, 'nothing inserted into hooks table')
-        self.assertFalse(len(vals) > 1, 'too many rows in hooks table')
-        ctx = dict(zip(self.fields, vals[0]))
-
-        return ctx
-
-    def check_hooks(self, state):
-        ctx = self.get_ctx_vars(state)
-
-        self.assertEqual(ctx['state'], state)
-        self.assertEqual(ctx['target_name'], 'default2')
-        self.assertEqual(ctx['target_schema'], self.unique_schema())
-        self.assertEqual(ctx['target_threads'], 3)
-        self.assertEqual(ctx['target_type'], 'bigquery')
-        self.assertTrue(ctx['run_started_at'] is not None and len(ctx['run_started_at']) > 0, 'run_started_at was not set')
-        self.assertTrue(ctx['invocation_id'] is not None and len(ctx['invocation_id']) > 0, 'invocation_id was not set')
-
-    @use_profile('bigquery')
-    def test_pre_and_post_model_hooks_bigquery(self):
-        self.run_dbt(['run'])
-
-        self.check_hooks('start')
-        self.check_hooks('end')
-
-
-class TestBigqueryPrePostModelHooksOnSeeds(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "model_hooks_014"
-
-    @property
-    def models(self):
-        return "seed-models-bq"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds'],
-            'models': {},
-            'seeds': {
-                'post-hook': [
-                    'insert into {{ this }} (a, b, c) VALUES (10, 11, 12)',
-                ],
-                'quote_columns': False,
-            },
-        }
-
-    @use_profile('bigquery')
-    def test_hooks_on_seeds_bigquery(self):
-        res = self.run_dbt(['seed'])
-        self.assertEqual(len(res), 1, 'Expected exactly one item')
-        res = self.run_dbt(['test'])
-        self.assertEqual(len(res), 1, 'Expected exactly one item')
-        result = self.run_sql(
-            'select a, b, c from `{schema}`.`example_seed` where a = 10',
-            fetch='all'
-        )
-        self.assertFalse(len(result) == 0, 'nothing inserted into table by hook')
-        self.assertFalse(len(result) > 1, 'too many rows in table')
diff --git a/tests/integration/hook_tests/test_run_hooks_bq.py b/tests/integration/hook_tests/test_run_hooks_bq.py
deleted file mode 100644
index ef1b7cae0..000000000
--- a/tests/integration/hook_tests/test_run_hooks_bq.py
+++ /dev/null
@@ -1,102 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-class TestBigqueryPrePostRunHooks(DBTIntegrationTest):
-
-    def setUp(self):
-        DBTIntegrationTest.setUp(self)
-        self.use_profile('bigquery')
-        self.use_default_project()
-        self.run_sql_file("seed_run_bigquery.sql")
-
-        self.fields = [
-            'state',
-            'target_name',
-            'target_schema',
-            'target_threads',
-            'target_type',
-            'run_started_at',
-            'invocation_id'
-        ]
-
-    @property
-    def schema(self):
-        return "run_hooks_014"
-
-    @property
-    def profile_config(self):
-        profile = self.bigquery_profile()
-        profile['test']['outputs']['default2']['threads'] = 3
-        return profile
-
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'macro-paths': ['macros'],
-            'seed-paths': ['seeds'],
-
-            # The create and drop table statements here validate that these hooks run
-            # in the same order that they are defined. Drop before create is an error.
-            # Also check that the table does not exist below.
-            "on-run-start": [
-                "{{ custom_run_hook_bq('start', target, run_started_at, invocation_id) }}",
-                "create table {{ target.schema }}.start_hook_order_test ( id INT64 )",
-                "drop table {{ target.schema }}.start_hook_order_test",
-            ],
-            "on-run-end": [
-                "{{ custom_run_hook_bq('end', target, run_started_at, invocation_id) }}",
-                "create table {{ target.schema }}.end_hook_order_test ( id INT64 )",
-                "drop table {{ target.schema }}.end_hook_order_test",
-            ],
-            'seeds': {
-                'quote_columns': False,
-            },
-        }
-
-    @property
-    def models(self):
-        return "models"
-
-    def get_ctx_vars(self, state):
-        field_list = ", ".join(self.fields)
-        query = "select {field_list} from `{schema}.on_run_hook` where state = '{state}'".format(field_list=field_list, schema=self.unique_schema(), state=state)
-
-        vals = self.run_sql(query, fetch='all')
-        self.assertFalse(len(vals) == 0, 'nothing inserted into on_run_hook table')
-        self.assertFalse(len(vals) > 1, 'too many rows in hooks table')
-        ctx = dict(zip(self.fields, vals[0]))
-
-        return ctx
-
-    def check_hooks(self, state):
-        ctx = self.get_ctx_vars(state)
-
-        self.assertEqual(ctx['state'], state)
-        self.assertEqual(ctx['target_name'], 'default2')
-        self.assertEqual(ctx['target_schema'], self.unique_schema())
-        self.assertEqual(ctx['target_threads'], 3)
-        self.assertEqual(ctx['target_type'], 'bigquery')
-
-        self.assertTrue(ctx['run_started_at'] is not None and len(ctx['run_started_at']) > 0, 'run_started_at was not set')
-        self.assertTrue(ctx['invocation_id'] is not None and len(ctx['invocation_id']) > 0, 'invocation_id was not set')
-
-    @use_profile('bigquery')
-    def test_bigquery_pre_and_post_run_hooks(self):
-        self.run_dbt(['run'])
-
-        self.check_hooks('start')
-        self.check_hooks('end')
-
-        self.assertTableDoesNotExist("start_hook_order_test")
-        self.assertTableDoesNotExist("end_hook_order_test")
-
-    @use_profile('bigquery')
-    def test_bigquery_pre_and_post_seed_hooks(self):
-        self.run_dbt(['seed'])
-
-        self.check_hooks('start')
-        self.check_hooks('end')
-
-        self.assertTableDoesNotExist("start_hook_order_test")
-        self.assertTableDoesNotExist("end_hook_order_test")

From 13d8ead054118b8975810fe3828af7c74e2f5144 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Tue, 21 Feb 2023 09:47:40 -0600
Subject: [PATCH 560/860] add new workflow for cutting .latest branches (#539)

* add new workflow

* rename workflow extension, update comments

* update PR body
---
 .github/workflows/cut-release-branch.yml | 42 ++++++++++++++++++++++++
 1 file changed, 42 insertions(+)
 create mode 100644 .github/workflows/cut-release-branch.yml

diff --git a/.github/workflows/cut-release-branch.yml b/.github/workflows/cut-release-branch.yml
new file mode 100644
index 000000000..827a2fff9
--- /dev/null
+++ b/.github/workflows/cut-release-branch.yml
@@ -0,0 +1,42 @@
+# **what?**
+# Calls a centralize3d workflow that will:
+# 1. Cut a new branch (generally `*.latest`)
+# 2. Also cleans up all files in `.changes/unreleased` and `.changes/previous version on
+# `main` and bumps `main` to the input version.
+
+# **why?**
+# Generally reduces the workload of engineers and reduces error.  Allow automation.
+
+# **when?**
+# This will run when called manually.
+
+name: Cut new release branch
+
+on:
+  workflow_dispatch:
+    inputs:
+      version_to_bump_main:
+        description: 'The alpha version main should bump to (ex. 1.6.0a1)'
+        required: true
+      new_branch_name:
+        description: 'The full name of the new branch (ex. 1.5.latest)'
+        required: true
+
+defaults:
+  run:
+    shell: bash
+
+permissions:
+  contents: write
+
+jobs:
+  cut_branch:
+    name: "Cut branch and clean up main for dbt-bigquery"
+    uses: dbt-labs/actions/.github/workflows/cut-release-branch.yml@main
+    with:
+      version_to_bump_main: ${{ inputs.version_to_bump_main }}
+      new_branch_name: ${{ inputs.new_branch_name }}
+      PR_title: "Cleanup main after cutting new ${{ inputs.new_branch_name }} branch"
+      PR_body: "This PR will fail CI until the dbt-core PR has been merged due to release version conflicts.  dev-requirements.txt needs to be updated to have the dbt-core dependencies point to this new branch."
+    secrets:
+      FISHTOWN_BOT_PAT: ${{ secrets.FISHTOWN_BOT_PAT }}

From 0aca25089fafaf6edc98badcafffaf2a859c17a3 Mon Sep 17 00:00:00 2001
From: Chenyu Li <chenyu.li@dbtlabs.com>
Date: Tue, 21 Feb 2023 09:29:14 -0800
Subject: [PATCH 561/860] Chenyu/remove manual retry (#423)

* remove manual retry

* add changelog
---
 .../unreleased/Under the Hood-20221209-161550.yaml |  7 +++++++
 dbt/adapters/bigquery/python_submissions.py        | 14 ++------------
 2 files changed, 9 insertions(+), 12 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20221209-161550.yaml

diff --git a/.changes/unreleased/Under the Hood-20221209-161550.yaml b/.changes/unreleased/Under the Hood-20221209-161550.yaml
new file mode 100644
index 000000000..846cda6f6
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20221209-161550.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: Remove manual retry to get python job status on cluster
+time: 2022-12-09T16:15:50.834922-08:00
+custom:
+  Author: ChenyuLInx
+  Issue: "422"
+  PR: "423"
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index f82b72d7c..76aff3fd9 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -1,5 +1,4 @@
 from typing import Dict, Union
-import time
 
 from dbt.adapters.base import PythonJobHelper
 from dbt.adapters.bigquery import BigQueryConnectionManager, BigQueryCredentials
@@ -67,14 +66,6 @@ def _get_job_client(
     def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         raise NotImplementedError("_submit_dataproc_job not implemented")
 
-    def _wait_operation(self, operation):
-        # can't use due to https://github.com/googleapis/python-api-core/issues/458
-        # response = operation.result(retry=self.retry)
-        # Temp solution to wait for the job to finish
-        start = time.time()
-        while not operation.done(retry=None) and time.time() - start < self.timeout:
-            time.sleep(OPERATION_RETRY_TIME)
-
 
 class ClusterDataprocHelper(BaseDataProcHelper):
     def _get_job_client(self) -> dataproc_v1.JobControllerClient:
@@ -105,8 +96,7 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
                 "job": job,
             }
         )
-        self._wait_operation(operation)
-        response = operation.metadata
+        response = operation.result(retry=self.retry)
         # check if job failed
         if response.status.state == 6:
             raise ValueError(response.status.details)
@@ -144,7 +134,7 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         )
         # make the request
         operation = self.job_client.create_batch(request=request)  # type: ignore
-        # this takes quite a while, waiting on GCP response to resolve
+        # this takes quite a while, waiting on GCP response to resolve(not a google-api-core issue, more likely a dataproc serverless issue)
         response = operation.result(retry=self.retry)
         return response
         # there might be useful results here that we can parse and return

From f7003d96c72456c38b93156948dbec519274b434 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Tue, 21 Feb 2023 13:00:44 -0600
Subject: [PATCH 562/860] Fix nightly release test (#545) (#547)

* fix vversion bugs and workflow steps

* point to main
# Conflicts:
#	.bumpversion.cfg

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .bumpversion.cfg                      | 12 ++++++++----
 .github/workflows/nightly-release.yml | 15 +++++++++++----
 setup.py                              |  3 ++-
 3 files changed, 21 insertions(+), 9 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index f2d5f57d4..ec4968247 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -4,12 +4,16 @@ current_version = 1.5.0a1
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
-	(((?P<prekind>a|b|rc) # optional pre-release type
-	?(?P<num>[\d]+?)) # optional pre-release version number
-	\.?(?P<nightly>[a-z0-9]+\+[a-z]+)? # optional nightly release indicator
-	)? # expected matches: `1.5.0`, `1.5.0a1`, `1.5.0a1.dev123457+nightly`, expected failures: `1`, `1.5`, `1.5.2-a1`, `text1.5.0`
+	(?P<prerelease>  # optional pre-release - ex: a1, b2, rc25
+	(?P<prekind>a|b|rc) # pre-release type
+	(?P<num>[\d]+) # pre-release version number
+	)?
+	( # optional nightly release indicator
+	\.(?P<nightly>dev[0-9]+) # ex: .dev02142023
+	)? # expected matches: `1.15.0`, `1.5.0a11`, `1.5.0a1.dev123`, `1.5.0.dev123457`, expected failures: `1`, `1.5`, `1.5.2-a1`, `text1.5.0`
 serialize =
 	{major}.{minor}.{patch}{prekind}{num}.{nightly}
+	{major}.{minor}.{patch}.{nightly}
 	{major}.{minor}.{patch}{prekind}{num}
 	{major}.{minor}.{patch}
 commit = False
diff --git a/.github/workflows/nightly-release.yml b/.github/workflows/nightly-release.yml
index b668d62ec..54c5fdc69 100644
--- a/.github/workflows/nightly-release.yml
+++ b/.github/workflows/nightly-release.yml
@@ -26,7 +26,7 @@ defaults:
     shell: bash
 
 env:
-  RELEASE_BRANCH: "main"
+  RELEASE_BRANCH: "1.4.latest"
 
 jobs:
   aggregate-release-data:
@@ -65,10 +65,17 @@ jobs:
         id: current-date
         run: echo "date=$(date +'%m%d%Y')" >> $GITHUB_OUTPUT
 
+      # Bump to the next patch because when this is a previously released patch, the changelog
+      # markdown will already exist and cause a failure in another step
+      - name: "Bump Patch Number"
+        id: bump_patch
+        run: |
+          echo "patch=$((${{ steps.semver.outputs.patch }}+1))" >> $GITHUB_OUTPUT
+
       - name: "Generate Nightly Release Version Number"
         id: nightly-release-version
         run: |
-          number="${{ steps.semver.outputs.version }}.dev${{ steps.current-date.outputs.date }}+nightly"
+          number="${{ steps.semver.outputs.major }}.${{ steps.semver.outputs.minor }}.${{ steps.bump_patch.outputs.patch }}.dev${{ steps.current-date.outputs.date }}"
           echo "number=$number" >> $GITHUB_OUTPUT
 
       - name: "Audit Nightly Release Version And Parse Into Parts"
@@ -98,12 +105,12 @@ jobs:
     uses: ./.github/workflows/release.yml
     with:
       sha: ${{ needs.aggregate-release-data.outputs.commit_sha }}
-      target_branch: ${{ needs.aggregate-release-data.outputs.release-branch }}
+      target_branch: ${{ needs.aggregate-release-data.outputs.release_branch }}
       version_number: ${{ needs.aggregate-release-data.outputs.version_number }}
       build_script_path: "scripts/build-dist.sh"
       env_setup_script_path: "scripts/env-setup.sh"
       s3_bucket_name: "core-team-artifacts"
-      package_test_command: "dbt --version"
+      package_test_command: "dbt -h"
       test_run: true
       nightly_release: true
     secrets: inherit
diff --git a/setup.py b/setup.py
index a7b278203..6da4b88bd 100644
--- a/setup.py
+++ b/setup.py
@@ -41,7 +41,8 @@ def _dbt_core_version(plugin_version: str) -> str:
     Determine the compatible version of dbt-core using this package's version
     """
     try:
-        major, minor, plugin_patch = plugin_version.split(".")
+        # *_ may indicate a dev release which won't affect the core version needed
+        major, minor, plugin_patch, *_ = plugin_version.split(".", maxsplit=3)
     except ValueError:
         raise ValueError(f"Invalid version: {plugin_version}")
 

From d3af14c4fc6c1fa39fab9267a25038ef7004b0a4 Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Wed, 22 Feb 2023 19:34:56 +0000
Subject: [PATCH 563/860] Bumping version to 1.5.0b1 and generate changelog

---
 .bumpversion.cfg                              |  3 +--
 .changes/1.5.0-b1.md                          | 21 ++++++++++++++++
 .../Features-20220826-115320.yaml             |  0
 .../Features-20221220-193731.yaml             |  0
 .../Fixes-20221213-102005.yaml                |  0
 .../Fixes-20230125-174159.yaml                |  0
 .../Fixes-20230213-203317.yaml                |  0
 .../Under the Hood-20221209-161550.yaml       |  0
 .../Under the Hood-20230130-170118.yaml       |  0
 CHANGELOG.md                                  | 24 +++++++++++++++++++
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 12 files changed, 48 insertions(+), 4 deletions(-)
 create mode 100644 .changes/1.5.0-b1.md
 rename .changes/{unreleased => 1.5.0}/Features-20220826-115320.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Features-20221220-193731.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Fixes-20221213-102005.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Fixes-20230125-174159.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Fixes-20230213-203317.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Under the Hood-20221209-161550.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Under the Hood-20230130-170118.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index ec4968247..c8ca74834 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,6 +1,5 @@
 [bumpversion]
-current_version = 1.5.0a1
-
+current_version = 1.5.0b1
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.5.0-b1.md b/.changes/1.5.0-b1.md
new file mode 100644
index 000000000..4afeff282
--- /dev/null
+++ b/.changes/1.5.0-b1.md
@@ -0,0 +1,21 @@
+## dbt-bigquery 1.5.0-b1 - February 22, 2023
+
+### Features
+
+- add option to specify excluded columns on merge ([#5260](https://github.com/dbt-labs/dbt-bigquery/issues/5260))
+- dbt-constraints support for BigQuery as per dbt-core issue #1358 ([#444](https://github.com/dbt-labs/dbt-bigquery/issues/444))
+
+### Fixes
+
+- Make BQ access_grant updates thread safe ([#266](https://github.com/dbt-labs/dbt-bigquery/issues/266))
+- Use IEC standard abbreviations (GiB, TiB, etc) ([#477](https://github.com/dbt-labs/dbt-bigquery/issues/477))
+- Pin dataproc serverless spark runtime to `1.1` ([#531](https://github.com/dbt-labs/dbt-bigquery/issues/531))
+
+### Under the Hood
+
+- Remove manual retry to get python job status on cluster ([#422](https://github.com/dbt-labs/dbt-bigquery/issues/422))
+- remove tox call to integration tests ([#459](https://github.com/dbt-labs/dbt-bigquery/issues/459))
+
+### Contributors
+- [@dave-connors-3](https://github.com/dave-connors-3) ([#5260](https://github.com/dbt-labs/dbt-bigquery/issues/5260))
+- [@victoriapm](https://github.com/victoriapm) ([#444](https://github.com/dbt-labs/dbt-bigquery/issues/444))
diff --git a/.changes/unreleased/Features-20220826-115320.yaml b/.changes/1.5.0/Features-20220826-115320.yaml
similarity index 100%
rename from .changes/unreleased/Features-20220826-115320.yaml
rename to .changes/1.5.0/Features-20220826-115320.yaml
diff --git a/.changes/unreleased/Features-20221220-193731.yaml b/.changes/1.5.0/Features-20221220-193731.yaml
similarity index 100%
rename from .changes/unreleased/Features-20221220-193731.yaml
rename to .changes/1.5.0/Features-20221220-193731.yaml
diff --git a/.changes/unreleased/Fixes-20221213-102005.yaml b/.changes/1.5.0/Fixes-20221213-102005.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20221213-102005.yaml
rename to .changes/1.5.0/Fixes-20221213-102005.yaml
diff --git a/.changes/unreleased/Fixes-20230125-174159.yaml b/.changes/1.5.0/Fixes-20230125-174159.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230125-174159.yaml
rename to .changes/1.5.0/Fixes-20230125-174159.yaml
diff --git a/.changes/unreleased/Fixes-20230213-203317.yaml b/.changes/1.5.0/Fixes-20230213-203317.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230213-203317.yaml
rename to .changes/1.5.0/Fixes-20230213-203317.yaml
diff --git a/.changes/unreleased/Under the Hood-20221209-161550.yaml b/.changes/1.5.0/Under the Hood-20221209-161550.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20221209-161550.yaml
rename to .changes/1.5.0/Under the Hood-20221209-161550.yaml
diff --git a/.changes/unreleased/Under the Hood-20230130-170118.yaml b/.changes/1.5.0/Under the Hood-20230130-170118.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20230130-170118.yaml
rename to .changes/1.5.0/Under the Hood-20230130-170118.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 56e5e7226..b922a3e12 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -4,6 +4,30 @@
 - Changes are listed under the (pre)release in which they first appear. Subsequent releases include changes from previous releases.
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
+
+## dbt-bigquery 1.5.0-b1 - February 22, 2023
+
+### Features
+
+- add option to specify excluded columns on merge ([#5260](https://github.com/dbt-labs/dbt-bigquery/issues/5260))
+- dbt-constraints support for BigQuery as per dbt-core issue #1358 ([#444](https://github.com/dbt-labs/dbt-bigquery/issues/444))
+
+### Fixes
+
+- Make BQ access_grant updates thread safe ([#266](https://github.com/dbt-labs/dbt-bigquery/issues/266))
+- Use IEC standard abbreviations (GiB, TiB, etc) ([#477](https://github.com/dbt-labs/dbt-bigquery/issues/477))
+- Pin dataproc serverless spark runtime to `1.1` ([#531](https://github.com/dbt-labs/dbt-bigquery/issues/531))
+
+### Under the Hood
+
+- Remove manual retry to get python job status on cluster ([#422](https://github.com/dbt-labs/dbt-bigquery/issues/422))
+- remove tox call to integration tests ([#459](https://github.com/dbt-labs/dbt-bigquery/issues/459))
+
+### Contributors
+- [@dave-connors-3](https://github.com/dave-connors-3) ([#5260](https://github.com/dbt-labs/dbt-bigquery/issues/5260))
+- [@victoriapm](https://github.com/victoriapm) ([#444](https://github.com/dbt-labs/dbt-bigquery/issues/444))
+
+
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 * [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 219c289b1..c3758128c 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.5.0a1"
+version = "1.5.0b1"
diff --git a/setup.py b/setup.py
index 6da4b88bd..9fa89a319 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.5.0a1"
+package_version = "1.5.0b1"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From c45e624c103ff23c9dfa2a262b29a412c9d70f93 Mon Sep 17 00:00:00 2001
From: Gerda Shank <gerda@dbtlabs.com>
Date: Thu, 23 Feb 2023 14:47:57 -0500
Subject: [PATCH 564/860] Rename constraints_enabled to contract (#554)

---
 .changes/unreleased/Under the Hood-20230223-105149.yaml | 6 ++++++
 dbt/include/bigquery/macros/adapters.sql                | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Under the Hood-20230223-105149.yaml

diff --git a/.changes/unreleased/Under the Hood-20230223-105149.yaml b/.changes/unreleased/Under the Hood-20230223-105149.yaml
new file mode 100644
index 000000000..d06d2e60e
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20230223-105149.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Rename constraints_enabled to contract
+time: 2023-02-23T10:51:49.737457-05:00
+custom:
+  Author: gshank
+  Issue: "548"
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index d80adc70c..4e0dfc904 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -52,7 +52,7 @@
     {{ sql_header if sql_header is not none }}
 
     create or replace table {{ relation }}
-      {% if config.get('constraints_enabled', False) %}
+      {% if config.get('contract', False) %}
         {{ get_assert_columns_equivalent(sql) }}
         {{ get_columns_spec_ddl() }}
       {% endif %}

From a4abd145197702db2e48ab6544664b4eebcfc25f Mon Sep 17 00:00:00 2001
From: Nathaniel May <nathaniel.may@fishtownanalytics.com>
Date: Thu, 23 Feb 2023 17:47:39 -0500
Subject: [PATCH 565/860] add codeowners file (#558)

---
 .github/CODEOWNERS | 3 +++
 1 file changed, 3 insertions(+)
 create mode 100644 .github/CODEOWNERS

diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS
new file mode 100644
index 000000000..f6283d123
--- /dev/null
+++ b/.github/CODEOWNERS
@@ -0,0 +1,3 @@
+# This codeowners file is used to ensure all PRs require reviews from the adapters team
+
+* @dbt-labs/core-adapters

From 6b17008efec3ccc53449950f4b7891ac694baba9 Mon Sep 17 00:00:00 2001
From: Nathaniel May <nathaniel.may@fishtownanalytics.com>
Date: Mon, 27 Feb 2023 12:25:19 -0500
Subject: [PATCH 566/860] mirror issues to ADAP jira project (#541)

---
 .github/workflows/jira-creation.yml   | 4 +++-
 .github/workflows/jira-label.yml      | 4 +++-
 .github/workflows/jira-transition.yml | 7 ++++++-
 3 files changed, 12 insertions(+), 3 deletions(-)

diff --git a/.github/workflows/jira-creation.yml b/.github/workflows/jira-creation.yml
index b4016befc..2611a8bdd 100644
--- a/.github/workflows/jira-creation.yml
+++ b/.github/workflows/jira-creation.yml
@@ -19,7 +19,9 @@ permissions:
 
 jobs:
   call-label-action:
-    uses: dbt-labs/jira-actions/.github/workflows/jira-creation.yml@main
+    uses: dbt-labs/actions/.github/workflows/jira-creation.yml@main
+    with:
+      project_key: ADAP
     secrets:
       JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
       JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
diff --git a/.github/workflows/jira-label.yml b/.github/workflows/jira-label.yml
index 3da2e3a38..1637cbe38 100644
--- a/.github/workflows/jira-label.yml
+++ b/.github/workflows/jira-label.yml
@@ -19,7 +19,9 @@ permissions:
 
 jobs:
   call-label-action:
-    uses: dbt-labs/jira-actions/.github/workflows/jira-label.yml@main
+    uses: dbt-labs/actions/.github/workflows/jira-label.yml@main
+    with:
+      project_key: ADAP
     secrets:
       JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
       JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
diff --git a/.github/workflows/jira-transition.yml b/.github/workflows/jira-transition.yml
index ed9f9cd4f..99158a15f 100644
--- a/.github/workflows/jira-transition.yml
+++ b/.github/workflows/jira-transition.yml
@@ -15,9 +15,14 @@ on:
   issues:
     types: [closed, deleted, reopened]
 
+# no special access is needed
+permissions: read-all
+
 jobs:
   call-label-action:
-    uses: dbt-labs/jira-actions/.github/workflows/jira-transition.yml@main
+    uses: dbt-labs/actions/.github/workflows/jira-transition.yml@main
+    with:
+      project_key: ADAP
     secrets:
       JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
       JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}

From 93303d4b268b58e3b39134f512fd5cbdc0c916a7 Mon Sep 17 00:00:00 2001
From: Nathaniel May <nathaniel.may@fishtownanalytics.com>
Date: Mon, 27 Feb 2023 12:31:39 -0500
Subject: [PATCH 567/860] Revert "mirror issues to ADAP jira project (#541)"
 (#565)

This reverts commit 6b17008efec3ccc53449950f4b7891ac694baba9.
---
 .github/workflows/jira-creation.yml   | 4 +---
 .github/workflows/jira-label.yml      | 4 +---
 .github/workflows/jira-transition.yml | 7 +------
 3 files changed, 3 insertions(+), 12 deletions(-)

diff --git a/.github/workflows/jira-creation.yml b/.github/workflows/jira-creation.yml
index 2611a8bdd..b4016befc 100644
--- a/.github/workflows/jira-creation.yml
+++ b/.github/workflows/jira-creation.yml
@@ -19,9 +19,7 @@ permissions:
 
 jobs:
   call-label-action:
-    uses: dbt-labs/actions/.github/workflows/jira-creation.yml@main
-    with:
-      project_key: ADAP
+    uses: dbt-labs/jira-actions/.github/workflows/jira-creation.yml@main
     secrets:
       JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
       JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
diff --git a/.github/workflows/jira-label.yml b/.github/workflows/jira-label.yml
index 1637cbe38..3da2e3a38 100644
--- a/.github/workflows/jira-label.yml
+++ b/.github/workflows/jira-label.yml
@@ -19,9 +19,7 @@ permissions:
 
 jobs:
   call-label-action:
-    uses: dbt-labs/actions/.github/workflows/jira-label.yml@main
-    with:
-      project_key: ADAP
+    uses: dbt-labs/jira-actions/.github/workflows/jira-label.yml@main
     secrets:
       JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
       JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
diff --git a/.github/workflows/jira-transition.yml b/.github/workflows/jira-transition.yml
index 99158a15f..ed9f9cd4f 100644
--- a/.github/workflows/jira-transition.yml
+++ b/.github/workflows/jira-transition.yml
@@ -15,14 +15,9 @@ on:
   issues:
     types: [closed, deleted, reopened]
 
-# no special access is needed
-permissions: read-all
-
 jobs:
   call-label-action:
-    uses: dbt-labs/actions/.github/workflows/jira-transition.yml@main
-    with:
-      project_key: ADAP
+    uses: dbt-labs/jira-actions/.github/workflows/jira-transition.yml@main
     secrets:
       JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
       JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}

From 626bf1604985fd2160f1e49b863d77d0afce676b Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Mon, 27 Feb 2023 12:40:02 -0500
Subject: [PATCH 568/860] Ct 2183/setup tools distro info (#561)

* unpin dev requirements
* remove duplicated downstream dependencies to reduce version conflicts
* add docker image for development purposes
* add tox to the image
* add py3.9 and py3.8 containers to research broken test
---
 .dockerignore                |  2 ++
 Makefile                     | 21 +++++++++++++++
 dev-requirements.txt         | 37 +++++++++++++-------------
 docker_dev/README.md         |  5 ++++
 docker_dev/ubuntu.Dockerfile | 50 ++++++++++++++++++++++++++++++++++++
 setup.py                     |  6 +----
 6 files changed, 98 insertions(+), 23 deletions(-)
 create mode 100644 .dockerignore
 create mode 100644 Makefile
 create mode 100644 docker_dev/README.md
 create mode 100644 docker_dev/ubuntu.Dockerfile

diff --git a/.dockerignore b/.dockerignore
new file mode 100644
index 000000000..af1374065
--- /dev/null
+++ b/.dockerignore
@@ -0,0 +1,2 @@
+*
+!docker_dev
diff --git a/Makefile b/Makefile
new file mode 100644
index 000000000..2d8b44e86
--- /dev/null
+++ b/Makefile
@@ -0,0 +1,21 @@
+.DEFAULT_GOAL:=help
+
+.PHONY: dev
+dev: ## Installs adapter in develop mode along with development dependencies
+	@\
+	pip install -e . -r dev-requirements.txt && pre-commit install
+
+.PHONY: ubuntu-py311
+ubuntu-py311: ## Builds and runs an Ubuntu Python 3.11 development container
+	docker build -f docker_dev/ubuntu.Dockerfile -t dbt-bigquery-ubuntu-py311 .
+	docker run --rm -it --name dbt-bigquery-ubuntu-py311 -v $(shell pwd):/opt/code dbt-bigquery-ubuntu-py311
+
+.PHONY: ubuntu-py39
+ubuntu-py39: ## Builds and runs an Ubuntu Python 3.9 development container
+	docker build -f docker_dev/ubuntu.Dockerfile -t dbt-bigquery-ubuntu-py39 . --build-arg version=3.9
+	docker run --rm -it --name dbt-bigquery-ubuntu-py39 -v $(shell pwd):/opt/code dbt-bigquery-ubuntu-py39
+
+.PHONY: ubuntu-py38
+ubuntu-py38: ## Builds and runs an Ubuntu Python 3.8 development container
+	docker build -f docker_dev/ubuntu.Dockerfile -t dbt-bigquery-ubuntu-py38 . --build-arg version=3.8
+	docker run --rm -it --name dbt-bigquery-ubuntu-py38 -v $(shell pwd):/opt/code dbt-bigquery-ubuntu-py38
diff --git a/dev-requirements.txt b/dev-requirements.txt
index f489a04b3..d1dfb1be5 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -2,22 +2,23 @@
 # TODO: how to automate switching from develop to version branches?
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
-black~=22.8.0
-bumpversion~=0.6.0
+
+black
+bumpversion
 flake8
-flaky~=3.7.0
-freezegun~=1.1.0
-ipdb~=0.13.9
-mypy~=0.971.0
-pip-tools~=6.11.0
-pre-commit~=2.20.0
-pytest~=7.2.0
-pytest-dotenv~=0.5.2
-pytest-logbook~=1.2.0
-pytest-csv~=3.0.0
-pytest-xdist~=3.1.0
-pytz~=2022.6.0
-tox~=3.13
-types-requests~=2.28.11
-twine~=4.0.2
-wheel~=0.37.1
+flaky
+freezegun
+ipdb
+mypy
+pip-tools
+pre-commit
+pytest
+pytest-dotenv
+pytest-logbook
+pytest-csv
+pytest-xdist
+pytz
+tox
+types-requests
+twine
+wheel
diff --git a/docker_dev/README.md b/docker_dev/README.md
new file mode 100644
index 000000000..dd487fea7
--- /dev/null
+++ b/docker_dev/README.md
@@ -0,0 +1,5 @@
+# Docker Dev Images
+
+These images are solely for development purposes. They are
+saved here for convenience. There should be no expectation
+of stability or maintenance.
diff --git a/docker_dev/ubuntu.Dockerfile b/docker_dev/ubuntu.Dockerfile
new file mode 100644
index 000000000..bac3f5993
--- /dev/null
+++ b/docker_dev/ubuntu.Dockerfile
@@ -0,0 +1,50 @@
+FROM ubuntu:latest
+
+# default to py3.11, this can be overridden at build, e.g. `docker build ... --build-arg version=3.10`
+ARG version=3.11
+
+# prevent python installation from asking for time zone region
+ARG DEBIAN_FRONTEND=noninteractive
+
+# get add-apt-repository
+RUN apt-get update && \
+    apt-get install -y software-properties-common
+
+# add the python repository
+RUN apt-get update && \
+    add-apt-repository -y ppa:deadsnakes/ppa
+
+# install python and git (for installing dbt-core)
+RUN apt-get update && \
+    apt-get install -y --no-install-recommends \
+        python$version \
+        python$version-dev \
+        python$version-distutils \
+        python$version-venv \
+        python3-pip \
+        python3-wheel \
+        build-essential \
+        git-all
+
+# clean up
+RUN apt-get clean && \
+    rm -rf \
+        /var/lib/apt/lists/* \
+        /tmp/* \
+        /var/tmp/*
+
+# update the default system interpreter to the newly installed version
+RUN update-alternatives --install /usr/bin/python3 python3 /usr/bin/python$version 1
+
+# setup mount for our code
+WORKDIR /opt/code
+VOLUME /opt/code
+
+# install tox in the system interpreter (it creates it's own virtual environments)
+RUN pip install tox
+
+# explicitly create a virtual environment as well for interactive testing
+RUN python3 -m venv /opt/venv
+
+# send stdout/stderr to terminal
+ENV PYTHONUNBUFFERED=1
diff --git a/setup.py b/setup.py
index 9fa89a319..2d015f7a5 100644
--- a/setup.py
+++ b/setup.py
@@ -75,14 +75,10 @@ def _dbt_core_version(plugin_version: str) -> str:
     include_package_data=True,
     install_requires=[
         f"dbt-core~={_dbt_core_version(_dbt_bigquery_version())}",
-        "protobuf>=3.13.0,<4",
-        "google-cloud-core~=2.0",
         "google-cloud-bigquery~=3.0",
-        "google-api-core~=2.0",
-        "googleapis-common-protos~=1.6",
         "google-cloud-storage~=2.4",
         "google-cloud-dataproc~=5.0",
-        "agate>=1.6.3,<1.7",
+        "agate~=1.6.3",
     ],
     zip_safe=False,
     classifiers=[

From acebf2ed391c434d0446724ab167d576b79062b1 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 1 Mar 2023 15:11:09 -0500
Subject: [PATCH 569/860] Ct 2183/setup tools distro info (#572)

* remove duplicated downstream dependencies to reduce version conflicts
* add docker images for development purposes
* add tox to the image
* pin dev dependencies in alignment with our standard
* updated requirements and pre-commit-config to standard for adapters
* update flake8 config
---
 .flake8                       | 12 +++++----
 .pre-commit-config.yaml       | 24 +++++++++++++-----
 Makefile                      |  5 ++++
 dbt/adapters/bigquery/impl.py |  3 ---
 dev-requirements.txt          | 46 ++++++++++++++++++++---------------
 5 files changed, 57 insertions(+), 33 deletions(-)

diff --git a/.flake8 b/.flake8
index 5e6fa8903..11baa8ee0 100644
--- a/.flake8
+++ b/.flake8
@@ -4,9 +4,11 @@ select =
     W
     F
 ignore =
-    W503 # makes Flake8 work like black
-    W504
-    E203 # makes Flake8 work like black
-    E741
-    E501
+    # makes Flake8 work like black
+    W503,
+    W504,
+    # makes Flake8 work like black
+    E203,
+    E741,
+    E501,
 exclude = tests
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index c75c98b70..7867b440f 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -8,7 +8,7 @@ default_language_version:
 
 repos:
 - repo: https://github.com/pre-commit/pre-commit-hooks
-  rev: v3.2.0
+  rev: v4.4.0
   hooks:
   - id: check-yaml
     args: [--unsafe]
@@ -17,37 +17,49 @@ repos:
   - id: trailing-whitespace
   - id: check-case-conflict
 - repo: https://github.com/psf/black
-  rev: 22.3.0
+  rev: 23.1.0
   hooks:
   - id: black
+    additional_dependencies: ['click~=8.1']
     args:
     - "--line-length=99"
     - "--target-version=py38"
   - id: black
     alias: black-check
     stages: [manual]
+    additional_dependencies: ['click~=8.1']
     args:
     - "--line-length=99"
     - "--target-version=py38"
     - "--check"
     - "--diff"
 - repo: https://github.com/pycqa/flake8
-  rev: 4.0.1
+  rev: 6.0.0
   hooks:
   - id: flake8
   - id: flake8
     alias: flake8-check
     stages: [manual]
 - repo: https://github.com/pre-commit/mirrors-mypy
-  rev: v0.942
+  rev: v1.0.1
   hooks:
   - id: mypy
-    args: [--show-error-codes, --ignore-missing-imports]
+    # N.B.: Mypy is... a bit fragile.
+    #
+    # By using `language: system` we run this hook in the local
+    # environment instead of a pre-commit isolated one.  This is needed
+    # to ensure mypy correctly parses the project.
+
+    # It may cause trouble in that it adds environmental variables out
+    # of our control to the mix.  Unfortunately, there's nothing we can
+    # do about per pre-commit's author.
+    # See https://github.com/pre-commit/pre-commit/issues/730 for details.
+    args: [--show-error-codes, --ignore-missing-imports, --explicit-package-bases]
     files: ^dbt/adapters/.*
     language: system
   - id: mypy
     alias: mypy-check
     stages: [manual]
-    args: [--show-error-codes, --pretty, --ignore-missing-imports]
+    args: [--show-error-codes, --pretty, --ignore-missing-imports, --explicit-package-bases]
     files: ^dbt/adapters
     language: system
diff --git a/Makefile b/Makefile
index 2d8b44e86..c75b0c6a9 100644
--- a/Makefile
+++ b/Makefile
@@ -5,6 +5,11 @@ dev: ## Installs adapter in develop mode along with development dependencies
 	@\
 	pip install -e . -r dev-requirements.txt && pre-commit install
 
+.PHONY: dev-uninstall
+dev-uninstall: ## Uninstalls all packages while maintaining the virtual environment
+               ## Useful when updating versions, or if you accidentally installed into the system interpreter
+	pip freeze | grep -v "^-e" | cut -d "@" -f1 | xargs pip uninstall -y
+
 .PHONY: ubuntu-py311
 ubuntu-py311: ## Builds and runs an Ubuntu Python 3.11 development container
 	docker build -f docker_dev/ubuntu.Dockerfile -t dbt-bigquery-ubuntu-py311 .
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index c41df44c3..d26f1bdb1 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -141,7 +141,6 @@ class BigqueryConfig(AdapterConfig):
 
 
 class BigQueryAdapter(BaseAdapter):
-
     RELATION_TYPES = {
         "TABLE": RelationType.Table,
         "VIEW": RelationType.View,
@@ -182,7 +181,6 @@ def truncate_relation(self, relation: BigQueryRelation) -> None:
     def rename_relation(
         self, from_relation: BigQueryRelation, to_relation: BigQueryRelation
     ) -> None:
-
         conn = self.connections.get_thread_connection()
         client = conn.handle
 
@@ -668,7 +666,6 @@ def update_table_description(
 
     @available.parse_none
     def alter_table_add_columns(self, relation, columns):
-
         logger.debug('Adding columns ({}) to table {}".'.format(columns, relation))
 
         conn = self.connections.get_thread_connection()
diff --git a/dev-requirements.txt b/dev-requirements.txt
index d1dfb1be5..e7f7bd2fa 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -3,22 +3,30 @@
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
 
-black
-bumpversion
-flake8
-flaky
-freezegun
-ipdb
-mypy
-pip-tools
-pre-commit
-pytest
-pytest-dotenv
-pytest-logbook
-pytest-csv
-pytest-xdist
-pytz
-tox
-types-requests
-twine
-wheel
+# if version 1.x or greater -> pin to major version
+# if version 0.x -> pin to minor
+black~=23.1
+bumpversion~=0.6.0
+click~=8.1
+flake8~=5.0;python_version=="3.7"
+flake8~=6.0;python_version>="3.8"
+flaky~=3.7
+freezegun~=1.2
+ipdb~=0.13.11
+mypy==1.0.1  # patch updates have historically introduced breaking changes
+pip-tools~=6.12
+pre-commit~=2.21;python_version=="3.7"
+pre-commit~=3.1;python_version>="3.8"
+pre-commit-hooks~=4.4
+pytest~=7.2
+pytest-csv~=3.0
+pytest-dotenv~=0.5.2
+pytest-logbook~=1.2
+pytest-xdist~=3.2
+pytz~=2022.7
+tox~=3.0;python_version=="3.7"
+tox~=4.4;python_version>="3.8"
+types-pytz~=2022.7
+types-requests~=2.28
+twine~=4.0
+wheel~=0.38

From 1ca10a62c498dab0ff0daee91ce8b7ec4dbd88bf Mon Sep 17 00:00:00 2001
From: Nathaniel May <nathaniel.may@fishtownanalytics.com>
Date: Wed, 1 Mar 2023 16:53:03 -0500
Subject: [PATCH 570/860] Revert "Revert "mirror issues to ADAP jira project
 (#541)" (#565)" (#574)

This reverts commit 93303d4b268b58e3b39134f512fd5cbdc0c916a7.
---
 .github/workflows/jira-creation.yml   | 4 +++-
 .github/workflows/jira-label.yml      | 4 +++-
 .github/workflows/jira-transition.yml | 7 ++++++-
 3 files changed, 12 insertions(+), 3 deletions(-)

diff --git a/.github/workflows/jira-creation.yml b/.github/workflows/jira-creation.yml
index b4016befc..2611a8bdd 100644
--- a/.github/workflows/jira-creation.yml
+++ b/.github/workflows/jira-creation.yml
@@ -19,7 +19,9 @@ permissions:
 
 jobs:
   call-label-action:
-    uses: dbt-labs/jira-actions/.github/workflows/jira-creation.yml@main
+    uses: dbt-labs/actions/.github/workflows/jira-creation.yml@main
+    with:
+      project_key: ADAP
     secrets:
       JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
       JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
diff --git a/.github/workflows/jira-label.yml b/.github/workflows/jira-label.yml
index 3da2e3a38..1637cbe38 100644
--- a/.github/workflows/jira-label.yml
+++ b/.github/workflows/jira-label.yml
@@ -19,7 +19,9 @@ permissions:
 
 jobs:
   call-label-action:
-    uses: dbt-labs/jira-actions/.github/workflows/jira-label.yml@main
+    uses: dbt-labs/actions/.github/workflows/jira-label.yml@main
+    with:
+      project_key: ADAP
     secrets:
       JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
       JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
diff --git a/.github/workflows/jira-transition.yml b/.github/workflows/jira-transition.yml
index ed9f9cd4f..99158a15f 100644
--- a/.github/workflows/jira-transition.yml
+++ b/.github/workflows/jira-transition.yml
@@ -15,9 +15,14 @@ on:
   issues:
     types: [closed, deleted, reopened]
 
+# no special access is needed
+permissions: read-all
+
 jobs:
   call-label-action:
-    uses: dbt-labs/jira-actions/.github/workflows/jira-transition.yml@main
+    uses: dbt-labs/actions/.github/workflows/jira-transition.yml@main
+    with:
+      project_key: ADAP
     secrets:
       JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
       JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}

From c211f0a6ba1afeb0d6ae3de14860ad5d215f7445 Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Thu, 2 Mar 2023 19:26:38 +0000
Subject: [PATCH 571/860] Bumping version to 1.5.0b2 and generate changelog

---
 .bumpversion.cfg                                         | 2 +-
 .changes/1.5.0-b2.md                                     | 5 +++++
 .../Under the Hood-20230223-105149.yaml                  | 0
 CHANGELOG.md                                             | 9 ++++++++-
 dbt/adapters/bigquery/__version__.py                     | 2 +-
 setup.py                                                 | 2 +-
 6 files changed, 16 insertions(+), 4 deletions(-)
 create mode 100644 .changes/1.5.0-b2.md
 rename .changes/{unreleased => 1.5.0}/Under the Hood-20230223-105149.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index c8ca74834..8d5440a31 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.5.0b1
+current_version = 1.5.0b2
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.5.0-b2.md b/.changes/1.5.0-b2.md
new file mode 100644
index 000000000..a214d458c
--- /dev/null
+++ b/.changes/1.5.0-b2.md
@@ -0,0 +1,5 @@
+## dbt-bigquery 1.5.0-b2 - March 02, 2023
+
+### Under the Hood
+
+- Rename constraints_enabled to contract ([#548](https://github.com/dbt-labs/dbt-bigquery/issues/548))
diff --git a/.changes/unreleased/Under the Hood-20230223-105149.yaml b/.changes/1.5.0/Under the Hood-20230223-105149.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20230223-105149.yaml
rename to .changes/1.5.0/Under the Hood-20230223-105149.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index b922a3e12..48ed63e9d 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,6 +5,14 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.5.0-b2 - March 02, 2023
+
+### Under the Hood
+
+- Rename constraints_enabled to contract ([#548](https://github.com/dbt-labs/dbt-bigquery/issues/548))
+
+
+
 ## dbt-bigquery 1.5.0-b1 - February 22, 2023
 
 ### Features
@@ -27,7 +35,6 @@
 - [@dave-connors-3](https://github.com/dave-connors-3) ([#5260](https://github.com/dbt-labs/dbt-bigquery/issues/5260))
 - [@victoriapm](https://github.com/victoriapm) ([#444](https://github.com/dbt-labs/dbt-bigquery/issues/444))
 
-
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 * [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index c3758128c..4f8b15313 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.5.0b1"
+version = "1.5.0b2"
diff --git a/setup.py b/setup.py
index 2d015f7a5..7603a3498 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.5.0b1"
+package_version = "1.5.0b2"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From 3aa71d28da95f3b22a396d0c8457aa1c87535dd6 Mon Sep 17 00:00:00 2001
From: Christophe Oudar <kayrnt@gmail.com>
Date: Thu, 2 Mar 2023 22:43:06 +0100
Subject: [PATCH 572/860] Fix time ingestion partitioning option regression
 (#485)

* Fix regression require_partition_filter usage
with insert_overwrite fails on second run

* review changes

* merge remote

* test clean up

* test clean up

* remove log call

---------

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Co-authored-by: Colin <colin.rogers@dbtlabs.com>
---
 .../unreleased/Fixes-20230202-010332.yaml     |  8 +++
 dbt/adapters/bigquery/impl.py                 | 27 +++++--
 .../incremental_strategy/common.sql           |  2 +-
 .../incremental_strategy/insert_overwrite.sql |  2 +-
 .../test_incremental_strategies.py            | 66 -----------------
 .../incremental_strategy_fixtures.py          | 45 ++++++++++++
 .../{bigquery_test => incremental}/seeds.py   |  8 +++
 .../test_incremental_strategies.py            | 72 +++++++++++++++++++
 .../fixtures.py                               |  0
 .../adapter/simple_bigquery_view/seeds.py     | 57 +++++++++++++++
 .../test_simple_bigquery_view.py              |  4 +-
 11 files changed, 214 insertions(+), 77 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230202-010332.yaml
 delete mode 100644 tests/functional/adapter/bigquery_test/test_incremental_strategies.py
 rename tests/functional/adapter/{bigquery_test => incremental}/incremental_strategy_fixtures.py (88%)
 rename tests/functional/adapter/{bigquery_test => incremental}/seeds.py (89%)
 create mode 100644 tests/functional/adapter/incremental/test_incremental_strategies.py
 rename tests/functional/adapter/{bigquery_test => simple_bigquery_view}/fixtures.py (100%)
 create mode 100644 tests/functional/adapter/simple_bigquery_view/seeds.py
 rename tests/functional/adapter/{bigquery_test => simple_bigquery_view}/test_simple_bigquery_view.py (96%)

diff --git a/.changes/unreleased/Fixes-20230202-010332.yaml b/.changes/unreleased/Fixes-20230202-010332.yaml
new file mode 100644
index 000000000..f6062e7ea
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230202-010332.yaml
@@ -0,0 +1,8 @@
+kind: Fixes
+body: Fix time ingestion partitioning option regression when combined using `require_partition_filter`
+  option on incremental run
+time: 2023-02-02T01:03:32.577336+01:00
+custom:
+  Author: Kayrnt
+  Issue: "483"
+  PR: "485"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index d26f1bdb1..bf7eabeda 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -72,24 +72,37 @@ class PartitionConfig(dbtClassMixin):
     time_ingestion_partitioning: bool = False
     copy_partitions: bool = False
 
+    def data_type_for_partition(self):
+        """Return the data type of partitions for replacement."""
+        return self.data_type if not self.time_ingestion_partitioning else "timestamp"
+
     def reject_partition_field_column(self, columns: List[Any]) -> List[str]:
         return [c for c in columns if not c.name.upper() == self.field.upper()]
 
+    def data_type_should_be_truncated(self):
+        """Return true if the data type should be truncated instead of cast to the data type."""
+        return not (
+            self.data_type.lower() == "int64"
+            or (self.data_type.lower() == "date" and self.granularity.lower() == "day")
+        )
+
     def render(self, alias: Optional[str] = None):
         column: str = self.field if not self.time_ingestion_partitioning else "_PARTITIONTIME"
         if alias:
             column = f"{alias}.{column}"
 
-        if self.data_type.lower() == "int64" or (
-            self.data_type.lower() == "date" and self.granularity.lower() == "day"
-        ):
-            return column
-        else:
+        if self.data_type_should_be_truncated():
             return f"{self.data_type}_trunc({column}, {self.granularity})"
+        else:
+            return column
 
     def render_wrapped(self, alias: Optional[str] = None):
-        """Wrap the partitioning column when time involved to ensure it is properly casted to matching time."""
-        if self.data_type in ("date", "timestamp", "datetime"):
+        """Wrap the partitioning column when time involved to ensure it is properly cast to matching time."""
+        # if data type is going to be truncated, no need to wrap
+        if (
+            self.data_type in ("date", "timestamp", "datetime")
+            and not self.data_type_should_be_truncated()
+        ):
             return f"{self.data_type}({self.render(alias)})"
         else:
             return self.render(alias)
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
index 237dfbbf0..b9f8560d9 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
@@ -12,7 +12,7 @@
   {#-- TODO: revisit partitioning with python models --#}
   {%- if '_dbt_max_partition' in compiled_code and language == 'sql' -%}
 
-    declare _dbt_max_partition {{ partition_by.data_type }} default (
+    declare _dbt_max_partition {{ partition_by.data_type_for_partition() }} default (
       select max({{ partition_by.field }}) from {{ this }}
       where {{ partition_by.field }} is not null
     );
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
index 5cd5a5354..63df65e2a 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
@@ -119,7 +119,7 @@
       {%- endset -%}
 
       -- generated script to merge partitions into {{ target_relation }}
-      declare dbt_partitions_for_replacement array<{{ partition_by.data_type }}>;
+      declare dbt_partitions_for_replacement array<{{ partition_by.data_type_for_partition() }}>;
 
       {# have we already created the temp table to check for schema changes? #}
       {% if not tmp_relation_exists %}
diff --git a/tests/functional/adapter/bigquery_test/test_incremental_strategies.py b/tests/functional/adapter/bigquery_test/test_incremental_strategies.py
deleted file mode 100644
index 76b7ac054..000000000
--- a/tests/functional/adapter/bigquery_test/test_incremental_strategies.py
+++ /dev/null
@@ -1,66 +0,0 @@
-import pytest
-from dbt.tests.util import (
-    check_relations_equal,
-    run_dbt
-)
-from dbt.tests.adapter.simple_seed.test_seed import SeedConfigBase
-from tests.functional.adapter.bigquery_test.seeds import *
-from tests.functional.adapter.bigquery_test.incremental_strategy_fixtures import *
-
-
-class TestBigQueryScripting(SeedConfigBase):
-    @pytest.fixture(scope="class")
-    def schema(self):
-        return "bigquery_test"
-
-    @pytest.fixture(scope="class")
-    def models(self):
-        return {
-            "incremental_merge_range.sql": merge_range_sql,
-            "incremental_merge_time.sql": merge_time_sql,
-            "incremental_overwrite_date.sql": overwrite_date_sql,
-            "incremental_overwrite_day.sql": overwrite_day_sql,
-            "incremental_overwrite_day_with_copy_partitions.sql": overwrite_day_with_copy_partitions_sql,
-            "incremental_overwrite_partitions.sql": overwrite_partitions_sql,
-            "incremental_overwrite_range.sql": overwrite_range_sql,
-            "incremental_overwrite_time.sql": overwrite_time_sql,
-        }
-
-    @pytest.fixture(scope="class")
-    def seeds(self):
-        return {
-            "data_seed.csv": seed_data_csv,
-            "merge_expected.csv": seed_merge_expected_csv,
-            "incremental_overwrite_time_expected.csv": seed_incremental_overwrite_time_expected_csv,
-            "incremental_overwrite_date_expected.csv": seed_incremental_overwrite_date_expected_csv,
-            "incremental_overwrite_day_expected.csv": seed_incremental_overwrite_day_expected_csv,
-            "incremental_overwrite_range_expected.csv": seed_incremental_overwrite_range_expected_csv
-        }
-
-    def test__bigquery_assert_incrementals(self, project):
-        results = run_dbt()
-        assert len(results) == 8
-
-        results = run_dbt()
-        assert len(results) == 8
-
-        seed_results = run_dbt(['seed'])
-
-        db_with_schema = f"{project.database}.{project.test_schema}"
-        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_merge_range",
-                                                f"{db_with_schema}.merge_expected"])
-        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_merge_time",
-                                                f"{db_with_schema}.merge_expected"])
-        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_overwrite_time",
-                                                f"{db_with_schema}.incremental_overwrite_time_expected"])
-        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_overwrite_date",
-                                                f"{db_with_schema}.incremental_overwrite_date_expected"])
-        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_overwrite_partitions",
-                                                f"{db_with_schema}.incremental_overwrite_date_expected"])
-        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_overwrite_day",
-                                                f"{db_with_schema}.incremental_overwrite_day_expected"])
-        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_overwrite_range",
-                                                f"{db_with_schema}.incremental_overwrite_range_expected"])
-        check_relations_equal(project.adapter, [f"{db_with_schema}.incremental_overwrite_day_with_copy_partitions",
-                                                f"{db_with_schema}.incremental_overwrite_day_expected"])
-
diff --git a/tests/functional/adapter/bigquery_test/incremental_strategy_fixtures.py b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
similarity index 88%
rename from tests/functional/adapter/bigquery_test/incremental_strategy_fixtures.py
rename to tests/functional/adapter/incremental/incremental_strategy_fixtures.py
index 7baee6c3d..084f4da6b 100644
--- a/tests/functional/adapter/bigquery_test/incremental_strategy_fixtures.py
+++ b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
@@ -355,3 +355,48 @@
 where date_hour >= _dbt_max_partition
 {% endif %}
 """.lstrip()
+
+overwrite_day_with_time_ingestion_sql = """
+{{
+    config(
+        materialized="incremental",
+        incremental_strategy='insert_overwrite',
+        cluster_by="id",
+        partition_by={
+            "field": "date_time",
+            "data_type": "datetime",
+            "time_ingestion_partitioning": true
+        },
+        require_partition_filter=true
+    )
+}}
+
+
+with data as (
+
+    {% if not is_incremental() %}
+
+        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 4 as id, cast('2020-01-01' as datetime) as date_time
+
+    {% else %}
+
+        -- we want to overwrite the 4 records in the 2020-01-01 partition
+        -- with the 2 records below, but add two more in the 2020-01-02 partition
+        select 10 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 20 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 30 as id, cast('2020-01-02' as datetime) as date_time union all
+        select 40 as id, cast('2020-01-02' as datetime) as date_time
+
+    {% endif %}
+
+)
+
+select * from data
+
+{% if is_incremental() %}
+where date_time > '2020-01-01'
+{% endif %}
+""".lstrip()
\ No newline at end of file
diff --git a/tests/functional/adapter/bigquery_test/seeds.py b/tests/functional/adapter/incremental/seeds.py
similarity index 89%
rename from tests/functional/adapter/bigquery_test/seeds.py
rename to tests/functional/adapter/incremental/seeds.py
index f7e7bb9d3..ccc265e5f 100644
--- a/tests/functional/adapter/bigquery_test/seeds.py
+++ b/tests/functional/adapter/incremental/seeds.py
@@ -47,3 +47,11 @@
 5,2020-01-02 00:00:00
 6,2020-01-02 00:00:00
 """.lstrip()
+
+seed_incremental_overwrite_day_with_time_partition_expected_csv = """
+id
+10
+20
+30
+40
+""".lstrip()
\ No newline at end of file
diff --git a/tests/functional/adapter/incremental/test_incremental_strategies.py b/tests/functional/adapter/incremental/test_incremental_strategies.py
new file mode 100644
index 000000000..aae438214
--- /dev/null
+++ b/tests/functional/adapter/incremental/test_incremental_strategies.py
@@ -0,0 +1,72 @@
+import pytest
+from dbt.tests.util import (
+    check_relations_equal,
+    get_relation_columns,
+    run_dbt,
+)
+from dbt.tests.adapter.simple_seed.test_seed import SeedConfigBase
+from tests.functional.adapter.incremental.seeds import *
+from tests.functional.adapter.incremental.incremental_strategy_fixtures import *
+
+
+class TestBigQueryScripting(SeedConfigBase):
+    @pytest.fixture(scope="class")
+    def schema(self):
+        return "bigquery_test"
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "incremental_merge_range.sql": merge_range_sql,
+            "incremental_merge_time.sql": merge_time_sql,
+            "incremental_overwrite_date.sql": overwrite_date_sql,
+            "incremental_overwrite_day.sql": overwrite_day_sql,
+            "incremental_overwrite_day_with_copy_partitions.sql": overwrite_day_with_copy_partitions_sql,
+            "incremental_overwrite_partitions.sql": overwrite_partitions_sql,
+            "incremental_overwrite_range.sql": overwrite_range_sql,
+            "incremental_overwrite_time.sql": overwrite_time_sql,
+            "incremental_overwrite_day_with_time_ingestion.sql": overwrite_day_with_time_ingestion_sql
+        }
+
+    @pytest.fixture(scope="class")
+    def seeds(self):
+        return {
+            "data_seed.csv": seed_data_csv,
+            "merge_expected.csv": seed_merge_expected_csv,
+            "incremental_overwrite_time_expected.csv": seed_incremental_overwrite_time_expected_csv,
+            "incremental_overwrite_date_expected.csv": seed_incremental_overwrite_date_expected_csv,
+            "incremental_overwrite_day_expected.csv": seed_incremental_overwrite_day_expected_csv,
+            "incremental_overwrite_range_expected.csv": seed_incremental_overwrite_range_expected_csv,
+            "incremental_overwrite_day_with_time_partition_expected.csv": seed_incremental_overwrite_day_with_time_partition_expected_csv
+        }
+
+    def test__bigquery_assert_incremental_configurations_apply_the_right_strategy(self, project):
+        run_dbt(['seed'])
+        results = run_dbt()
+        assert len(results) == 9
+
+        results = run_dbt()
+        assert len(results) == 9
+        incremental_strategies = [
+            ('incremental_merge_range', 'merge_expected'),
+            ("incremental_merge_time", "merge_expected"),
+            ("incremental_overwrite_time",
+             "incremental_overwrite_time_expected"),
+            ("incremental_overwrite_date",
+             "incremental_overwrite_date_expected"),
+            ("incremental_overwrite_partitions",
+             "incremental_overwrite_date_expected"),
+            ("incremental_overwrite_day", "incremental_overwrite_day_expected"),
+            ("incremental_overwrite_range", "incremental_overwrite_range_expected"),
+        ]
+        db_with_schema = f"{project.database}.{project.test_schema}"
+        for incremental_strategy in incremental_strategies:
+            created_table = f"{db_with_schema}.{incremental_strategy[0]}"
+            expected_table = f"{db_with_schema}.{incremental_strategy[1]}"
+            check_relations_equal(project.adapter, [created_table, expected_table])
+
+        # since this table requires a partition filter which check_relations_equal doesn't support extra where clauses
+        # we just check column types
+        created = get_relation_columns(project.adapter, "incremental_overwrite_day_with_copy_partitions")
+        expected = get_relation_columns(project.adapter, "incremental_overwrite_day_expected")
+        assert created == expected
diff --git a/tests/functional/adapter/bigquery_test/fixtures.py b/tests/functional/adapter/simple_bigquery_view/fixtures.py
similarity index 100%
rename from tests/functional/adapter/bigquery_test/fixtures.py
rename to tests/functional/adapter/simple_bigquery_view/fixtures.py
diff --git a/tests/functional/adapter/simple_bigquery_view/seeds.py b/tests/functional/adapter/simple_bigquery_view/seeds.py
new file mode 100644
index 000000000..ccc265e5f
--- /dev/null
+++ b/tests/functional/adapter/simple_bigquery_view/seeds.py
@@ -0,0 +1,57 @@
+seed_data_csv = """
+id,dupe
+1,a
+2,a
+3,a
+4,a
+""".lstrip()
+
+seed_incremental_overwrite_date_expected_csv = """
+id,date_day
+10,2020-01-01
+20,2020-01-01
+30,2020-01-02
+40,2020-01-02
+""".lstrip()
+
+seed_incremental_overwrite_day_expected_csv = """
+id,date_time
+10,2020-01-01 00:00:00
+20,2020-01-01 00:00:00
+30,2020-01-02 00:00:00
+40,2020-01-02 00:00:00
+""".lstrip()
+
+seed_incremental_overwrite_range_expected_csv = """
+id,date_int
+10,20200101
+20,20200101
+30,20200102
+40,20200102
+""".lstrip()
+
+seed_incremental_overwrite_time_expected_csv = """
+id,date_hour
+10,2020-01-01 01:00:00
+20,2020-01-01 01:00:00
+30,2020-01-01 02:00:00
+40,2020-01-01 02:00:00
+""".lstrip()
+
+seed_merge_expected_csv = """
+id,date_time
+1,2020-01-01 00:00:00
+2,2020-01-01 00:00:00
+3,2020-01-01 00:00:00
+4,2020-01-02 00:00:00
+5,2020-01-02 00:00:00
+6,2020-01-02 00:00:00
+""".lstrip()
+
+seed_incremental_overwrite_day_with_time_partition_expected_csv = """
+id
+10
+20
+30
+40
+""".lstrip()
\ No newline at end of file
diff --git a/tests/functional/adapter/bigquery_test/test_simple_bigquery_view.py b/tests/functional/adapter/simple_bigquery_view/test_simple_bigquery_view.py
similarity index 96%
rename from tests/functional/adapter/bigquery_test/test_simple_bigquery_view.py
rename to tests/functional/adapter/simple_bigquery_view/test_simple_bigquery_view.py
index fd2a9f0f6..75d6eb3d8 100644
--- a/tests/functional/adapter/bigquery_test/test_simple_bigquery_view.py
+++ b/tests/functional/adapter/simple_bigquery_view/test_simple_bigquery_view.py
@@ -3,8 +3,8 @@
 import time
 from dbt.tests.util import run_dbt
 from dbt.tests.adapter.simple_seed.test_seed import SeedConfigBase
-from tests.functional.adapter.bigquery_test.seeds import *
-from tests.functional.adapter.bigquery_test.fixtures import *
+from tests.functional.adapter.simple_bigquery_view.seeds import *
+from tests.functional.adapter.simple_bigquery_view.fixtures import *
 
 
 class BaseBigQueryRun(SeedConfigBase):

From 5214ebc294b9187c18fd2eee9868089eb2fc3e81 Mon Sep 17 00:00:00 2001
From: Michelle Ark <MichelleArk@users.noreply.github.com>
Date: Fri, 3 Mar 2023 15:06:18 -0500
Subject: [PATCH 573/860] implement get_column_schema_from_query (#551)

* implement get_column_schema_from_query
---
 .../unreleased/Features-20230223-145508.yaml  |  6 +++++
 dbt/adapters/bigquery/impl.py                 | 11 ++++++++
 .../macros/utils/get_columns_spec_ddl.sql     |  4 +++
 tests/functional/adapter/test_constraints.py  | 27 ++++++++++++++++++-
 4 files changed, 47 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Features-20230223-145508.yaml

diff --git a/.changes/unreleased/Features-20230223-145508.yaml b/.changes/unreleased/Features-20230223-145508.yaml
new file mode 100644
index 000000000..ade28e2b5
--- /dev/null
+++ b/.changes/unreleased/Features-20230223-145508.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: 'implement get_column_schema_from_query '
+time: 2023-02-23T14:55:08.186645-05:00
+custom:
+  Author: michelleark
+  Issue: "529"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index bf7eabeda..af59e2666 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -465,6 +465,17 @@ def copy_table(self, source, destination, materialization):
 
         return "COPY TABLE with materialization: {}".format(materialization)
 
+    @available.parse(lambda *a, **k: [])
+    def get_column_schema_from_query(self, sql: str) -> List[BigQueryColumn]:
+        """Get a list of the column names and data types from the given sql.
+
+        :param str sql: The sql to execute.
+        :return: List[BigQueryColumn]
+        """
+        _, iterator = self.connections.raw_execute(sql)
+        columns = [self.Column.create_from_field(field) for field in iterator.schema]
+        return columns
+
     @available.parse(lambda *a, **k: False)
     def get_columns_in_select_sql(self, select_sql: str) -> List[BigQueryColumn]:
         try:
diff --git a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
index 4f2720b7e..f074f1bee 100644
--- a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
+++ b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
@@ -14,3 +14,7 @@
       {{exceptions.warn("We noticed you have `constraints_check` configs, these are NOT compatible with BigQuery and will be ignored")}}
   {%- endif %}
 {% endmacro %}
+
+{% macro bigquery__format_column(column) -%}
+  {{ return(column.column.lower() ~ " " ~ column.data_type) }}
+{%- endmacro -%}
diff --git a/tests/functional/adapter/test_constraints.py b/tests/functional/adapter/test_constraints.py
index b9a271b8c..85abc7acc 100644
--- a/tests/functional/adapter/test_constraints.py
+++ b/tests/functional/adapter/test_constraints.py
@@ -31,6 +31,7 @@
 # - raises an explicit error, if you try to set a primary key constraint, because it's not enforced
 constraints_yml = model_schema_yml.replace("text", "string").replace("primary key", "")
 
+
 class TestBigQueryConstraintsColumnsEqual(BaseConstraintsColumnsEqual):
     @pytest.fixture(scope="class")
     def models(self):
@@ -40,6 +41,30 @@ def models(self):
             "constraints_schema.yml": constraints_yml,
         }
 
+    @pytest.fixture
+    def string_type(self):
+        return "STRING"
+
+    @pytest.fixture
+    def int_type(self):
+        return "INT64"
+
+    @pytest.fixture
+    def data_types(self, int_type, string_type):
+        # sql_column_value, schema_data_type, error_data_type
+        return [
+            ['1', int_type, int_type],
+            ["'1'", string_type, string_type],
+            ["cast('2019-01-01' as date)", 'date', 'DATE'],
+            ["true", 'bool', 'BOOL'],
+            ["cast('2013-11-03 00:00:00-07' as TIMESTAMP)", 'timestamp', 'TIMESTAMP'],
+            ["['a','b','c']", f'ARRAY<{string_type}>', f'ARRAY<{string_type}>'],
+            ["[1,2,3]", f'ARRAY<{int_type}>', f'ARRAY<{int_type}>'],
+            ["cast(1 as NUMERIC)", 'numeric', 'NUMERIC'],
+            ["""JSON '{"name": "Cooper", "forname": "Alice"}'""", 'json', 'JSON'],
+            ['STRUCT("Rudisha" AS name, [23.4, 26.3, 26.4, 26.1] AS laps)', 'STRUCT<name STRING, laps ARRAY<FLOAT64>>', 'STRUCT<name STRING, laps ARRAY<FLOAT64>>']
+        ]
+
 
 class TestBigQueryConstraintsRuntimeEnforcement(BaseConstraintsRuntimeEnforcement):
     @pytest.fixture(scope="class")
@@ -48,7 +73,7 @@ def models(self):
             "my_model.sql": my_model_sql,
             "constraints_schema.yml": constraints_yml,
         }
-    
+
     @pytest.fixture(scope="class")
     def expected_sql(self, project):
         relation = relation_from_name(project.adapter, "my_model")

From d0353e659256fe576a697a4cf9b7dd692a70604e Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Wed, 8 Mar 2023 11:27:48 -0600
Subject: [PATCH 574/860] support contracts on models materialized as view
 (#584)

* first pass with contract check

* fix inheritence

* changelog

* fix test

* remove dbt-core pin
---
 .../unreleased/Features-20230223-145508.yaml  |  6 ++--
 dbt/include/bigquery/macros/adapters.sql      |  3 ++
 tests/functional/adapter/test_constraints.py  | 35 +++++++++++++------
 3 files changed, 31 insertions(+), 13 deletions(-)

diff --git a/.changes/unreleased/Features-20230223-145508.yaml b/.changes/unreleased/Features-20230223-145508.yaml
index ade28e2b5..cab391802 100644
--- a/.changes/unreleased/Features-20230223-145508.yaml
+++ b/.changes/unreleased/Features-20230223-145508.yaml
@@ -1,6 +1,6 @@
 kind: Features
-body: 'implement get_column_schema_from_query '
+body: Enforce contracts on models materialized as tables and views
 time: 2023-02-23T14:55:08.186645-05:00
 custom:
-  Author: michelleark
-  Issue: "529"
+  Author: michelleark emmyoop
+  Issue: 529 555
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index 4e0dfc904..59785db23 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -90,6 +90,9 @@
 
   create or replace view {{ relation }}
   {{ bigquery_view_options(config, model) }}
+  {% if config.get('contract', False) -%}
+    {{ get_assert_columns_equivalent(sql) }}
+  {%- endif %}
   as {{ sql }};
 
 {% endmacro %}
diff --git a/tests/functional/adapter/test_constraints.py b/tests/functional/adapter/test_constraints.py
index 85abc7acc..fb15c6a9d 100644
--- a/tests/functional/adapter/test_constraints.py
+++ b/tests/functional/adapter/test_constraints.py
@@ -1,13 +1,16 @@
 import pytest
 from dbt.tests.util import relation_from_name
 from dbt.tests.adapter.constraints.test_constraints import (
-    BaseConstraintsColumnsEqual,
+    BaseTableConstraintsColumnsEqual,
+    BaseViewConstraintsColumnsEqual,
     BaseConstraintsRuntimeEnforcement
 )
 from dbt.tests.adapter.constraints.fixtures import (
     my_model_sql,
     my_model_wrong_order_sql,
     my_model_wrong_name_sql,
+    my_model_view_wrong_order_sql,
+    my_model_view_wrong_name_sql,
     model_schema_yml,
 )
 
@@ -32,15 +35,7 @@
 constraints_yml = model_schema_yml.replace("text", "string").replace("primary key", "")
 
 
-class TestBigQueryConstraintsColumnsEqual(BaseConstraintsColumnsEqual):
-    @pytest.fixture(scope="class")
-    def models(self):
-        return {
-            "my_model_wrong_order.sql": my_model_wrong_order_sql,
-            "my_model_wrong_name.sql": my_model_wrong_name_sql,
-            "constraints_schema.yml": constraints_yml,
-        }
-
+class BigQueryColumnEqualSetup:
     @pytest.fixture
     def string_type(self):
         return "STRING"
@@ -66,6 +61,26 @@ def data_types(self, int_type, string_type):
         ]
 
 
+class TestBigQueryTableConstraintsColumnsEqual(BigQueryColumnEqualSetup, BaseTableConstraintsColumnsEqual):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model_wrong_order.sql": my_model_wrong_order_sql,
+            "my_model_wrong_name.sql": my_model_wrong_name_sql,
+            "constraints_schema.yml": constraints_yml,
+        }
+
+
+class TestBigQueryViewConstraintsColumnsEqual(BigQueryColumnEqualSetup, BaseViewConstraintsColumnsEqual):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model_wrong_order.sql": my_model_view_wrong_order_sql,
+            "my_model_wrong_name.sql": my_model_view_wrong_name_sql,
+            "constraints_schema.yml": constraints_yml,
+        }
+
+
 class TestBigQueryConstraintsRuntimeEnforcement(BaseConstraintsRuntimeEnforcement):
     @pytest.fixture(scope="class")
     def models(self):

From fb136e305e0e116e67cd6618fe1baf499ee069f0 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 8 Mar 2023 15:32:26 -0600
Subject: [PATCH 575/860] add new scripts for cut-release workflow (#582)

* init push of adding two new bash scripts to be triggered off cut-release action

* add changelog

* remove unused stuff from sh file

* remove unneeded changelog
---
 .github/scripts/update_dependencies.sh   | 15 +++++++++++++++
 .github/scripts/update_release_branch.sh | 11 +++++++++++
 .github/workflows/cut-release-branch.yml |  2 +-
 3 files changed, 27 insertions(+), 1 deletion(-)
 create mode 100644 .github/scripts/update_dependencies.sh
 create mode 100644 .github/scripts/update_release_branch.sh

diff --git a/.github/scripts/update_dependencies.sh b/.github/scripts/update_dependencies.sh
new file mode 100644
index 000000000..c3df48e52
--- /dev/null
+++ b/.github/scripts/update_dependencies.sh
@@ -0,0 +1,15 @@
+#!/bin/bash -e
+set -e
+
+git_branch=$1
+target_req_file="dev-requirements.txt"
+core_req_sed_pattern="s|dbt-core.git.*#egg=dbt-core|dbt-core.git@${git_branch}#egg=dbt-core|g"
+tests_req_sed_pattern="s|dbt-core.git.*#egg=dbt-tests|dbt-core.git@${git_branch}#egg=dbt-tests|g"
+if [[ "$OSTYPE" == darwin* ]]; then
+ # mac ships with a different version of sed that requires a delimiter arg
+ sed -i "" "$core_req_sed_pattern" $target_req_file
+ sed -i "" "$tests_req_sed_pattern" $target_req_file
+else
+ sed -i "$core_req_sed_pattern" $target_req_file
+ sed -i "$tests_req_sed_pattern" $target_req_file
+fi
diff --git a/.github/scripts/update_release_branch.sh b/.github/scripts/update_release_branch.sh
new file mode 100644
index 000000000..75b9ccef6
--- /dev/null
+++ b/.github/scripts/update_release_branch.sh
@@ -0,0 +1,11 @@
+#!/bin/bash -e
+set -e
+
+release_branch=$1
+target_req_file=".github/workflows/nightly-release.yml"
+if [[ "$OSTYPE" == darwin* ]]; then
+ # mac ships with a different version of sed that requires a delimiter arg
+ sed -i "" "s|[0-9].[0-9].latest|$release_branch|" $target_req_file
+else
+ sed -i "s|[0-9].[0-9].latest|$release_branch|" $target_req_file
+fi
diff --git a/.github/workflows/cut-release-branch.yml b/.github/workflows/cut-release-branch.yml
index 827a2fff9..23edf45f2 100644
--- a/.github/workflows/cut-release-branch.yml
+++ b/.github/workflows/cut-release-branch.yml
@@ -37,6 +37,6 @@ jobs:
       version_to_bump_main: ${{ inputs.version_to_bump_main }}
       new_branch_name: ${{ inputs.new_branch_name }}
       PR_title: "Cleanup main after cutting new ${{ inputs.new_branch_name }} branch"
-      PR_body: "This PR will fail CI until the dbt-core PR has been merged due to release version conflicts.  dev-requirements.txt needs to be updated to have the dbt-core dependencies point to this new branch."
+      PR_body: "This PR will fail CI until the dbt-core PR has been merged due to release version conflicts."
     secrets:
       FISHTOWN_BOT_PAT: ${{ secrets.FISHTOWN_BOT_PAT }}

From f2f0e4272c42b2c2b59847279373c36fb08e3e37 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 9 Mar 2023 09:31:37 -0800
Subject: [PATCH 576/860] Dataproc config for serverless in profile and fix
 quoting (#578)

* Support configuring GCP Dataproc serverless jobs

This adds a `dataproc_batch` key for specifying the Dataproc Batch
configuration. At runtime this is used to populate the
google.cloud.dataproc_v1.types.Batch object before it is submitted to
the Dataproc service.

To avoid having to add explicit support for every option offered by the
service, and having to chase after a moving target as Google's API evolves,
this key accepts arbitrary yaml, which is mapped to the Batch object on
a best effort basis.

Signed-off-by: Torkjel Hongve <th@kinver.io>

* Fixes and tests

- Make dataproc_batch key optional.
- Unit tests
- Move configuration of the `google.cloud.dataproc_v1.Batch` object
  to a separate function.

Signed-off-by: Torkjel Hongve <th@kinver.io>

* Do not reinvent protobuf parsing.

* ws

* Fix unit tests to run without gcloud credentials.

* formatting

* Update dev-requirements.txt

* fix quote policy for py models, add python-test to tox.ini and cleanup python_submissions.py

* have mypy install types

* add changie

* add types-protobuf to dev-requirements.txt

* remove mypy install

* remove branch update from dev-requirements.txt

---------

Signed-off-by: Torkjel Hongve <th@kinver.io>
Co-authored-by: Torkjel Hongve <th@kinver.io>
Co-authored-by: Torkjel Hongve <torkjelh@gmail.com>
Co-authored-by: Chenyu Li <chenyu.li@dbtlabs.com>
Co-authored-by: Florian Eiden <florian.eiden@fleid.fr>
---
 .../unreleased/Features-20230303-132509.yaml  |  6 ++
 dbt/adapters/bigquery/connections.py          | 19 +++++-
 dbt/adapters/bigquery/python_submissions.py   | 68 ++++++++++++++-----
 .../bigquery/macros/python_model/python.sql   |  3 +
 dev-requirements.txt                          |  1 +
 tests/unit/test_bigquery_adapter.py           | 56 +++++++++++++++
 tests/unit/test_configure_dataproc_batch.py   | 57 ++++++++++++++++
 tox.ini                                       | 15 +++-
 8 files changed, 204 insertions(+), 21 deletions(-)
 create mode 100644 .changes/unreleased/Features-20230303-132509.yaml
 create mode 100644 dbt/include/bigquery/macros/python_model/python.sql
 create mode 100644 tests/unit/test_configure_dataproc_batch.py

diff --git a/.changes/unreleased/Features-20230303-132509.yaml b/.changes/unreleased/Features-20230303-132509.yaml
new file mode 100644
index 000000000..3a0ba8403
--- /dev/null
+++ b/.changes/unreleased/Features-20230303-132509.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: add dataproc serverless config to profile
+time: 2023-03-03T13:25:09.02695-08:00
+custom:
+  Author: colin-rogers-dbt torkjel
+  Issue: "530"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 7bcc63c4b..7e16f7485 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -1,7 +1,9 @@
 import json
 import re
 from contextlib import contextmanager
-from dataclasses import dataclass
+from dataclasses import dataclass, field
+from mashumaro.helper import pass_through
+
 from functools import lru_cache
 import agate
 from requests.exceptions import ConnectionError
@@ -35,7 +37,7 @@
 from dbt.events.types import SQLQuery
 from dbt.version import __version__ as dbt_version
 
-from dbt.dataclass_schema import StrEnum
+from dbt.dataclass_schema import ExtensibleDbtClassMixin, StrEnum
 
 logger = AdapterLogger("BigQuery")
 
@@ -92,6 +94,12 @@ class BigQueryAdapterResponse(AdapterResponse):
     slot_ms: Optional[int] = None
 
 
+@dataclass
+class DataprocBatchConfig(ExtensibleDbtClassMixin):
+    def __init__(self, batch_config):
+        self.batch_config = batch_config
+
+
 @dataclass
 class BigQueryCredentials(Credentials):
     method: BigQueryConnectionMethod
@@ -124,6 +132,13 @@ class BigQueryCredentials(Credentials):
     dataproc_cluster_name: Optional[str] = None
     gcs_bucket: Optional[str] = None
 
+    dataproc_batch: Optional[DataprocBatchConfig] = field(
+        metadata={
+            "serialization_strategy": pass_through,
+        },
+        default=None,
+    )
+
     scopes: Optional[Tuple[str, ...]] = (
         "https://www.googleapis.com/auth/bigquery",
         "https://www.googleapis.com/auth/cloud-platform",
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 76aff3fd9..e5fbf037e 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -2,9 +2,11 @@
 
 from dbt.adapters.base import PythonJobHelper
 from dbt.adapters.bigquery import BigQueryConnectionManager, BigQueryCredentials
+from dbt.adapters.bigquery.connections import DataprocBatchConfig
 from google.api_core import retry
 from google.api_core.client_options import ClientOptions
 from google.cloud import storage, dataproc_v1  # type: ignore
+from google.protobuf.json_format import ParseDict
 
 OPERATION_RETRY_TIME = 10
 
@@ -110,31 +112,17 @@ def _get_job_client(self) -> dataproc_v1.BatchControllerClient:
         )
 
     def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
-        # create the Dataproc Serverless job config
-        # need to pin dataproc version to 1.1 as it now defaults to 2.0
-        batch = dataproc_v1.Batch({"runtime_config": dataproc_v1.RuntimeConfig(version="1.1")})
-        batch.pyspark_batch.main_python_file_uri = self.gcs_location
-        # how to keep this up to date?
-        # we should probably also open this up to be configurable
-        jar_file_uri = self.parsed_model["config"].get(
-            "jar_file_uri",
-            "gs://spark-lib/bigquery/spark-bigquery-with-dependencies_2.12-0.21.1.jar",
-        )
-        batch.pyspark_batch.jar_file_uris = [jar_file_uri]
-        # should we make all of these spark/dataproc properties configurable?
-        # https://cloud.google.com/dataproc-serverless/docs/concepts/properties
-        # https://cloud.google.com/dataproc-serverless/docs/reference/rest/v1/projects.locations.batches#runtimeconfig
-        batch.runtime_config.properties = {
-            "spark.executor.instances": "2",
-        }
+        batch = self._configure_batch()
         parent = f"projects/{self.credential.execution_project}/locations/{self.credential.dataproc_region}"
+
         request = dataproc_v1.CreateBatchRequest(
             parent=parent,
             batch=batch,
         )
         # make the request
         operation = self.job_client.create_batch(request=request)  # type: ignore
-        # this takes quite a while, waiting on GCP response to resolve(not a google-api-core issue, more likely a dataproc serverless issue)
+        # this takes quite a while, waiting on GCP response to resolve
+        # (not a google-api-core issue, more likely a dataproc serverless issue)
         response = operation.result(retry=self.retry)
         return response
         # there might be useful results here that we can parse and return
@@ -147,3 +135,47 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         #     .blob(f"{matches.group(2)}.000000000")
         #     .download_as_string()
         # )
+
+    def _configure_batch(self):
+        # create the Dataproc Serverless job config
+        # need to pin dataproc version to 1.1 as it now defaults to 2.0
+        # https://cloud.google.com/dataproc-serverless/docs/concepts/properties
+        # https://cloud.google.com/dataproc-serverless/docs/reference/rest/v1/projects.locations.batches#runtimeconfig
+        batch = dataproc_v1.Batch(
+            {
+                "runtime_config": dataproc_v1.RuntimeConfig(
+                    version="1.1",
+                    properties={
+                        "spark.executor.instances": "2",
+                    },
+                )
+            }
+        )
+        # Apply defaults
+        batch.pyspark_batch.main_python_file_uri = self.gcs_location
+        jar_file_uri = self.parsed_model["config"].get(
+            "jar_file_uri",
+            "gs://spark-lib/bigquery/spark-bigquery-with-dependencies_2.12-0.21.1.jar",
+        )
+        batch.pyspark_batch.jar_file_uris = [jar_file_uri]
+
+        # Apply configuration from dataproc_batch key, possibly overriding defaults.
+        if self.credential.dataproc_batch:
+            self._update_batch_from_config(self.credential.dataproc_batch, batch)
+        return batch
+
+    @classmethod
+    def _update_batch_from_config(
+        cls, config_dict: Union[Dict, DataprocBatchConfig], target: dataproc_v1.Batch
+    ):
+        try:
+            # updates in place
+            ParseDict(config_dict, target._pb)
+        except Exception as e:
+            docurl = (
+                "https://cloud.google.com/dataproc-serverless/docs/reference/rpc/google.cloud.dataproc.v1"
+                "#google.cloud.dataproc.v1.Batch"
+            )
+            raise ValueError(
+                f"Unable to parse dataproc_batch as valid batch specification. See {docurl}. {str(e)}"
+            ) from e
diff --git a/dbt/include/bigquery/macros/python_model/python.sql b/dbt/include/bigquery/macros/python_model/python.sql
new file mode 100644
index 000000000..adbab752e
--- /dev/null
+++ b/dbt/include/bigquery/macros/python_model/python.sql
@@ -0,0 +1,3 @@
+{% macro bigquery__resolve_model_name(input_model_name) -%}
+    {{ input_model_name | string | replace('`', '') | replace('"', '\"') }}
+{%- endmacro -%}
diff --git a/dev-requirements.txt b/dev-requirements.txt
index e7f7bd2fa..c38fae3f1 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -28,5 +28,6 @@ tox~=3.0;python_version=="3.7"
 tox~=4.4;python_version>="3.8"
 types-pytz~=2022.7
 types-requests~=2.28
+types-protobuf~=4.0
 twine~=4.0
 wheel~=0.38
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 6ac77eee4..1277d64f4 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -108,6 +108,43 @@ def setUp(self):
                     "threads": 1,
                     "location": "Solar Station",
                 },
+                'dataproc-serverless-configured' : {
+                    'type': 'bigquery',
+                    'method': 'oauth',
+                    'schema': 'dummy_schema',
+                    'threads': 1,
+                    'gcs_bucket': 'dummy-bucket',
+                    'dataproc_region': 'europe-west1',
+                    'submission_method': 'serverless',
+                    'dataproc_batch': {
+                        'environment_config' : {
+                            'execution_config' : {
+                                'service_account': 'dbt@dummy-project.iam.gserviceaccount.com',
+                                'subnetwork_uri': 'dataproc',
+                                'network_tags': [ "foo", "bar" ]
+                            }
+                        },
+                        'labels': {
+                            'dbt': 'rocks',
+                            'number': '1'
+                        },
+                        'runtime_config': {
+                            'properties': {
+                                'spark.executor.instances': '4',
+                                'spark.driver.memory': '1g'
+                            }
+                        }
+                    }
+                },
+                'dataproc-serverless-default' : {
+                    'type': 'bigquery',
+                    'method': 'oauth',
+                    'schema': 'dummy_schema',
+                    'threads': 1,
+                    'gcs_bucket': 'dummy-bucket',
+                    'dataproc_region': 'europe-west1',
+                    'submission_method': 'serverless'
+                }
             },
             "target": "oauth",
         }
@@ -184,6 +221,25 @@ def test_acquire_connection_oauth_validations(self, mock_open_connection):
         connection.handle
         mock_open_connection.assert_called_once()
 
+    @patch('dbt.adapters.bigquery.connections.get_bigquery_defaults', return_value=('credentials', 'project_id'))
+    @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn())
+    def test_acquire_connection_dataproc_serverless(self, mock_open_connection, mock_get_bigquery_defaults):
+        adapter = self.get_adapter('dataproc-serverless-configured')
+        mock_get_bigquery_defaults.assert_called_once()
+        try:
+            connection = adapter.acquire_connection('dummy')
+            self.assertEqual(connection.type, 'bigquery')
+
+        except dbt.exceptions.ValidationException as e:
+            self.fail('got ValidationException: {}'.format(str(e)))
+
+        except BaseException as e:
+            raise
+
+        mock_open_connection.assert_not_called()
+        connection.handle
+        mock_open_connection.assert_called_once()
+
     @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn())
     def test_acquire_connection_service_account_validations(self, mock_open_connection):
         adapter = self.get_adapter('service_account')
diff --git a/tests/unit/test_configure_dataproc_batch.py b/tests/unit/test_configure_dataproc_batch.py
new file mode 100644
index 000000000..8222645f4
--- /dev/null
+++ b/tests/unit/test_configure_dataproc_batch.py
@@ -0,0 +1,57 @@
+from unittest.mock import patch
+
+from dbt.adapters.bigquery.python_submissions import ServerlessDataProcHelper
+from google.cloud import dataproc_v1
+
+from .test_bigquery_adapter import BaseTestBigQueryAdapter
+
+# Test application of dataproc_batch configuration to a
+# google.cloud.dataproc_v1.Batch object.
+# This reuses the machinery from BaseTestBigQueryAdapter to get hold of the
+# parsed credentials
+class TestConfigureDataprocBatch(BaseTestBigQueryAdapter):
+
+    @patch('dbt.adapters.bigquery.connections.get_bigquery_defaults', return_value=('credentials', 'project_id'))
+    def test_update_dataproc_serverless_batch(self, mock_get_bigquery_defaults):
+        adapter = self.get_adapter('dataproc-serverless-configured')
+        mock_get_bigquery_defaults.assert_called_once()
+
+        credentials = adapter.acquire_connection('dummy').credentials
+        self.assertIsNotNone(credentials)
+
+        batchConfig = credentials.dataproc_batch
+        self.assertIsNotNone(batchConfig)
+
+        raw_batch_config = self.raw_profile['outputs']['dataproc-serverless-configured']['dataproc_batch']
+        raw_environment_config = raw_batch_config['environment_config']
+        raw_execution_config = raw_environment_config['execution_config']
+        raw_labels: dict[str, any] = raw_batch_config['labels']
+        raw_rt_config = raw_batch_config['runtime_config']
+
+        raw_batch_config = self.raw_profile['outputs']['dataproc-serverless-configured']['dataproc_batch']
+
+        batch = dataproc_v1.Batch()
+
+        ServerlessDataProcHelper._update_batch_from_config(raw_batch_config, batch)
+
+        # google's protobuf types expose maps as dict[str, str]
+        to_str_values = lambda d: dict([(k, str(v)) for (k, v) in d.items()])
+
+        self.assertEqual(batch.environment_config.execution_config.service_account, raw_execution_config['service_account'])
+        self.assertFalse(batch.environment_config.execution_config.network_uri)
+        self.assertEqual(batch.environment_config.execution_config.subnetwork_uri, raw_execution_config['subnetwork_uri'])
+        self.assertEqual(batch.environment_config.execution_config.network_tags, raw_execution_config['network_tags'])
+        self.assertEqual(batch.labels, to_str_values(raw_labels))
+        self.assertEqual(batch.runtime_config.properties, to_str_values(raw_rt_config['properties']))
+
+
+    @patch('dbt.adapters.bigquery.connections.get_bigquery_defaults', return_value=('credentials', 'project_id'))
+    def test_default_dataproc_serverless_batch(self, mock_get_bigquery_defaults):
+        adapter = self.get_adapter('dataproc-serverless-default')
+        mock_get_bigquery_defaults.assert_called_once()
+
+        credentials = adapter.acquire_connection('dummy').credentials
+        self.assertIsNotNone(credentials)
+
+        batchConfig = credentials.dataproc_batch
+        self.assertIsNone(batchConfig)
diff --git a/tox.ini b/tox.ini
index 1721428ee..d08321901 100644
--- a/tox.ini
+++ b/tox.ini
@@ -16,6 +16,19 @@ deps =
 [testenv:{integration,py37,py38,py39,py310,py311,py}-{bigquery}]
 description = adapter plugin integration testing
 skip_install = true
+passenv =
+    DBT_*
+    BIGQUERY_TEST_*
+    PYTEST_ADDOPTS
+commands =
+  bigquery: {envpython} -m pytest {posargs} -vv tests/functional -k "not TestPython" --profile service_account
+deps =
+  -rdev-requirements.txt
+  -e.
+
+[testenv:{python-tests,py37,py38,py39,py310,py311,py}]
+description = python integration testing
+skip_install = true
 passenv =
     DBT_*
     BIGQUERY_TEST_*
@@ -23,7 +36,7 @@ passenv =
     DATAPROC_*
     GCS_BUCKET
 commands =
-  bigquery: {envpython} -m pytest {posargs} -vv tests/functional -k "not TestPython" --profile service_account
+  {envpython} -m pytest {posargs} -vv tests/functional -k "TestPython" --profile service_account
 deps =
   -rdev-requirements.txt
   -e.

From 850296fdeb16cab2692e90478a76e0e6544f42d8 Mon Sep 17 00:00:00 2001
From: Pat Kearns <pat.kearns@dbtlabs.com>
Date: Fri, 10 Mar 2023 09:45:22 +1100
Subject: [PATCH 577/860] Update connections.py with retry logic for 502 (#563)

* Update connections.py with retry logic for 502

* Create Fixes-20230227-110426.yaml

---------

Co-authored-by: Florian Eiden <florian.eiden@fleid.fr>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20230227-110426.yaml | 6 ++++++
 dbt/adapters/bigquery/connections.py           | 1 +
 2 files changed, 7 insertions(+)
 create mode 100644 .changes/unreleased/Fixes-20230227-110426.yaml

diff --git a/.changes/unreleased/Fixes-20230227-110426.yaml b/.changes/unreleased/Fixes-20230227-110426.yaml
new file mode 100644
index 000000000..c23c53ee6
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230227-110426.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Update connections.py with retry logic for 502 BadGateway
+time: 2023-02-27T11:04:26.978726+11:00
+custom:
+  Author: patkearns10
+  Issue: "562"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 7e16f7485..470475d7a 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -53,6 +53,7 @@
 RETRYABLE_ERRORS = (
     google.cloud.exceptions.ServerError,
     google.cloud.exceptions.BadRequest,
+    google.cloud.exceptions.BadGateway,
     ConnectionResetError,
     ConnectionError,
 )

From 384f13caa88ff990878398e26b73f559ae9cb99f Mon Sep 17 00:00:00 2001
From: Neelesh Salian <nssalian@users.noreply.github.com>
Date: Fri, 10 Mar 2023 10:16:05 -0800
Subject: [PATCH 578/860] [ADAP-353]: Fix for #581: Python incremental model
 (#594)

* Fixes for incremental strategy py model.WIP

* doc string

* Remove extra comment

* Uncomment change schema test

* Update dbt/include/bigquery/macros/materializations/incremental.sql

Add python language exception for time_ingestion_partitioning

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>

* Remove tox command flag for test python

* Adding the env vars in integration

* Switch to cluster execution for the python models

* Remove comment and add changie

* Skipping tests since dataproc is unstable. Restoring default to serverless

---------

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20230309-181313.yaml      |  6 ++++++
 .../macros/materializations/incremental.sql         | 13 +++++++++----
 tests/functional/adapter/test_python_model.py       | 11 +++++++----
 tox.ini                                             |  4 +++-
 4 files changed, 25 insertions(+), 9 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230309-181313.yaml

diff --git a/.changes/unreleased/Fixes-20230309-181313.yaml b/.changes/unreleased/Fixes-20230309-181313.yaml
new file mode 100644
index 000000000..8681f5eaf
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230309-181313.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fix for Python incremental model regression
+time: 2023-03-09T18:13:13.512904-08:00
+custom:
+  Author: nssalian
+  Issue: "581"
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index 88fc91eae..9f1479749 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -23,12 +23,17 @@
 
 {% endmacro %}
 {% macro bq_create_table_as(is_time_ingestion_partitioning, temporary, relation, compiled_code, language='sql') %}
-  {% if is_time_ingestion_partitioning %}
+  {% if is_time_ingestion_partitioning and language == 'python' %}
+    {% do exceptions.raise_compiler_error(
+      "Python models do not support ingestion time partitioning"
+    ) %}
+  {% endif %}
+  {% if is_time_ingestion_partitioning and language == 'sql' %}
     {#-- Create the table before inserting data as ingestion time partitioned tables can't be created with the transformed data --#}
-    {% do run_query(create_ingestion_time_partitioned_table_as_sql(temporary, relation, sql)) %}
-    {{ return(bq_insert_into_ingestion_time_partitioned_table_sql(relation, sql)) }}
+    {% do run_query(create_ingestion_time_partitioned_table_as_sql(temporary, relation, compiled_code)) %}
+    {{ return(bq_insert_into_ingestion_time_partitioned_table_sql(relation, compiled_code)) }}
   {% else %}
-    {{ return(create_table_as(temporary, relation, sql)) }}
+    {{ return(create_table_as(temporary, relation, compiled_code, language)) }}
   {% endif %}
 {% endmacro %}
 
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
index a4b4d90c7..734fe67d5 100644
--- a/tests/functional/adapter/test_python_model.py
+++ b/tests/functional/adapter/test_python_model.py
@@ -3,13 +3,16 @@
 from dbt.tests.util import run_dbt, write_file
 import dbt.tests.adapter.python_model.test_python_model as dbt_tests
 
-# ToDo: Fix and schedule these tests:
-# https://github.com/dbt-labs/dbt-bigquery/issues/306
+TEST_SKIP_MESSAGE = "Skipping the Tests since Dataproc serverless is not stable. " \
+                    "TODO: Fix later"
+
+
+@pytest.mark.skip(reason=TEST_SKIP_MESSAGE)
 class TestPythonModelDataproc(dbt_tests.BasePythonModelTests):
     pass
 
 
-@pytest.mark.skip(reason="Currently Broken")
+@pytest.mark.skip(reason=TEST_SKIP_MESSAGE)
 class TestPythonIncrementalMatsDataproc(dbt_tests.BasePythonIncrementalTests):
     pass
 
@@ -37,7 +40,7 @@ def model(dbt, spark):
 """
 
 
-@pytest.mark.skip(reason="Currently Broken")
+@pytest.mark.skip(reason=TEST_SKIP_MESSAGE)
 class TestChangingSchemaDataproc:
 
     @pytest.fixture(scope="class")
diff --git a/tox.ini b/tox.ini
index d08321901..4d552ab44 100644
--- a/tox.ini
+++ b/tox.ini
@@ -20,8 +20,10 @@ passenv =
     DBT_*
     BIGQUERY_TEST_*
     PYTEST_ADDOPTS
+    DATAPROC_*
+    GCS_BUCKET
 commands =
-  bigquery: {envpython} -m pytest {posargs} -vv tests/functional -k "not TestPython" --profile service_account
+  bigquery: {envpython} -m pytest {posargs} -vv tests/functional --profile service_account
 deps =
   -rdev-requirements.txt
   -e.

From 463a292f9e77fe36ee410fd5a2029e9c9b42070d Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Mon, 13 Mar 2023 12:38:07 -0600
Subject: [PATCH 579/860] add triage-labels workflow (#601)

* add triage-labels workflow

* fix end of files
---
 .github/workflows/triage-labels.yml | 33 +++++++++++++++++++++++++++++
 1 file changed, 33 insertions(+)
 create mode 100644 .github/workflows/triage-labels.yml

diff --git a/.github/workflows/triage-labels.yml b/.github/workflows/triage-labels.yml
new file mode 100644
index 000000000..a71dc5e1f
--- /dev/null
+++ b/.github/workflows/triage-labels.yml
@@ -0,0 +1,33 @@
+# **what?**
+# When the core team triages, we sometimes need more information from the issue creator.  In
+# those cases we remove the `triage` label and add the `awaiting_response` label.  Once we
+# recieve a response in the form of a comment, we want the `awaiting_response` label removed
+# in favor of the `triage` label so we are aware that the issue needs action.
+
+# **why?**
+# To help with out team triage issue tracking
+
+# **when?**
+# This will run when a comment is added to an issue and that issue has to `awaiting_response` label.
+
+name: Update Triage Label
+
+on: issue_comment
+
+defaults:
+  run:
+    shell: bash
+
+permissions:
+  issues: write
+
+jobs:
+  triage_label:
+    if: contains(github.event.issue.labels.*.name, 'awaiting_response')
+    runs-on: ubuntu-latest
+    steps:
+      - name: initial labeling
+        uses: andymckay/labeler@master
+        with:
+          add-labels: "triage"
+          remove-labels: "awaiting_response"

From 3eedb617b9ebfa192a5bfa81cadc51822a897ca2 Mon Sep 17 00:00:00 2001
From: garsir <125480552+garsir@users.noreply.github.com>
Date: Tue, 14 Mar 2023 16:28:31 +0000
Subject: [PATCH 580/860] Allow column policy_tags to be removed by removing
 them from model (#542)

* Add new test to show deletion of policy_tags

* Allow policy_tags to be deleted

* Add changelog entry

---------

Co-authored-by: Florian Eiden <florian.eiden@fleid.fr>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .../unreleased/Fixes-20230216-140756.yaml     |  6 ++
 dbt/adapters/bigquery/impl.py                 |  3 +-
 tests/functional/test_delete_column_policy.py | 74 +++++++++++++++++++
 3 files changed, 81 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230216-140756.yaml
 create mode 100644 tests/functional/test_delete_column_policy.py

diff --git a/.changes/unreleased/Fixes-20230216-140756.yaml b/.changes/unreleased/Fixes-20230216-140756.yaml
new file mode 100644
index 000000000..b273c563e
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230216-140756.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Allow policy_tags to be removed
+time: 2023-02-16T14:07:56.313767Z
+custom:
+  Author: garsir
+  Issue: "349"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index af59e2666..c9d08f8d3 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -644,8 +644,7 @@ def _update_column_dict(self, bq_column_dict, dbt_columns, parent=""):
         if dotted_column_name in dbt_columns:
             column_config = dbt_columns[dotted_column_name]
             bq_column_dict["description"] = column_config.get("description")
-            if column_config.get("policy_tags"):
-                bq_column_dict["policyTags"] = {"names": column_config.get("policy_tags")}
+            bq_column_dict["policyTags"] = {"names": column_config.get("policy_tags", list())}
 
         new_fields = []
         for child_col_dict in bq_column_dict.get("fields", list()):
diff --git a/tests/functional/test_delete_column_policy.py b/tests/functional/test_delete_column_policy.py
new file mode 100644
index 000000000..9ba2353a3
--- /dev/null
+++ b/tests/functional/test_delete_column_policy.py
@@ -0,0 +1,74 @@
+import pytest
+from dbt.tests.util import (
+    run_dbt, get_connection, relation_from_name, write_config_file
+)
+
+from dbt.adapters.bigquery import BigQueryRelation
+
+_POLICY_TAG_MODEL = """{{
+  config(
+    materialized='table',
+    persist_docs={ 'columns': true }
+  )
+}}
+
+select
+  1 field
+"""
+
+_POLICY_TAG_YML = """version: 2
+
+models:
+- name: policy_tag_table
+  columns:
+  - name: field
+    policy_tags:
+      - '{{ var("policy_tag") }}'
+"""
+
+_POLICY_TAG_YML_NO_POLICY_TAGS = """version: 2
+
+models:
+- name: policy_tag_table
+  columns:
+  - name: field
+"""
+
+# Manually generated https://console.cloud.google.com/bigquery/policy-tags?project=dbt-test-env
+_POLICY_TAG = "projects/dbt-test-env/locations/us/taxonomies/5785568062805976401/policyTags/135489647357012267"
+_POLICY_TAG_MODEL_NAME = "policy_tag_table"
+
+
+class TestBigqueryDeleteColumnPolicy:
+    """See BQ docs for more info on policy tags:
+    https://cloud.google.com/bigquery/docs/column-level-security#work_with_policy_tags"""
+
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            'config-version': 2,
+            'vars': {
+                'policy_tag': _POLICY_TAG
+            }
+        }
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            f"{_POLICY_TAG_MODEL_NAME}.sql": _POLICY_TAG_MODEL,
+            "schema.yml": _POLICY_TAG_YML
+        }
+
+    def test_bigquery_delete_column_policy_tag(self, project):
+        results = run_dbt(['run', '-f', '--models', 'policy_tag_table'])
+        assert len(results) == 1
+        write_config_file(_POLICY_TAG_YML_NO_POLICY_TAGS, project.project_root + '/models', "schema.yml") #update the model to remove the policy tag
+        new_results = run_dbt(['run', '-f', '--models', 'policy_tag_table'])
+        assert len(new_results) == 1
+        relation: BigQueryRelation = relation_from_name(project.adapter, _POLICY_TAG_MODEL_NAME)
+        adapter = project.adapter
+        with get_connection(project.adapter) as conn:
+            table = conn.handle.get_table(
+                adapter.connections.get_bq_table(relation.database, relation.schema, relation.table))
+            for schema_field in table.schema:
+                assert schema_field.policy_tags is None

From 88b94b699e2e2ff3f0af83807d5d4bf319f7c1f8 Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Thu, 16 Mar 2023 12:02:18 -0700
Subject: [PATCH 581/860] Remove checks on non-leaf-field tags since they no
 longer can exist. (#607)

* Remove checks on non-leaf-field tags since they no longer can exist.

* add changelog

* remove dead code

* Document what's happening.

---------

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .../unreleased/Fixes-20230315-130504.yaml     |  6 +++++
 tests/functional/adapter/test_persist_docs.py | 22 ++-----------------
 2 files changed, 8 insertions(+), 20 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230315-130504.yaml

diff --git a/.changes/unreleased/Fixes-20230315-130504.yaml b/.changes/unreleased/Fixes-20230315-130504.yaml
new file mode 100644
index 000000000..5229dca17
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230315-130504.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fix failing test by removing no erroneous asserts.
+time: 2023-03-15T13:05:04.747323-07:00
+custom:
+  Author: versusfacit
+  Issue: "605"
diff --git a/tests/functional/adapter/test_persist_docs.py b/tests/functional/adapter/test_persist_docs.py
index 169d1074e..7717b190b 100644
--- a/tests/functional/adapter/test_persist_docs.py
+++ b/tests/functional/adapter/test_persist_docs.py
@@ -41,18 +41,10 @@
 models:
   - name: table_model_nested
     columns:
-      - name: level_1
-        description: level_1 column description
-      - name: level_1.level_2
-        description: level_2 column description
       - name: level_1.level_2.level_3_a
         description: level_3 column description
   - name: view_model_nested
     columns:
-      - name: level_1
-        description: level_1 column description
-      - name: level_1.level_2
-        description: level_2 column description
       - name: level_1.level_2.level_3_a
         description: level_3 column description
 """
@@ -177,6 +169,8 @@ def test_persist_docs(self, project):
         colunmn descriptions are persisted on the test model table and view.
 
         Next, generate the catalog and check if the comments are also included.
+
+        Note: dbt-bigquery does not allow comments on models with children nodes
         """
         run_dbt(['seed'])
         run_dbt()
@@ -201,13 +195,7 @@ def test_persist_docs(self, project):
                 bq_schema = client.get_table(table_id).schema
 
                 level_1_field = bq_schema[0]
-                assert level_1_field.description == \
-                       "level_1 column description"
-
                 level_2_field = level_1_field.fields[0]
-                assert level_2_field.description == \
-                       "level_2 column description"
-
                 level_3_field = level_2_field.fields[0]
                 assert level_3_field.description == \
                        "level_3 column description"
@@ -215,11 +203,5 @@ def test_persist_docs(self, project):
             # check the descriptions in the catalog
             node = catalog_data['nodes']['model.test.{}'.format(node_id)]
 
-            level_1_column = node['columns']['level_1']
-            assert level_1_column['comment'] == "level_1 column description"
-
-            level_2_column = node['columns']['level_1.level_2']
-            assert level_2_column['comment'] == "level_2 column description"
-
             level_3_column = node['columns']['level_1.level_2.level_3_a']
             assert level_3_column['comment'] == "level_3 column description"

From 8bcc44302680035a903c25ee478c36655843c28c Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Thu, 16 Mar 2023 19:53:51 +0000
Subject: [PATCH 582/860] Bumping version to 1.5.0b3 and generate changelog

---
 .bumpversion.cfg                              |  2 +-
 .changes/1.5.0-b3.md                          | 20 ++++++++++++++++
 .../Features-20230223-145508.yaml             |  0
 .../Features-20230303-132509.yaml             |  0
 .../Fixes-20230202-010332.yaml                |  0
 .../Fixes-20230216-140756.yaml                |  0
 .../Fixes-20230227-110426.yaml                |  0
 .../Fixes-20230309-181313.yaml                |  0
 .../Fixes-20230315-130504.yaml                |  0
 CHANGELOG.md                                  | 24 +++++++++++++++++--
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 12 files changed, 45 insertions(+), 5 deletions(-)
 create mode 100644 .changes/1.5.0-b3.md
 rename .changes/{unreleased => 1.5.0}/Features-20230223-145508.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Features-20230303-132509.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Fixes-20230202-010332.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Fixes-20230216-140756.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Fixes-20230227-110426.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Fixes-20230309-181313.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Fixes-20230315-130504.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 8d5440a31..fadf60275 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.5.0b2
+current_version = 1.5.0b3
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.5.0-b3.md b/.changes/1.5.0-b3.md
new file mode 100644
index 000000000..f2415ad97
--- /dev/null
+++ b/.changes/1.5.0-b3.md
@@ -0,0 +1,20 @@
+## dbt-bigquery 1.5.0-b3 - March 16, 2023
+
+### Features
+
+- Enforce contracts on models materialized as tables and views ([#529](https://github.com/dbt-labs/dbt-bigquery/issues/529), [#555](https://github.com/dbt-labs/dbt-bigquery/issues/555))
+- add dataproc serverless config to profile ([#530](https://github.com/dbt-labs/dbt-bigquery/issues/530))
+
+### Fixes
+
+- Fix time ingestion partitioning option regression when combined using `require_partition_filter` option on incremental run ([#483](https://github.com/dbt-labs/dbt-bigquery/issues/483))
+- Allow policy_tags to be removed ([#349](https://github.com/dbt-labs/dbt-bigquery/issues/349))
+- Update connections.py with retry logic for 502 BadGateway ([#562](https://github.com/dbt-labs/dbt-bigquery/issues/562))
+- Fix for Python incremental model regression ([#581](https://github.com/dbt-labs/dbt-bigquery/issues/581))
+- Fix failing test by removing no erroneous asserts. ([#605](https://github.com/dbt-labs/dbt-bigquery/issues/605))
+
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#483](https://github.com/dbt-labs/dbt-bigquery/issues/483))
+- [@garsir](https://github.com/garsir) ([#349](https://github.com/dbt-labs/dbt-bigquery/issues/349))
+- [@patkearns10](https://github.com/patkearns10) ([#562](https://github.com/dbt-labs/dbt-bigquery/issues/562))
+- [@torkjel](https://github.com/torkjel) ([#530](https://github.com/dbt-labs/dbt-bigquery/issues/530))
diff --git a/.changes/unreleased/Features-20230223-145508.yaml b/.changes/1.5.0/Features-20230223-145508.yaml
similarity index 100%
rename from .changes/unreleased/Features-20230223-145508.yaml
rename to .changes/1.5.0/Features-20230223-145508.yaml
diff --git a/.changes/unreleased/Features-20230303-132509.yaml b/.changes/1.5.0/Features-20230303-132509.yaml
similarity index 100%
rename from .changes/unreleased/Features-20230303-132509.yaml
rename to .changes/1.5.0/Features-20230303-132509.yaml
diff --git a/.changes/unreleased/Fixes-20230202-010332.yaml b/.changes/1.5.0/Fixes-20230202-010332.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230202-010332.yaml
rename to .changes/1.5.0/Fixes-20230202-010332.yaml
diff --git a/.changes/unreleased/Fixes-20230216-140756.yaml b/.changes/1.5.0/Fixes-20230216-140756.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230216-140756.yaml
rename to .changes/1.5.0/Fixes-20230216-140756.yaml
diff --git a/.changes/unreleased/Fixes-20230227-110426.yaml b/.changes/1.5.0/Fixes-20230227-110426.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230227-110426.yaml
rename to .changes/1.5.0/Fixes-20230227-110426.yaml
diff --git a/.changes/unreleased/Fixes-20230309-181313.yaml b/.changes/1.5.0/Fixes-20230309-181313.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230309-181313.yaml
rename to .changes/1.5.0/Fixes-20230309-181313.yaml
diff --git a/.changes/unreleased/Fixes-20230315-130504.yaml b/.changes/1.5.0/Fixes-20230315-130504.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230315-130504.yaml
rename to .changes/1.5.0/Fixes-20230315-130504.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 48ed63e9d..a71fcd8bf 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,14 +5,34 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.5.0-b3 - March 16, 2023
+
+### Features
+
+- Enforce contracts on models materialized as tables and views ([#529](https://github.com/dbt-labs/dbt-bigquery/issues/529), [#555](https://github.com/dbt-labs/dbt-bigquery/issues/555))
+- add dataproc serverless config to profile ([#530](https://github.com/dbt-labs/dbt-bigquery/issues/530))
+
+### Fixes
+
+- Fix time ingestion partitioning option regression when combined using `require_partition_filter` option on incremental run ([#483](https://github.com/dbt-labs/dbt-bigquery/issues/483))
+- Allow policy_tags to be removed ([#349](https://github.com/dbt-labs/dbt-bigquery/issues/349))
+- Update connections.py with retry logic for 502 BadGateway ([#562](https://github.com/dbt-labs/dbt-bigquery/issues/562))
+- Fix for Python incremental model regression ([#581](https://github.com/dbt-labs/dbt-bigquery/issues/581))
+- Fix failing test by removing no erroneous asserts. ([#605](https://github.com/dbt-labs/dbt-bigquery/issues/605))
+
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#483](https://github.com/dbt-labs/dbt-bigquery/issues/483))
+- [@garsir](https://github.com/garsir) ([#349](https://github.com/dbt-labs/dbt-bigquery/issues/349))
+- [@patkearns10](https://github.com/patkearns10) ([#562](https://github.com/dbt-labs/dbt-bigquery/issues/562))
+- [@torkjel](https://github.com/torkjel) ([#530](https://github.com/dbt-labs/dbt-bigquery/issues/530))
+
+
 ## dbt-bigquery 1.5.0-b2 - March 02, 2023
 
 ### Under the Hood
 
 - Rename constraints_enabled to contract ([#548](https://github.com/dbt-labs/dbt-bigquery/issues/548))
 
-
-
 ## dbt-bigquery 1.5.0-b1 - February 22, 2023
 
 ### Features
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 4f8b15313..649c005ac 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.5.0b2"
+version = "1.5.0b3"
diff --git a/setup.py b/setup.py
index 7603a3498..3186aa435 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.5.0b2"
+package_version = "1.5.0b3"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From f6fc13fd1e97be0f82d0838c42d9ff9f03c2c57f Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Sun, 19 Mar 2023 15:56:48 -0400
Subject: [PATCH 583/860] Update wheel requirement from ~=0.38 to ~=0.40 (#604)

Updates the requirements on [wheel](https://github.com/pypa/wheel) to permit the latest version.
- [Release notes](https://github.com/pypa/wheel/releases)
- [Changelog](https://github.com/pypa/wheel/blob/main/docs/news.rst)
- [Commits](https://github.com/pypa/wheel/compare/0.38.0...0.40.0)

---
updated-dependencies:
- dependency-name: wheel
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index c38fae3f1..fde7a7459 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -30,4 +30,4 @@ types-pytz~=2022.7
 types-requests~=2.28
 types-protobuf~=4.0
 twine~=4.0
-wheel~=0.38
+wheel~=0.40

From 43842bf291bf34b2c6be34b250295ddb03bb7858 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Sun, 19 Mar 2023 16:47:13 -0400
Subject: [PATCH 584/860] Update ipdb requirement from ~=0.13.11 to ~=0.13.13
 (#596)

Updates the requirements on [ipdb](https://github.com/gotcha/ipdb) to permit the latest version.
- [Release notes](https://github.com/gotcha/ipdb/releases)
- [Changelog](https://github.com/gotcha/ipdb/blob/master/HISTORY.txt)
- [Commits](https://github.com/gotcha/ipdb/compare/0.13.11...0.13.13)

---
updated-dependencies:
- dependency-name: ipdb
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index fde7a7459..f9a386c30 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -12,7 +12,7 @@ flake8~=5.0;python_version=="3.7"
 flake8~=6.0;python_version>="3.8"
 flaky~=3.7
 freezegun~=1.2
-ipdb~=0.13.11
+ipdb~=0.13.13
 mypy==1.0.1  # patch updates have historically introduced breaking changes
 pip-tools~=6.12
 pre-commit~=2.21;python_version=="3.7"

From 1cb3c7c97da26470ddb89ec9ea845d9c3c97d3eb Mon Sep 17 00:00:00 2001
From: "Bruno S. de Lima" <bruno.szdl@gmail.com>
Date: Mon, 20 Mar 2023 15:38:18 -0300
Subject: [PATCH 585/860] Add `bytes_billed` to `BigQueryAdapterResponse`
 (#570)

* adding bytes_billed to adapter response

* adding changie

---------

Co-authored-by: Florian Eiden <florian.eiden@fleid.fr>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .changes/unreleased/Features-20230228-094234.yaml | 6 ++++++
 dbt/adapters/bigquery/connections.py              | 4 ++++
 2 files changed, 10 insertions(+)
 create mode 100644 .changes/unreleased/Features-20230228-094234.yaml

diff --git a/.changes/unreleased/Features-20230228-094234.yaml b/.changes/unreleased/Features-20230228-094234.yaml
new file mode 100644
index 000000000..4929c0834
--- /dev/null
+++ b/.changes/unreleased/Features-20230228-094234.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: 'Adding `bytes_billed` to `BigQueryAdapterResponse` '
+time: 2023-02-28T09:42:34.557696-03:00
+custom:
+  Author: bruno-szdl
+  Issue: "560"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 470475d7a..a561eb5b9 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -89,6 +89,7 @@ class BigQueryConnectionMethod(StrEnum):
 @dataclass
 class BigQueryAdapterResponse(AdapterResponse):
     bytes_processed: Optional[int] = None
+    bytes_billed: Optional[int] = None
     location: Optional[str] = None
     project_id: Optional[str] = None
     job_id: Optional[str] = None
@@ -471,6 +472,7 @@ def execute(
         code = None
         num_rows = None
         bytes_processed = None
+        bytes_billed = None
         location = None
         job_id = None
         project_id = None
@@ -505,6 +507,7 @@ def execute(
 
         # set common attributes
         bytes_processed = query_job.total_bytes_processed
+        bytes_billed = query_job.total_bytes_billed
         slot_ms = query_job.slot_millis
         processed_bytes = self.format_bytes(bytes_processed)
         location = query_job.location
@@ -526,6 +529,7 @@ def execute(
             rows_affected=num_rows,
             code=code,
             bytes_processed=bytes_processed,
+            bytes_billed=bytes_billed,
             location=location,
             project_id=project_id,
             job_id=job_id,

From ea0833424c6445cff853e82ab9a5d563a55d3856 Mon Sep 17 00:00:00 2001
From: Gerda Shank <gerda@dbtlabs.com>
Date: Mon, 20 Mar 2023 15:51:31 -0400
Subject: [PATCH 586/860] Use contract column order in create table (#606)

---
 .changes/unreleased/Features-20230315-120554.yaml |  6 ++++++
 dbt/include/bigquery/macros/adapters.sql          |  3 ++-
 tests/functional/adapter/test_constraints.py      | 14 +++++++++-----
 3 files changed, 17 insertions(+), 6 deletions(-)
 create mode 100644 .changes/unreleased/Features-20230315-120554.yaml

diff --git a/.changes/unreleased/Features-20230315-120554.yaml b/.changes/unreleased/Features-20230315-120554.yaml
new file mode 100644
index 000000000..65718d06b
--- /dev/null
+++ b/.changes/unreleased/Features-20230315-120554.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Modify create_table_as to use contract column order
+time: 2023-03-15T12:05:54.52431-04:00
+custom:
+  Author: gshank
+  Issue: "579"
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index 59785db23..4c9f00dbf 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -53,8 +53,9 @@
 
     create or replace table {{ relation }}
       {% if config.get('contract', False) %}
-        {{ get_assert_columns_equivalent(sql) }}
+        {{ get_assert_columns_equivalent(compiled_code) }}
         {{ get_columns_spec_ddl() }}
+        {%- set compiled_code = get_select_subquery(compiled_code) %}
       {% endif %}
     {{ partition_by(partition_config) }}
     {{ cluster_by(raw_cluster_by) }}
diff --git a/tests/functional/adapter/test_constraints.py b/tests/functional/adapter/test_constraints.py
index fb15c6a9d..198debfd7 100644
--- a/tests/functional/adapter/test_constraints.py
+++ b/tests/functional/adapter/test_constraints.py
@@ -18,14 +18,18 @@
 create or replace table {0} (
     id integer  not null    ,
     color string  ,
-    date_day date
+    date_day string
 )
 OPTIONS()
 as (
-    select
-        1 as id,
-        'blue' as color,
-        cast('2019-01-01' as date) as date_day
+    select id,
+    color, 
+    date_day from 
+  ( 
+    select 'blue' as color, 
+    1 as id, 
+    '2019-01-01' as date_day
+  ) as model_subq
 );
 """
 

From 9380601a2f0379c6a877225be73612f981539e8b Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 21 Mar 2023 00:16:33 -0700
Subject: [PATCH 587/860] Bump mypy from 1.0.1 to 1.1.1 (#592)

* Bump mypy from 1.0.1 to 1.1.1

Bumps [mypy](https://github.com/python/mypy) from 1.0.1 to 1.1.1.
- [Release notes](https://github.com/python/mypy/releases)
- [Commits](https://github.com/python/mypy/compare/v1.0.1...v1.1.1)

---
updated-dependencies:
- dependency-name: mypy
  dependency-type: direct:development
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <support@github.com>

* update pre commit config

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .pre-commit-config.yaml | 2 +-
 dev-requirements.txt    | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index 7867b440f..b721d3ab7 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -41,7 +41,7 @@ repos:
     alias: flake8-check
     stages: [manual]
 - repo: https://github.com/pre-commit/mirrors-mypy
-  rev: v1.0.1
+  rev: v1.1.1
   hooks:
   - id: mypy
     # N.B.: Mypy is... a bit fragile.
diff --git a/dev-requirements.txt b/dev-requirements.txt
index f9a386c30..7f1eb9a67 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -13,7 +13,7 @@ flake8~=6.0;python_version>="3.8"
 flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
-mypy==1.0.1  # patch updates have historically introduced breaking changes
+mypy==1.1.1  # patch updates have historically introduced breaking changes
 pip-tools~=6.12
 pre-commit~=2.21;python_version=="3.7"
 pre-commit~=3.1;python_version>="3.8"

From a6f0d3e2a21f346bcaf92264ff08890a8ea1d458 Mon Sep 17 00:00:00 2001
From: Peter Webb <peter.webb@dbtlabs.com>
Date: Wed, 22 Mar 2023 13:29:02 -0400
Subject: [PATCH 588/860] Modify adapter to support unified constraint fields
 (#603)

* CT-2213: Modify adapter to support unified constraint fields

* CT-2213: Remove leaked (and now deactivated) credentials

* CT-2213: Add temporary branch pin

* CT-2213: Modify constraint generation to use adapter method

* CT-2213: Revert requirements in preperation for merge

---------

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .../unreleased/Features-20230314-171221.yaml  |  6 ++++
 .../macros/utils/get_columns_spec_ddl.sql     | 29 +++++++++++++------
 tests/functional/adapter/test_constraints.py  |  4 +--
 3 files changed, 28 insertions(+), 11 deletions(-)
 create mode 100644 .changes/unreleased/Features-20230314-171221.yaml

diff --git a/.changes/unreleased/Features-20230314-171221.yaml b/.changes/unreleased/Features-20230314-171221.yaml
new file mode 100644
index 000000000..067812f09
--- /dev/null
+++ b/.changes/unreleased/Features-20230314-171221.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Modify addapter to support unified constraint fields
+time: 2023-03-14T17:12:21.287702-04:00
+custom:
+  Author: peterallenwebb
+  Issue: "567"
diff --git a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
index f074f1bee..091209ae3 100644
--- a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
+++ b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
@@ -1,18 +1,29 @@
 {% macro bigquery__get_columns_spec_ddl() %}
   {# loop through user_provided_columns to create DDL with data types and constraints #}
-    {%- set ns = namespace(at_least_one_check=False) -%}
+    {%- set ns = namespace(at_least_one_check=False, at_least_one_pk=False) -%}
     {%- set user_provided_columns = model['columns'] -%}
     (
-    {% for i in user_provided_columns -%}
+    {% for i in user_provided_columns %}
       {%- set col = user_provided_columns[i] -%}
-      {% set constraints = col['constraints'] -%}
-      {%- set ns.at_least_one_check = ns.at_least_one_check or col['constraints_check'] %}
-      {{ col['name'] }} {{ col['data_type'] }} {% for x in constraints %} {{ x or "" }} {% endfor %} {{ "," if not loop.last }}
-    {%- endfor %}
-  )
+      {%- set constraints = col['constraints'] -%}
+      {{ col['name'] }} {{ col['data_type'] }}
+      {%- for c in constraints -%}
+        {%- if c.type == "check" -%}
+          {%- set ns.at_least_one_check = True -%}
+        {%- elif c.type == "primary_key" -%}
+          {%- set ns.at_least_one_pk = True -%}
+        {%- else %} {{ adapter.render_raw_column_constraint(c) }}
+        {%- endif -%}
+      {%- endfor -%}
+      {{ "," if not loop.last }}
+    {% endfor -%}
+    )
   {%- if ns.at_least_one_check -%}
-      {{exceptions.warn("We noticed you have `constraints_check` configs, these are NOT compatible with BigQuery and will be ignored")}}
-  {%- endif %}
+      {{exceptions.warn("We noticed you have check constraints in your configs. These are not compatible with BigQuery and will be ignored.")}}
+  {%- endif -%}
+  {%- if ns.at_least_one_pk -%}
+    {{exceptions.warn("We noticed you have primary key constraints in your configs. These are not compatible with BigQuery and will be ignored.")}}
+  {%- endif -%}
 {% endmacro %}
 
 {% macro bigquery__format_column(column) -%}
diff --git a/tests/functional/adapter/test_constraints.py b/tests/functional/adapter/test_constraints.py
index 198debfd7..ae3d874d2 100644
--- a/tests/functional/adapter/test_constraints.py
+++ b/tests/functional/adapter/test_constraints.py
@@ -16,8 +16,8 @@
 
 _expected_sql_bigquery = """
 create or replace table {0} (
-    id integer  not null    ,
-    color string  ,
+    id integer not null,
+    color string,
     date_day string
 )
 OPTIONS()

From bf6da74606c391e3f7252fa8af856422f8080f6a Mon Sep 17 00:00:00 2001
From: Gerda Shank <gerda@dbtlabs.com>
Date: Wed, 22 Mar 2023 16:53:49 -0400
Subject: [PATCH 589/860] Use _make_ref_key_dict instead of _make_ref_key_msg
 (#622)

---
 .changes/unreleased/Fixes-20230322-162200.yaml | 6 ++++++
 dbt/adapters/bigquery/impl.py                  | 6 +++---
 2 files changed, 9 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230322-162200.yaml

diff --git a/.changes/unreleased/Fixes-20230322-162200.yaml b/.changes/unreleased/Fixes-20230322-162200.yaml
new file mode 100644
index 000000000..a80a3f963
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230322-162200.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Use _make_ref_key_dict instead of _make_ref_key_msg
+time: 2023-03-22T16:22:00.091222-04:00
+custom:
+  Author: gshank
+  Issue: "621"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index c9d08f8d3..158cbc151 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -18,7 +18,7 @@
     PythonJobHelper,
 )
 
-from dbt.adapters.cache import _make_ref_key_msg
+from dbt.adapters.cache import _make_ref_key_dict
 
 from dbt.adapters.bigquery.relation import BigQueryRelation
 from dbt.adapters.bigquery.dataset import add_access_entry_to_dataset
@@ -332,7 +332,7 @@ def create_schema(self, relation: BigQueryRelation) -> None:
         # use SQL 'create schema'
         relation = relation.without_identifier()  # type: ignore
 
-        fire_event(SchemaCreation(relation=_make_ref_key_msg(relation)))
+        fire_event(SchemaCreation(relation=_make_ref_key_dict(relation)))
         kwargs = {
             "relation": relation,
         }
@@ -346,7 +346,7 @@ def drop_schema(self, relation: BigQueryRelation) -> None:
         database = relation.database
         schema = relation.schema
         logger.debug('Dropping schema "{}.{}".', database, schema)  # in lieu of SQL
-        fire_event(SchemaDrop(relation=_make_ref_key_msg(relation)))
+        fire_event(SchemaDrop(relation=_make_ref_key_dict(relation)))
         self.connections.drop_dataset(database, schema)
         self.cache.drop_schema(database, schema)
 

From 43e2883c379daa10734e4837855f4e75829d2a06 Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Thu, 23 Mar 2023 11:55:28 -0700
Subject: [PATCH 590/860] Adap 13/simple copy test conversion (#620)

* convert test.

* Finish bigquery conversion.

* Nix unneeded test. Model agnostic

* Revert requirement file

---------

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .../adapter/simple_copy/fixtures.py           | 534 ++++++++++++++++++
 .../adapter/simple_copy/test_simple_copy.py   |  51 ++
 .../incremental_update_cols.sql               |  17 -
 .../models/advanced_incremental.sql           |  17 -
 .../simple_copy_test/models/compound_sort.sql |   9 -
 .../simple_copy_test/models/disabled.sql      |   8 -
 .../simple_copy_test/models/empty.sql         |   5 -
 .../simple_copy_test/models/get_and_ref.sql   |   3 -
 .../simple_copy_test/models/incremental.sql   |  11 -
 .../models/interleaved_sort.sql               |   9 -
 .../simple_copy_test/models/materialized.sql  |  12 -
 .../simple_copy_test/models/schema.yml        |   7 -
 .../simple_copy_test/models/view_model.sql    |   7 -
 .../simple_copy_test/seed-initial/seed.csv    | 101 ----
 .../simple_copy_test/seed-update/seed.csv     | 201 -------
 .../seeds-merge-cols-initial/seed.csv         | 101 ----
 .../expected_result.csv                       | 201 -------
 .../seeds-merge-cols-update/seed.csv          | 204 -------
 .../simple_copy_test/test_simple_copy.py      |  96 ----
 .../models-bq/statement_actual.sql            |  23 -
 .../integration/statement_test/seed/seed.csv  | 101 ----
 .../seed/statement_expected.csv               |   3 -
 .../statement_test/test_statements.py         |  36 --
 23 files changed, 585 insertions(+), 1172 deletions(-)
 create mode 100644 tests/functional/adapter/simple_copy/fixtures.py
 create mode 100644 tests/functional/adapter/simple_copy/test_simple_copy.py
 delete mode 100644 tests/integration/simple_copy_test/models-merge-update/incremental_update_cols.sql
 delete mode 100644 tests/integration/simple_copy_test/models/advanced_incremental.sql
 delete mode 100644 tests/integration/simple_copy_test/models/compound_sort.sql
 delete mode 100644 tests/integration/simple_copy_test/models/disabled.sql
 delete mode 100644 tests/integration/simple_copy_test/models/empty.sql
 delete mode 100644 tests/integration/simple_copy_test/models/get_and_ref.sql
 delete mode 100644 tests/integration/simple_copy_test/models/incremental.sql
 delete mode 100644 tests/integration/simple_copy_test/models/interleaved_sort.sql
 delete mode 100644 tests/integration/simple_copy_test/models/materialized.sql
 delete mode 100644 tests/integration/simple_copy_test/models/schema.yml
 delete mode 100644 tests/integration/simple_copy_test/models/view_model.sql
 delete mode 100644 tests/integration/simple_copy_test/seed-initial/seed.csv
 delete mode 100644 tests/integration/simple_copy_test/seed-update/seed.csv
 delete mode 100644 tests/integration/simple_copy_test/seeds-merge-cols-initial/seed.csv
 delete mode 100644 tests/integration/simple_copy_test/seeds-merge-cols-update/expected_result.csv
 delete mode 100644 tests/integration/simple_copy_test/seeds-merge-cols-update/seed.csv
 delete mode 100644 tests/integration/simple_copy_test/test_simple_copy.py
 delete mode 100644 tests/integration/statement_test/models-bq/statement_actual.sql
 delete mode 100644 tests/integration/statement_test/seed/seed.csv
 delete mode 100644 tests/integration/statement_test/seed/statement_expected.csv
 delete mode 100644 tests/integration/statement_test/test_statements.py

diff --git a/tests/functional/adapter/simple_copy/fixtures.py b/tests/functional/adapter/simple_copy/fixtures.py
new file mode 100644
index 000000000..d53b368b2
--- /dev/null
+++ b/tests/functional/adapter/simple_copy/fixtures.py
@@ -0,0 +1,534 @@
+_SEEDS__SEED_MERGE_COLS_INITIAL = """
+load_date,id,first_name,last_name,email,gender,ip_address
+2021-03-05,1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
+2021-03-05,2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
+2021-03-05,3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
+2021-03-05,4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
+2021-03-05,5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
+2021-03-05,6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220
+2021-03-05,7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64
+2021-03-05,8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13
+2021-03-05,9,Gary,Day,gday8@nih.gov,Male,35.81.68.186
+2021-03-05,10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100
+2021-03-05,11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67
+2021-03-05,12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193
+2021-03-05,13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5
+2021-03-05,14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250
+2021-03-05,15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245
+2021-03-05,16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54
+2021-03-05,17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96
+2021-03-05,18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72
+2021-03-05,19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174
+2021-03-05,20,Andrew,Davis,adavisj@patch.com,Male,9.255.67.25
+2021-03-05,21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253
+2021-03-05,22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153
+2021-03-05,23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201
+2021-03-05,24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122
+2021-03-05,25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95
+2021-03-05,26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52
+2021-03-05,27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26
+2021-03-05,28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118
+2021-03-05,29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28
+2021-03-05,30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177
+2021-03-05,31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233
+2021-03-05,32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203
+2021-03-05,33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149
+2021-03-05,34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167
+2021-03-05,35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110
+2021-03-05,36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68
+2021-03-05,37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89
+2021-03-05,38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81
+2021-03-05,39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15
+2021-03-05,40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255
+2021-03-05,41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140
+2021-03-05,42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24
+2021-03-05,43,Sean,Mason,smason16@icq.com,Male,159.219.155.249
+2021-03-05,44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218
+2021-03-05,45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198
+2021-03-05,46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18
+2021-03-05,47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238
+2021-03-05,48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61
+2021-03-05,49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21
+2021-03-05,50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209
+2021-03-05,51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87
+2021-03-05,52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142
+2021-03-05,53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126
+2021-03-05,54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212
+2021-03-05,55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194
+2021-03-05,56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22
+2021-03-05,57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60
+2021-03-05,58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50
+2021-03-05,59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222
+2021-03-05,60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115
+2021-03-05,61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155
+2021-03-05,62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94
+2021-03-05,63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106
+2021-03-05,64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68
+2021-03-05,65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41
+2021-03-05,66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109
+2021-03-05,67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77
+2021-03-05,68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194
+2021-03-05,69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135
+2021-03-05,70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87
+2021-03-05,71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44
+2021-03-05,72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182
+2021-03-05,73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241
+2021-03-05,74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24
+2021-03-05,75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214
+2021-03-05,76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199
+2021-03-05,77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41
+2021-03-05,78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255
+2021-03-05,79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144
+2021-03-05,80,Rose,King,rking27@ucoz.com,Female,212.123.168.231
+2021-03-05,81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188
+2021-03-05,82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61
+2021-03-05,83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30
+2021-03-05,84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192
+2021-03-05,85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232
+2021-03-05,86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109
+2021-03-05,87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156
+2021-03-05,88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84
+2021-03-05,89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235
+2021-03-05,90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53
+2021-03-05,91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221
+2021-03-05,92,Angela,Rogers,arogers2j@goodreads.com,Female,98.104.132.187
+2021-03-05,93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57
+2021-03-05,94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189
+2021-03-05,95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180
+2021-03-05,96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144
+2021-03-05,97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117
+2021-03-05,98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126
+2021-03-05,99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244
+2021-03-05,100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88
+""".lstrip()
+
+_SEEDS__SEED_MERGE_COLS_UPDATE = """
+load_date,id,first_name,last_name,email,gender,ip_address
+2021-03-05,1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
+2021-03-05,2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
+2021-03-05,3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
+2021-03-05,4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
+2021-03-05,5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
+2021-03-05,6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220
+2021-03-05,7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64
+2021-03-05,8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13
+2021-03-05,9,Gary,Day,gday8@nih.gov,Male,35.81.68.186
+2021-03-05,10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100
+2021-03-05,11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67
+2021-03-05,12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193
+2021-03-05,13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5
+2021-03-05,14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250
+2021-03-05,15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245
+2021-03-05,16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54
+2021-03-05,17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96
+2021-03-05,18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72
+2021-03-05,19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174
+2021-03-05,20,Andrew,Davis,adavisj@patch.com,Male,9.255.67.25
+2021-03-05,21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253
+2021-03-05,22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153
+2021-03-05,23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201
+2021-03-05,24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122
+2021-03-05,25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95
+2021-03-05,26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52
+2021-03-05,27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26
+2021-03-05,28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118
+2021-03-05,29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28
+2021-03-05,30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177
+2021-03-05,31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233
+2021-03-05,32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203
+2021-03-05,33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149
+2021-03-05,34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167
+2021-03-05,35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110
+2021-03-05,36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68
+2021-03-05,37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89
+2021-03-05,38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81
+2021-03-05,39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15
+2021-03-05,40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255
+2021-03-05,41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140
+2021-03-05,42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24
+2021-03-05,43,Sean,Mason,smason16@icq.com,Male,159.219.155.249
+2021-03-05,44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218
+2021-03-05,45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198
+2021-03-05,46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18
+2021-03-05,47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238
+2021-03-05,48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61
+2021-03-05,49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21
+2021-03-05,50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209
+2021-03-05,51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87
+2021-03-05,52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142
+2021-03-05,53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126
+2021-03-05,54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212
+2021-03-05,55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194
+2021-03-05,56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22
+2021-03-05,57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60
+2021-03-05,58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50
+2021-03-05,59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222
+2021-03-05,60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115
+2021-03-05,61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155
+2021-03-05,62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94
+2021-03-05,63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106
+2021-03-05,64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68
+2021-03-05,65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41
+2021-03-05,66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109
+2021-03-05,67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77
+2021-03-05,68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194
+2021-03-05,69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135
+2021-03-05,70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87
+2021-03-05,71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44
+2021-03-05,72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182
+2021-03-05,73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241
+2021-03-05,74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24
+2021-03-05,75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214
+2021-03-05,76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199
+2021-03-05,77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41
+2021-03-05,78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255
+2021-03-05,79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144
+2021-03-05,80,Rose,King,rking27@ucoz.com,Female,212.123.168.231
+2021-03-05,81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188
+2021-03-05,82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61
+2021-03-05,83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30
+2021-03-05,84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192
+2021-03-05,85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232
+2021-03-05,86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109
+2021-03-05,87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156
+2021-03-05,88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84
+2021-03-05,89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235
+2021-03-05,90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53
+2021-03-05,91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221
+2021-03-05,92,Angela,Rogers,arogers2j@goodreads.com,Female,98.104.132.187
+2021-03-05,93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57
+2021-03-05,94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189
+2021-03-05,95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180
+2021-03-05,96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144
+2021-03-05,97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117
+2021-03-05,98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126
+2021-03-05,99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244
+2021-03-05,100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88
+2021-03-06,20,Andrew,Davis,adavisj@reddit.com,Male,9.255.67.25
+2021-03-06,83,Josh,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30
+2021-03-06,92,Angela,Scott,ascott2j@goodreads.com,Female,98.119.208.155
+2021-03-06,101,Michael,Perez,mperez0@chronoengine.com,Male,106.239.70.175
+2021-03-06,102,Shawn,Mccoy,smccoy1@reddit.com,Male,24.165.76.182
+2021-03-06,103,Kathleen,Payne,kpayne2@cargocollective.com,Female,113.207.168.106
+2021-03-06,104,Jimmy,Cooper,jcooper3@cargocollective.com,Male,198.24.63.114
+2021-03-06,105,Katherine,Rice,krice4@typepad.com,Female,36.97.186.238
+2021-03-06,106,Sarah,Ryan,sryan5@gnu.org,Female,119.117.152.40
+2021-03-06,107,Martin,Mcdonald,mmcdonald6@opera.com,Male,8.76.38.115
+2021-03-06,108,Frank,Robinson,frobinson7@wunderground.com,Male,186.14.64.194
+2021-03-06,109,Jennifer,Franklin,jfranklin8@mail.ru,Female,91.216.3.131
+2021-03-06,110,Henry,Welch,hwelch9@list-manage.com,Male,176.35.182.168
+2021-03-06,111,Fred,Snyder,fsnydera@reddit.com,Male,217.106.196.54
+2021-03-06,112,Amy,Dunn,adunnb@nba.com,Female,95.39.163.195
+2021-03-06,113,Kathleen,Meyer,kmeyerc@cdc.gov,Female,164.142.188.214
+2021-03-06,114,Steve,Ferguson,sfergusond@reverbnation.com,Male,138.22.204.251
+2021-03-06,115,Teresa,Hill,thille@dion.ne.jp,Female,82.84.228.235
+2021-03-06,116,Amanda,Harper,aharperf@mail.ru,Female,16.123.56.176
+2021-03-06,117,Kimberly,Ray,krayg@xing.com,Female,48.66.48.12
+2021-03-06,118,Johnny,Knight,jknighth@jalbum.net,Male,99.30.138.123
+2021-03-06,119,Virginia,Freeman,vfreemani@tiny.cc,Female,225.172.182.63
+2021-03-06,120,Anna,Austin,aaustinj@diigo.com,Female,62.111.227.148
+2021-03-06,121,Willie,Hill,whillk@mail.ru,Male,0.86.232.249
+2021-03-06,122,Sean,Harris,sharrisl@zdnet.com,Male,117.165.133.249
+2021-03-06,123,Mildred,Adams,madamsm@usatoday.com,Female,163.44.97.46
+2021-03-06,124,David,Graham,dgrahamn@zimbio.com,Male,78.13.246.202
+2021-03-06,125,Victor,Hunter,vhuntero@ehow.com,Male,64.156.179.139
+2021-03-06,126,Aaron,Ruiz,aruizp@weebly.com,Male,34.194.68.78
+2021-03-06,127,Benjamin,Brooks,bbrooksq@jalbum.net,Male,20.192.189.107
+2021-03-06,128,Lisa,Wilson,lwilsonr@japanpost.jp,Female,199.152.130.217
+2021-03-06,129,Benjamin,King,bkings@comsenz.com,Male,29.189.189.213
+2021-03-06,130,Christina,Williamson,cwilliamsont@boston.com,Female,194.101.52.60
+2021-03-06,131,Jane,Gonzalez,jgonzalezu@networksolutions.com,Female,109.119.12.87
+2021-03-06,132,Thomas,Owens,towensv@psu.edu,Male,84.168.213.153
+2021-03-06,133,Katherine,Moore,kmoorew@naver.com,Female,183.150.65.24
+2021-03-06,134,Jennifer,Stewart,jstewartx@yahoo.com,Female,38.41.244.58
+2021-03-06,135,Sara,Tucker,stuckery@topsy.com,Female,181.130.59.184
+2021-03-06,136,Harold,Ortiz,hortizz@vkontakte.ru,Male,198.231.63.137
+2021-03-06,137,Shirley,James,sjames10@yelp.com,Female,83.27.160.104
+2021-03-06,138,Dennis,Johnson,djohnson11@slate.com,Male,183.178.246.101
+2021-03-06,139,Louise,Weaver,lweaver12@china.com.cn,Female,1.14.110.18
+2021-03-06,140,Maria,Armstrong,marmstrong13@prweb.com,Female,181.142.1.249
+2021-03-06,141,Gloria,Cruz,gcruz14@odnoklassniki.ru,Female,178.232.140.243
+2021-03-06,142,Diana,Spencer,dspencer15@ifeng.com,Female,125.153.138.244
+2021-03-06,143,Kelly,Nguyen,knguyen16@altervista.org,Female,170.13.201.119
+2021-03-06,144,Jane,Rodriguez,jrodriguez17@biblegateway.com,Female,12.102.249.81
+2021-03-06,145,Scott,Brown,sbrown18@geocities.jp,Male,108.174.99.192
+2021-03-06,146,Norma,Cruz,ncruz19@si.edu,Female,201.112.156.197
+2021-03-06,147,Marie,Peters,mpeters1a@mlb.com,Female,231.121.197.144
+2021-03-06,148,Lillian,Carr,lcarr1b@typepad.com,Female,206.179.164.163
+2021-03-06,149,Judy,Nichols,jnichols1c@t-online.de,Female,158.190.209.194
+2021-03-06,150,Billy,Long,blong1d@yahoo.com,Male,175.20.23.160
+2021-03-06,151,Howard,Reid,hreid1e@exblog.jp,Male,118.99.196.20
+2021-03-06,152,Laura,Ferguson,lferguson1f@tuttocitta.it,Female,22.77.87.110
+2021-03-06,153,Anne,Bailey,abailey1g@geocities.com,Female,58.144.159.245
+2021-03-06,154,Rose,Morgan,rmorgan1h@ehow.com,Female,118.127.97.4
+2021-03-06,155,Nicholas,Reyes,nreyes1i@google.ru,Male,50.135.10.252
+2021-03-06,156,Joshua,Kennedy,jkennedy1j@house.gov,Male,154.6.163.209
+2021-03-06,157,Paul,Watkins,pwatkins1k@upenn.edu,Male,177.236.120.87
+2021-03-06,158,Kathryn,Kelly,kkelly1l@businessweek.com,Female,70.28.61.86
+2021-03-06,159,Adam,Armstrong,aarmstrong1m@techcrunch.com,Male,133.235.24.202
+2021-03-06,160,Norma,Wallace,nwallace1n@phoca.cz,Female,241.119.227.128
+2021-03-06,161,Timothy,Reyes,treyes1o@google.cn,Male,86.28.23.26
+2021-03-06,162,Elizabeth,Patterson,epatterson1p@sun.com,Female,139.97.159.149
+2021-03-06,163,Edward,Gomez,egomez1q@google.fr,Male,158.103.108.255
+2021-03-06,164,David,Cox,dcox1r@friendfeed.com,Male,206.80.80.58
+2021-03-06,165,Brenda,Wood,bwood1s@over-blog.com,Female,217.207.44.179
+2021-03-06,166,Adam,Walker,awalker1t@blogs.com,Male,253.211.54.93
+2021-03-06,167,Michael,Hart,mhart1u@wix.com,Male,230.206.200.22
+2021-03-06,168,Jesse,Ellis,jellis1v@google.co.uk,Male,213.254.162.52
+2021-03-06,169,Janet,Powell,jpowell1w@un.org,Female,27.192.194.86
+2021-03-06,170,Helen,Ford,hford1x@creativecommons.org,Female,52.160.102.168
+2021-03-06,171,Gerald,Carpenter,gcarpenter1y@about.me,Male,36.30.194.218
+2021-03-06,172,Kathryn,Oliver,koliver1z@army.mil,Female,202.63.103.69
+2021-03-06,173,Alan,Berry,aberry20@gov.uk,Male,246.157.112.211
+2021-03-06,174,Harry,Andrews,handrews21@ameblo.jp,Male,195.108.0.12
+2021-03-06,175,Andrea,Hall,ahall22@hp.com,Female,149.162.163.28
+2021-03-06,176,Barbara,Wells,bwells23@behance.net,Female,224.70.72.1
+2021-03-06,177,Anne,Wells,awells24@apache.org,Female,180.168.81.153
+2021-03-06,178,Harry,Harper,hharper25@rediff.com,Male,151.87.130.21
+2021-03-06,179,Jack,Ray,jray26@wufoo.com,Male,220.109.38.178
+2021-03-06,180,Phillip,Hamilton,phamilton27@joomla.org,Male,166.40.47.30
+2021-03-06,181,Shirley,Hunter,shunter28@newsvine.com,Female,97.209.140.194
+2021-03-06,182,Arthur,Daniels,adaniels29@reuters.com,Male,5.40.240.86
+2021-03-06,183,Virginia,Rodriguez,vrodriguez2a@walmart.com,Female,96.80.164.184
+2021-03-06,184,Christina,Ryan,cryan2b@hibu.com,Female,56.35.5.52
+2021-03-06,185,Theresa,Mendoza,tmendoza2c@vinaora.com,Female,243.42.0.210
+2021-03-06,186,Jason,Cole,jcole2d@ycombinator.com,Male,198.248.39.129
+2021-03-06,187,Phillip,Bryant,pbryant2e@rediff.com,Male,140.39.116.251
+2021-03-06,188,Adam,Torres,atorres2f@sun.com,Male,101.75.187.135
+2021-03-06,189,Margaret,Johnston,mjohnston2g@ucsd.edu,Female,159.30.69.149
+2021-03-06,190,Paul,Payne,ppayne2h@hhs.gov,Male,199.234.140.220
+2021-03-06,191,Todd,Willis,twillis2i@businessweek.com,Male,191.59.136.214
+2021-03-06,192,Willie,Oliver,woliver2j@noaa.gov,Male,44.212.35.197
+2021-03-06,193,Frances,Robertson,frobertson2k@go.com,Female,31.117.65.136
+2021-03-06,194,Gregory,Hawkins,ghawkins2l@joomla.org,Male,91.3.22.49
+2021-03-06,195,Lisa,Perkins,lperkins2m@si.edu,Female,145.95.31.186
+2021-03-06,196,Jacqueline,Anderson,janderson2n@cargocollective.com,Female,14.176.0.187
+2021-03-06,197,Shirley,Diaz,sdiaz2o@ucla.edu,Female,207.12.95.46
+2021-03-06,198,Nicole,Meyer,nmeyer2p@flickr.com,Female,231.79.115.13
+2021-03-06,199,Mary,Gray,mgray2q@constantcontact.com,Female,210.116.64.253
+2021-03-06,200,Jean,Mcdonald,jmcdonald2r@baidu.com,Female,122.239.235.117
+""".lstrip()
+
+
+_SEEDS__SEED_MERGE_COLS_EXPECTED_RESULT = """
+load_date,id,first_name,last_name,email,gender,ip_address
+2021-03-05,1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
+2021-03-05,2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
+2021-03-05,3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
+2021-03-05,4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
+2021-03-05,5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
+2021-03-05,6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220
+2021-03-05,7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64
+2021-03-05,8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13
+2021-03-05,9,Gary,Day,gday8@nih.gov,Male,35.81.68.186
+2021-03-05,10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100
+2021-03-05,11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67
+2021-03-05,12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193
+2021-03-05,13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5
+2021-03-05,14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250
+2021-03-05,15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245
+2021-03-05,16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54
+2021-03-05,17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96
+2021-03-05,18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72
+2021-03-05,19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174
+2021-03-05,20,Andrew,Davis,adavisj@reddit.com,Male,9.255.67.25
+2021-03-05,21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253
+2021-03-05,22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153
+2021-03-05,23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201
+2021-03-05,24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122
+2021-03-05,25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95
+2021-03-05,26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52
+2021-03-05,27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26
+2021-03-05,28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118
+2021-03-05,29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28
+2021-03-05,30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177
+2021-03-05,31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233
+2021-03-05,32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203
+2021-03-05,33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149
+2021-03-05,34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167
+2021-03-05,35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110
+2021-03-05,36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68
+2021-03-05,37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89
+2021-03-05,38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81
+2021-03-05,39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15
+2021-03-05,40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255
+2021-03-05,41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140
+2021-03-05,42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24
+2021-03-05,43,Sean,Mason,smason16@icq.com,Male,159.219.155.249
+2021-03-05,44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218
+2021-03-05,45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198
+2021-03-05,46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18
+2021-03-05,47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238
+2021-03-05,48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61
+2021-03-05,49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21
+2021-03-05,50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209
+2021-03-05,51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87
+2021-03-05,52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142
+2021-03-05,53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126
+2021-03-05,54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212
+2021-03-05,55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194
+2021-03-05,56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22
+2021-03-05,57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60
+2021-03-05,58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50
+2021-03-05,59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222
+2021-03-05,60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115
+2021-03-05,61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155
+2021-03-05,62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94
+2021-03-05,63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106
+2021-03-05,64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68
+2021-03-05,65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41
+2021-03-05,66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109
+2021-03-05,67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77
+2021-03-05,68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194
+2021-03-05,69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135
+2021-03-05,70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87
+2021-03-05,71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44
+2021-03-05,72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182
+2021-03-05,73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241
+2021-03-05,74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24
+2021-03-05,75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214
+2021-03-05,76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199
+2021-03-05,77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41
+2021-03-05,78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255
+2021-03-05,79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144
+2021-03-05,80,Rose,King,rking27@ucoz.com,Female,212.123.168.231
+2021-03-05,81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188
+2021-03-05,82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61
+2021-03-05,83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30
+2021-03-05,84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192
+2021-03-05,85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232
+2021-03-05,86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109
+2021-03-05,87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156
+2021-03-05,88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84
+2021-03-05,89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235
+2021-03-05,90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53
+2021-03-05,91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221
+2021-03-05,92,Angela,Rogers,ascott2j@goodreads.com,Female,98.119.208.155
+2021-03-05,93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57
+2021-03-05,94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189
+2021-03-05,95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180
+2021-03-05,96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144
+2021-03-05,97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117
+2021-03-05,98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126
+2021-03-05,99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244
+2021-03-05,100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88
+2021-03-06,101,Michael,Perez,mperez0@chronoengine.com,Male,106.239.70.175
+2021-03-06,102,Shawn,Mccoy,smccoy1@reddit.com,Male,24.165.76.182
+2021-03-06,103,Kathleen,Payne,kpayne2@cargocollective.com,Female,113.207.168.106
+2021-03-06,104,Jimmy,Cooper,jcooper3@cargocollective.com,Male,198.24.63.114
+2021-03-06,105,Katherine,Rice,krice4@typepad.com,Female,36.97.186.238
+2021-03-06,106,Sarah,Ryan,sryan5@gnu.org,Female,119.117.152.40
+2021-03-06,107,Martin,Mcdonald,mmcdonald6@opera.com,Male,8.76.38.115
+2021-03-06,108,Frank,Robinson,frobinson7@wunderground.com,Male,186.14.64.194
+2021-03-06,109,Jennifer,Franklin,jfranklin8@mail.ru,Female,91.216.3.131
+2021-03-06,110,Henry,Welch,hwelch9@list-manage.com,Male,176.35.182.168
+2021-03-06,111,Fred,Snyder,fsnydera@reddit.com,Male,217.106.196.54
+2021-03-06,112,Amy,Dunn,adunnb@nba.com,Female,95.39.163.195
+2021-03-06,113,Kathleen,Meyer,kmeyerc@cdc.gov,Female,164.142.188.214
+2021-03-06,114,Steve,Ferguson,sfergusond@reverbnation.com,Male,138.22.204.251
+2021-03-06,115,Teresa,Hill,thille@dion.ne.jp,Female,82.84.228.235
+2021-03-06,116,Amanda,Harper,aharperf@mail.ru,Female,16.123.56.176
+2021-03-06,117,Kimberly,Ray,krayg@xing.com,Female,48.66.48.12
+2021-03-06,118,Johnny,Knight,jknighth@jalbum.net,Male,99.30.138.123
+2021-03-06,119,Virginia,Freeman,vfreemani@tiny.cc,Female,225.172.182.63
+2021-03-06,120,Anna,Austin,aaustinj@diigo.com,Female,62.111.227.148
+2021-03-06,121,Willie,Hill,whillk@mail.ru,Male,0.86.232.249
+2021-03-06,122,Sean,Harris,sharrisl@zdnet.com,Male,117.165.133.249
+2021-03-06,123,Mildred,Adams,madamsm@usatoday.com,Female,163.44.97.46
+2021-03-06,124,David,Graham,dgrahamn@zimbio.com,Male,78.13.246.202
+2021-03-06,125,Victor,Hunter,vhuntero@ehow.com,Male,64.156.179.139
+2021-03-06,126,Aaron,Ruiz,aruizp@weebly.com,Male,34.194.68.78
+2021-03-06,127,Benjamin,Brooks,bbrooksq@jalbum.net,Male,20.192.189.107
+2021-03-06,128,Lisa,Wilson,lwilsonr@japanpost.jp,Female,199.152.130.217
+2021-03-06,129,Benjamin,King,bkings@comsenz.com,Male,29.189.189.213
+2021-03-06,130,Christina,Williamson,cwilliamsont@boston.com,Female,194.101.52.60
+2021-03-06,131,Jane,Gonzalez,jgonzalezu@networksolutions.com,Female,109.119.12.87
+2021-03-06,132,Thomas,Owens,towensv@psu.edu,Male,84.168.213.153
+2021-03-06,133,Katherine,Moore,kmoorew@naver.com,Female,183.150.65.24
+2021-03-06,134,Jennifer,Stewart,jstewartx@yahoo.com,Female,38.41.244.58
+2021-03-06,135,Sara,Tucker,stuckery@topsy.com,Female,181.130.59.184
+2021-03-06,136,Harold,Ortiz,hortizz@vkontakte.ru,Male,198.231.63.137
+2021-03-06,137,Shirley,James,sjames10@yelp.com,Female,83.27.160.104
+2021-03-06,138,Dennis,Johnson,djohnson11@slate.com,Male,183.178.246.101
+2021-03-06,139,Louise,Weaver,lweaver12@china.com.cn,Female,1.14.110.18
+2021-03-06,140,Maria,Armstrong,marmstrong13@prweb.com,Female,181.142.1.249
+2021-03-06,141,Gloria,Cruz,gcruz14@odnoklassniki.ru,Female,178.232.140.243
+2021-03-06,142,Diana,Spencer,dspencer15@ifeng.com,Female,125.153.138.244
+2021-03-06,143,Kelly,Nguyen,knguyen16@altervista.org,Female,170.13.201.119
+2021-03-06,144,Jane,Rodriguez,jrodriguez17@biblegateway.com,Female,12.102.249.81
+2021-03-06,145,Scott,Brown,sbrown18@geocities.jp,Male,108.174.99.192
+2021-03-06,146,Norma,Cruz,ncruz19@si.edu,Female,201.112.156.197
+2021-03-06,147,Marie,Peters,mpeters1a@mlb.com,Female,231.121.197.144
+2021-03-06,148,Lillian,Carr,lcarr1b@typepad.com,Female,206.179.164.163
+2021-03-06,149,Judy,Nichols,jnichols1c@t-online.de,Female,158.190.209.194
+2021-03-06,150,Billy,Long,blong1d@yahoo.com,Male,175.20.23.160
+2021-03-06,151,Howard,Reid,hreid1e@exblog.jp,Male,118.99.196.20
+2021-03-06,152,Laura,Ferguson,lferguson1f@tuttocitta.it,Female,22.77.87.110
+2021-03-06,153,Anne,Bailey,abailey1g@geocities.com,Female,58.144.159.245
+2021-03-06,154,Rose,Morgan,rmorgan1h@ehow.com,Female,118.127.97.4
+2021-03-06,155,Nicholas,Reyes,nreyes1i@google.ru,Male,50.135.10.252
+2021-03-06,156,Joshua,Kennedy,jkennedy1j@house.gov,Male,154.6.163.209
+2021-03-06,157,Paul,Watkins,pwatkins1k@upenn.edu,Male,177.236.120.87
+2021-03-06,158,Kathryn,Kelly,kkelly1l@businessweek.com,Female,70.28.61.86
+2021-03-06,159,Adam,Armstrong,aarmstrong1m@techcrunch.com,Male,133.235.24.202
+2021-03-06,160,Norma,Wallace,nwallace1n@phoca.cz,Female,241.119.227.128
+2021-03-06,161,Timothy,Reyes,treyes1o@google.cn,Male,86.28.23.26
+2021-03-06,162,Elizabeth,Patterson,epatterson1p@sun.com,Female,139.97.159.149
+2021-03-06,163,Edward,Gomez,egomez1q@google.fr,Male,158.103.108.255
+2021-03-06,164,David,Cox,dcox1r@friendfeed.com,Male,206.80.80.58
+2021-03-06,165,Brenda,Wood,bwood1s@over-blog.com,Female,217.207.44.179
+2021-03-06,166,Adam,Walker,awalker1t@blogs.com,Male,253.211.54.93
+2021-03-06,167,Michael,Hart,mhart1u@wix.com,Male,230.206.200.22
+2021-03-06,168,Jesse,Ellis,jellis1v@google.co.uk,Male,213.254.162.52
+2021-03-06,169,Janet,Powell,jpowell1w@un.org,Female,27.192.194.86
+2021-03-06,170,Helen,Ford,hford1x@creativecommons.org,Female,52.160.102.168
+2021-03-06,171,Gerald,Carpenter,gcarpenter1y@about.me,Male,36.30.194.218
+2021-03-06,172,Kathryn,Oliver,koliver1z@army.mil,Female,202.63.103.69
+2021-03-06,173,Alan,Berry,aberry20@gov.uk,Male,246.157.112.211
+2021-03-06,174,Harry,Andrews,handrews21@ameblo.jp,Male,195.108.0.12
+2021-03-06,175,Andrea,Hall,ahall22@hp.com,Female,149.162.163.28
+2021-03-06,176,Barbara,Wells,bwells23@behance.net,Female,224.70.72.1
+2021-03-06,177,Anne,Wells,awells24@apache.org,Female,180.168.81.153
+2021-03-06,178,Harry,Harper,hharper25@rediff.com,Male,151.87.130.21
+2021-03-06,179,Jack,Ray,jray26@wufoo.com,Male,220.109.38.178
+2021-03-06,180,Phillip,Hamilton,phamilton27@joomla.org,Male,166.40.47.30
+2021-03-06,181,Shirley,Hunter,shunter28@newsvine.com,Female,97.209.140.194
+2021-03-06,182,Arthur,Daniels,adaniels29@reuters.com,Male,5.40.240.86
+2021-03-06,183,Virginia,Rodriguez,vrodriguez2a@walmart.com,Female,96.80.164.184
+2021-03-06,184,Christina,Ryan,cryan2b@hibu.com,Female,56.35.5.52
+2021-03-06,185,Theresa,Mendoza,tmendoza2c@vinaora.com,Female,243.42.0.210
+2021-03-06,186,Jason,Cole,jcole2d@ycombinator.com,Male,198.248.39.129
+2021-03-06,187,Phillip,Bryant,pbryant2e@rediff.com,Male,140.39.116.251
+2021-03-06,188,Adam,Torres,atorres2f@sun.com,Male,101.75.187.135
+2021-03-06,189,Margaret,Johnston,mjohnston2g@ucsd.edu,Female,159.30.69.149
+2021-03-06,190,Paul,Payne,ppayne2h@hhs.gov,Male,199.234.140.220
+2021-03-06,191,Todd,Willis,twillis2i@businessweek.com,Male,191.59.136.214
+2021-03-06,192,Willie,Oliver,woliver2j@noaa.gov,Male,44.212.35.197
+2021-03-06,193,Frances,Robertson,frobertson2k@go.com,Female,31.117.65.136
+2021-03-06,194,Gregory,Hawkins,ghawkins2l@joomla.org,Male,91.3.22.49
+2021-03-06,195,Lisa,Perkins,lperkins2m@si.edu,Female,145.95.31.186
+2021-03-06,196,Jacqueline,Anderson,janderson2n@cargocollective.com,Female,14.176.0.187
+2021-03-06,197,Shirley,Diaz,sdiaz2o@ucla.edu,Female,207.12.95.46
+2021-03-06,198,Nicole,Meyer,nmeyer2p@flickr.com,Female,231.79.115.13
+2021-03-06,199,Mary,Gray,mgray2q@constantcontact.com,Female,210.116.64.253
+2021-03-06,200,Jean,Mcdonald,jmcdonald2r@baidu.com,Female,122.239.235.117
+""".lstrip()
+
+
+_MODELS_INCREMENTAL_UPDATE_COLS = """
+{{
+    config(
+        materialized = "incremental",
+        unique_key = "id",
+        merge_update_columns = ["email", "ip_address"]
+    )
+}}
+
+select * from {{ ref('seed') }}
+
+{% if is_incremental() %}
+
+    where load_date > (select max(load_date) from {{this}})
+
+{% endif %}
+"""
diff --git a/tests/functional/adapter/simple_copy/test_simple_copy.py b/tests/functional/adapter/simple_copy/test_simple_copy.py
new file mode 100644
index 000000000..c590ab82f
--- /dev/null
+++ b/tests/functional/adapter/simple_copy/test_simple_copy.py
@@ -0,0 +1,51 @@
+import json
+import os
+import pytest
+
+from pathlib import Path
+from pytest import mark
+
+from dbt.tests.util import run_dbt, rm_file, write_file, check_relations_equal
+
+from dbt.tests.adapter.simple_copy.test_simple_copy import (
+   SimpleCopySetup,
+   SimpleCopyBase
+)
+
+from tests.functional.adapter.simple_copy.fixtures import (
+    _MODELS_INCREMENTAL_UPDATE_COLS,
+    _SEEDS__SEED_MERGE_COLS_INITIAL,
+    _SEEDS__SEED_MERGE_COLS_UPDATE,
+    _SEEDS__SEED_MERGE_COLS_EXPECTED_RESULT,
+)
+
+class TestSimpleCopyBase(SimpleCopyBase):
+    pass
+
+
+class TestIncrementalMergeColumns:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "incremental_update_cols.sql": _MODELS_INCREMENTAL_UPDATE_COLS
+        }
+
+    @pytest.fixture(scope="class")
+    def seeds(self):
+        return {"seed.csv": _SEEDS__SEED_MERGE_COLS_INITIAL}
+
+    def test_incremental_merge_columns(self, project):
+        run_dbt(["seed"])
+        run_dbt(["run"])
+
+        main_seed_file = project.project_root / Path("seeds") / Path("seed.csv")
+        expected_seed_file = project.project_root / Path("seeds") / Path("expected_result.csv")
+        rm_file(main_seed_file)
+        write_file(_SEEDS__SEED_MERGE_COLS_UPDATE, main_seed_file)
+        write_file(_SEEDS__SEED_MERGE_COLS_EXPECTED_RESULT, expected_seed_file)
+
+        run_dbt(["seed"])
+        run_dbt(["run"])
+        check_relations_equal(
+            project.adapter, ["incremental_update_cols", "expected_result"]
+        )
diff --git a/tests/integration/simple_copy_test/models-merge-update/incremental_update_cols.sql b/tests/integration/simple_copy_test/models-merge-update/incremental_update_cols.sql
deleted file mode 100644
index 6f79e08e9..000000000
--- a/tests/integration/simple_copy_test/models-merge-update/incremental_update_cols.sql
+++ /dev/null
@@ -1,17 +0,0 @@
-{{
-    config(
-        materialized = "incremental",
-        unique_key = "id",
-        merge_update_columns = ["email", "ip_address"]
-    )
-}}
-
-
-select *
-from {{ ref('seed') }}
-
-{% if is_incremental() %}
-
-    where load_date > (select max(load_date) from {{this}})
-
-{% endif %}
diff --git a/tests/integration/simple_copy_test/models/advanced_incremental.sql b/tests/integration/simple_copy_test/models/advanced_incremental.sql
deleted file mode 100644
index 7f4c12af8..000000000
--- a/tests/integration/simple_copy_test/models/advanced_incremental.sql
+++ /dev/null
@@ -1,17 +0,0 @@
-{{
-  config(
-    materialized = "incremental",
-    unique_key = "id",
-    persist_docs = {"relation": true}
-  )
-}}
-
-
-select *
-from {{ ref('seed') }}
-
-{% if is_incremental() %}
-
-    where id > (select max(id) from {{this}})
-
-{% endif %}
diff --git a/tests/integration/simple_copy_test/models/compound_sort.sql b/tests/integration/simple_copy_test/models/compound_sort.sql
deleted file mode 100644
index 64b41ca7e..000000000
--- a/tests/integration/simple_copy_test/models/compound_sort.sql
+++ /dev/null
@@ -1,9 +0,0 @@
-{{
-  config(
-    materialized = "table",
-    sort = 'first_name',
-    sort_type = 'compound'
-  )
-}}
-
-select * from {{ ref('seed') }}
diff --git a/tests/integration/simple_copy_test/models/disabled.sql b/tests/integration/simple_copy_test/models/disabled.sql
deleted file mode 100644
index 1d10a0c8d..000000000
--- a/tests/integration/simple_copy_test/models/disabled.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-{{
-  config(
-    materialized = "view",
-    enabled = False
-  )
-}}
-
-select * from {{ ref('seed') }}
diff --git a/tests/integration/simple_copy_test/models/empty.sql b/tests/integration/simple_copy_test/models/empty.sql
deleted file mode 100644
index 3f2ff2d6c..000000000
--- a/tests/integration/simple_copy_test/models/empty.sql
+++ /dev/null
@@ -1,5 +0,0 @@
-
-
-
-
-
diff --git a/tests/integration/simple_copy_test/models/get_and_ref.sql b/tests/integration/simple_copy_test/models/get_and_ref.sql
deleted file mode 100644
index 89d55c738..000000000
--- a/tests/integration/simple_copy_test/models/get_and_ref.sql
+++ /dev/null
@@ -1,3 +0,0 @@
-{%- do adapter.get_relation(database=target.database, schema=target.schema, identifier='materialized') -%}
-
-select * from {{ ref('materialized') }}
diff --git a/tests/integration/simple_copy_test/models/incremental.sql b/tests/integration/simple_copy_test/models/incremental.sql
deleted file mode 100644
index 020bf3516..000000000
--- a/tests/integration/simple_copy_test/models/incremental.sql
+++ /dev/null
@@ -1,11 +0,0 @@
-{{
-  config(
-    materialized = "incremental"
-  )
-}}
-
-select * from {{ ref('seed') }}
-
-{% if is_incremental() %}
-    where id > (select max(id) from {{this}})
-{% endif %}
diff --git a/tests/integration/simple_copy_test/models/interleaved_sort.sql b/tests/integration/simple_copy_test/models/interleaved_sort.sql
deleted file mode 100644
index 147370396..000000000
--- a/tests/integration/simple_copy_test/models/interleaved_sort.sql
+++ /dev/null
@@ -1,9 +0,0 @@
-{{
-  config(
-    materialized = "table",
-    sort = ['first_name', 'last_name'],
-    sort_type = 'interleaved'
-  )
-}}
-
-select * from {{ ref('seed') }}
diff --git a/tests/integration/simple_copy_test/models/materialized.sql b/tests/integration/simple_copy_test/models/materialized.sql
deleted file mode 100644
index 5b004a514..000000000
--- a/tests/integration/simple_copy_test/models/materialized.sql
+++ /dev/null
@@ -1,12 +0,0 @@
-{{
-  config(
-    materialized = "table"
-  )
-}}
--- ensure that dbt_utils' relation check will work
-{% set relation = ref('seed') %}
-{%- if not (relation is mapping and relation.get('metadata', {}).get('type', '').endswith('Relation')) -%}
-    {%- do exceptions.raise_compiler_error("Macro " ~ macro ~ " expected a Relation but received the value: " ~ relation) -%}
-{%- endif -%}
--- this is a unicode character: å
-select * from {{ relation }}
diff --git a/tests/integration/simple_copy_test/models/schema.yml b/tests/integration/simple_copy_test/models/schema.yml
deleted file mode 100644
index 9a813eed5..000000000
--- a/tests/integration/simple_copy_test/models/schema.yml
+++ /dev/null
@@ -1,7 +0,0 @@
-version: 2
-models:
-- name: disabled
-  columns:
-  - name: id
-    tests:
-    - unique
diff --git a/tests/integration/simple_copy_test/models/view_model.sql b/tests/integration/simple_copy_test/models/view_model.sql
deleted file mode 100644
index 9838c5b83..000000000
--- a/tests/integration/simple_copy_test/models/view_model.sql
+++ /dev/null
@@ -1,7 +0,0 @@
-{{
-  config(
-    materialized = "view"
-  )
-}}
-
-select * from {{ ref('seed') }}
diff --git a/tests/integration/simple_copy_test/seed-initial/seed.csv b/tests/integration/simple_copy_test/seed-initial/seed.csv
deleted file mode 100644
index 640af6c4e..000000000
--- a/tests/integration/simple_copy_test/seed-initial/seed.csv
+++ /dev/null
@@ -1,101 +0,0 @@
-id,first_name,last_name,email,gender,ip_address
-1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
-2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
-3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
-4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
-5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
-6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220
-7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64
-8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13
-9,Gary,Day,gday8@nih.gov,Male,35.81.68.186
-10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100
-11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67
-12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193
-13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5
-14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250
-15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245
-16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54
-17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96
-18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72
-19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174
-20,Andrew,Davis,adavisj@patch.com,Male,9.255.67.25
-21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253
-22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153
-23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201
-24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122
-25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95
-26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52
-27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26
-28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118
-29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28
-30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177
-31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233
-32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203
-33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149
-34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167
-35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110
-36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68
-37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89
-38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81
-39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15
-40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255
-41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140
-42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24
-43,Sean,Mason,smason16@icq.com,Male,159.219.155.249
-44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218
-45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198
-46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18
-47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238
-48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61
-49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21
-50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209
-51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87
-52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142
-53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126
-54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212
-55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194
-56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22
-57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60
-58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50
-59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222
-60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115
-61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155
-62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94
-63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106
-64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68
-65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41
-66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109
-67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77
-68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194
-69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135
-70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87
-71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44
-72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182
-73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241
-74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24
-75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214
-76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199
-77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41
-78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255
-79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144
-80,Rose,King,rking27@ucoz.com,Female,212.123.168.231
-81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188
-82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61
-83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30
-84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192
-85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232
-86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109
-87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156
-88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84
-89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235
-90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53
-91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221
-92,Angela,Rogers,arogers2j@goodreads.com,Female,98.104.132.187
-93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57
-94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189
-95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180
-96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144
-97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117
-98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126
-99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244
-100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88
diff --git a/tests/integration/simple_copy_test/seed-update/seed.csv b/tests/integration/simple_copy_test/seed-update/seed.csv
deleted file mode 100644
index 5b93306a2..000000000
--- a/tests/integration/simple_copy_test/seed-update/seed.csv
+++ /dev/null
@@ -1,201 +0,0 @@
-id,first_name,last_name,email,gender,ip_address
-1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
-2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
-3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
-4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
-5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
-6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220
-7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64
-8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13
-9,Gary,Day,gday8@nih.gov,Male,35.81.68.186
-10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100
-11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67
-12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193
-13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5
-14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250
-15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245
-16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54
-17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96
-18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72
-19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174
-20,Andrew,Davis,adavisj@patch.com,Male,9.255.67.25
-21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253
-22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153
-23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201
-24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122
-25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95
-26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52
-27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26
-28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118
-29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28
-30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177
-31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233
-32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203
-33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149
-34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167
-35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110
-36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68
-37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89
-38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81
-39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15
-40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255
-41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140
-42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24
-43,Sean,Mason,smason16@icq.com,Male,159.219.155.249
-44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218
-45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198
-46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18
-47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238
-48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61
-49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21
-50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209
-51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87
-52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142
-53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126
-54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212
-55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194
-56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22
-57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60
-58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50
-59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222
-60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115
-61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155
-62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94
-63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106
-64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68
-65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41
-66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109
-67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77
-68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194
-69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135
-70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87
-71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44
-72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182
-73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241
-74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24
-75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214
-76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199
-77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41
-78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255
-79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144
-80,Rose,King,rking27@ucoz.com,Female,212.123.168.231
-81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188
-82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61
-83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30
-84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192
-85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232
-86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109
-87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156
-88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84
-89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235
-90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53
-91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221
-92,Angela,Rogers,arogers2j@goodreads.com,Female,98.104.132.187
-93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57
-94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189
-95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180
-96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144
-97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117
-98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126
-99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244
-100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88
-101,Michael,Perez,mperez0@chronoengine.com,Male,106.239.70.175
-102,Shawn,Mccoy,smccoy1@reddit.com,Male,24.165.76.182
-103,Kathleen,Payne,kpayne2@cargocollective.com,Female,113.207.168.106
-104,Jimmy,Cooper,jcooper3@cargocollective.com,Male,198.24.63.114
-105,Katherine,Rice,krice4@typepad.com,Female,36.97.186.238
-106,Sarah,Ryan,sryan5@gnu.org,Female,119.117.152.40
-107,Martin,Mcdonald,mmcdonald6@opera.com,Male,8.76.38.115
-108,Frank,Robinson,frobinson7@wunderground.com,Male,186.14.64.194
-109,Jennifer,Franklin,jfranklin8@mail.ru,Female,91.216.3.131
-110,Henry,Welch,hwelch9@list-manage.com,Male,176.35.182.168
-111,Fred,Snyder,fsnydera@reddit.com,Male,217.106.196.54
-112,Amy,Dunn,adunnb@nba.com,Female,95.39.163.195
-113,Kathleen,Meyer,kmeyerc@cdc.gov,Female,164.142.188.214
-114,Steve,Ferguson,sfergusond@reverbnation.com,Male,138.22.204.251
-115,Teresa,Hill,thille@dion.ne.jp,Female,82.84.228.235
-116,Amanda,Harper,aharperf@mail.ru,Female,16.123.56.176
-117,Kimberly,Ray,krayg@xing.com,Female,48.66.48.12
-118,Johnny,Knight,jknighth@jalbum.net,Male,99.30.138.123
-119,Virginia,Freeman,vfreemani@tiny.cc,Female,225.172.182.63
-120,Anna,Austin,aaustinj@diigo.com,Female,62.111.227.148
-121,Willie,Hill,whillk@mail.ru,Male,0.86.232.249
-122,Sean,Harris,sharrisl@zdnet.com,Male,117.165.133.249
-123,Mildred,Adams,madamsm@usatoday.com,Female,163.44.97.46
-124,David,Graham,dgrahamn@zimbio.com,Male,78.13.246.202
-125,Victor,Hunter,vhuntero@ehow.com,Male,64.156.179.139
-126,Aaron,Ruiz,aruizp@weebly.com,Male,34.194.68.78
-127,Benjamin,Brooks,bbrooksq@jalbum.net,Male,20.192.189.107
-128,Lisa,Wilson,lwilsonr@japanpost.jp,Female,199.152.130.217
-129,Benjamin,King,bkings@comsenz.com,Male,29.189.189.213
-130,Christina,Williamson,cwilliamsont@boston.com,Female,194.101.52.60
-131,Jane,Gonzalez,jgonzalezu@networksolutions.com,Female,109.119.12.87
-132,Thomas,Owens,towensv@psu.edu,Male,84.168.213.153
-133,Katherine,Moore,kmoorew@naver.com,Female,183.150.65.24
-134,Jennifer,Stewart,jstewartx@yahoo.com,Female,38.41.244.58
-135,Sara,Tucker,stuckery@topsy.com,Female,181.130.59.184
-136,Harold,Ortiz,hortizz@vkontakte.ru,Male,198.231.63.137
-137,Shirley,James,sjames10@yelp.com,Female,83.27.160.104
-138,Dennis,Johnson,djohnson11@slate.com,Male,183.178.246.101
-139,Louise,Weaver,lweaver12@china.com.cn,Female,1.14.110.18
-140,Maria,Armstrong,marmstrong13@prweb.com,Female,181.142.1.249
-141,Gloria,Cruz,gcruz14@odnoklassniki.ru,Female,178.232.140.243
-142,Diana,Spencer,dspencer15@ifeng.com,Female,125.153.138.244
-143,Kelly,Nguyen,knguyen16@altervista.org,Female,170.13.201.119
-144,Jane,Rodriguez,jrodriguez17@biblegateway.com,Female,12.102.249.81
-145,Scott,Brown,sbrown18@geocities.jp,Male,108.174.99.192
-146,Norma,Cruz,ncruz19@si.edu,Female,201.112.156.197
-147,Marie,Peters,mpeters1a@mlb.com,Female,231.121.197.144
-148,Lillian,Carr,lcarr1b@typepad.com,Female,206.179.164.163
-149,Judy,Nichols,jnichols1c@t-online.de,Female,158.190.209.194
-150,Billy,Long,blong1d@yahoo.com,Male,175.20.23.160
-151,Howard,Reid,hreid1e@exblog.jp,Male,118.99.196.20
-152,Laura,Ferguson,lferguson1f@tuttocitta.it,Female,22.77.87.110
-153,Anne,Bailey,abailey1g@geocities.com,Female,58.144.159.245
-154,Rose,Morgan,rmorgan1h@ehow.com,Female,118.127.97.4
-155,Nicholas,Reyes,nreyes1i@google.ru,Male,50.135.10.252
-156,Joshua,Kennedy,jkennedy1j@house.gov,Male,154.6.163.209
-157,Paul,Watkins,pwatkins1k@upenn.edu,Male,177.236.120.87
-158,Kathryn,Kelly,kkelly1l@businessweek.com,Female,70.28.61.86
-159,Adam,Armstrong,aarmstrong1m@techcrunch.com,Male,133.235.24.202
-160,Norma,Wallace,nwallace1n@phoca.cz,Female,241.119.227.128
-161,Timothy,Reyes,treyes1o@google.cn,Male,86.28.23.26
-162,Elizabeth,Patterson,epatterson1p@sun.com,Female,139.97.159.149
-163,Edward,Gomez,egomez1q@google.fr,Male,158.103.108.255
-164,David,Cox,dcox1r@friendfeed.com,Male,206.80.80.58
-165,Brenda,Wood,bwood1s@over-blog.com,Female,217.207.44.179
-166,Adam,Walker,awalker1t@blogs.com,Male,253.211.54.93
-167,Michael,Hart,mhart1u@wix.com,Male,230.206.200.22
-168,Jesse,Ellis,jellis1v@google.co.uk,Male,213.254.162.52
-169,Janet,Powell,jpowell1w@un.org,Female,27.192.194.86
-170,Helen,Ford,hford1x@creativecommons.org,Female,52.160.102.168
-171,Gerald,Carpenter,gcarpenter1y@about.me,Male,36.30.194.218
-172,Kathryn,Oliver,koliver1z@army.mil,Female,202.63.103.69
-173,Alan,Berry,aberry20@gov.uk,Male,246.157.112.211
-174,Harry,Andrews,handrews21@ameblo.jp,Male,195.108.0.12
-175,Andrea,Hall,ahall22@hp.com,Female,149.162.163.28
-176,Barbara,Wells,bwells23@behance.net,Female,224.70.72.1
-177,Anne,Wells,awells24@apache.org,Female,180.168.81.153
-178,Harry,Harper,hharper25@rediff.com,Male,151.87.130.21
-179,Jack,Ray,jray26@wufoo.com,Male,220.109.38.178
-180,Phillip,Hamilton,phamilton27@joomla.org,Male,166.40.47.30
-181,Shirley,Hunter,shunter28@newsvine.com,Female,97.209.140.194
-182,Arthur,Daniels,adaniels29@reuters.com,Male,5.40.240.86
-183,Virginia,Rodriguez,vrodriguez2a@walmart.com,Female,96.80.164.184
-184,Christina,Ryan,cryan2b@hibu.com,Female,56.35.5.52
-185,Theresa,Mendoza,tmendoza2c@vinaora.com,Female,243.42.0.210
-186,Jason,Cole,jcole2d@ycombinator.com,Male,198.248.39.129
-187,Phillip,Bryant,pbryant2e@rediff.com,Male,140.39.116.251
-188,Adam,Torres,atorres2f@sun.com,Male,101.75.187.135
-189,Margaret,Johnston,mjohnston2g@ucsd.edu,Female,159.30.69.149
-190,Paul,Payne,ppayne2h@hhs.gov,Male,199.234.140.220
-191,Todd,Willis,twillis2i@businessweek.com,Male,191.59.136.214
-192,Willie,Oliver,woliver2j@noaa.gov,Male,44.212.35.197
-193,Frances,Robertson,frobertson2k@go.com,Female,31.117.65.136
-194,Gregory,Hawkins,ghawkins2l@joomla.org,Male,91.3.22.49
-195,Lisa,Perkins,lperkins2m@si.edu,Female,145.95.31.186
-196,Jacqueline,Anderson,janderson2n@cargocollective.com,Female,14.176.0.187
-197,Shirley,Diaz,sdiaz2o@ucla.edu,Female,207.12.95.46
-198,Nicole,Meyer,nmeyer2p@flickr.com,Female,231.79.115.13
-199,Mary,Gray,mgray2q@constantcontact.com,Female,210.116.64.253
-200,Jean,Mcdonald,jmcdonald2r@baidu.com,Female,122.239.235.117
diff --git a/tests/integration/simple_copy_test/seeds-merge-cols-initial/seed.csv b/tests/integration/simple_copy_test/seeds-merge-cols-initial/seed.csv
deleted file mode 100644
index 1333fd202..000000000
--- a/tests/integration/simple_copy_test/seeds-merge-cols-initial/seed.csv
+++ /dev/null
@@ -1,101 +0,0 @@
-load_date,id,first_name,last_name,email,gender,ip_address
-2021-03-05,1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
-2021-03-05,2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
-2021-03-05,3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
-2021-03-05,4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
-2021-03-05,5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
-2021-03-05,6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220
-2021-03-05,7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64
-2021-03-05,8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13
-2021-03-05,9,Gary,Day,gday8@nih.gov,Male,35.81.68.186
-2021-03-05,10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100
-2021-03-05,11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67
-2021-03-05,12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193
-2021-03-05,13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5
-2021-03-05,14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250
-2021-03-05,15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245
-2021-03-05,16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54
-2021-03-05,17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96
-2021-03-05,18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72
-2021-03-05,19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174
-2021-03-05,20,Andrew,Davis,adavisj@patch.com,Male,9.255.67.25
-2021-03-05,21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253
-2021-03-05,22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153
-2021-03-05,23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201
-2021-03-05,24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122
-2021-03-05,25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95
-2021-03-05,26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52
-2021-03-05,27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26
-2021-03-05,28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118
-2021-03-05,29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28
-2021-03-05,30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177
-2021-03-05,31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233
-2021-03-05,32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203
-2021-03-05,33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149
-2021-03-05,34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167
-2021-03-05,35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110
-2021-03-05,36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68
-2021-03-05,37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89
-2021-03-05,38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81
-2021-03-05,39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15
-2021-03-05,40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255
-2021-03-05,41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140
-2021-03-05,42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24
-2021-03-05,43,Sean,Mason,smason16@icq.com,Male,159.219.155.249
-2021-03-05,44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218
-2021-03-05,45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198
-2021-03-05,46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18
-2021-03-05,47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238
-2021-03-05,48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61
-2021-03-05,49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21
-2021-03-05,50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209
-2021-03-05,51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87
-2021-03-05,52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142
-2021-03-05,53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126
-2021-03-05,54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212
-2021-03-05,55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194
-2021-03-05,56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22
-2021-03-05,57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60
-2021-03-05,58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50
-2021-03-05,59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222
-2021-03-05,60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115
-2021-03-05,61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155
-2021-03-05,62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94
-2021-03-05,63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106
-2021-03-05,64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68
-2021-03-05,65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41
-2021-03-05,66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109
-2021-03-05,67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77
-2021-03-05,68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194
-2021-03-05,69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135
-2021-03-05,70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87
-2021-03-05,71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44
-2021-03-05,72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182
-2021-03-05,73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241
-2021-03-05,74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24
-2021-03-05,75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214
-2021-03-05,76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199
-2021-03-05,77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41
-2021-03-05,78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255
-2021-03-05,79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144
-2021-03-05,80,Rose,King,rking27@ucoz.com,Female,212.123.168.231
-2021-03-05,81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188
-2021-03-05,82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61
-2021-03-05,83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30
-2021-03-05,84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192
-2021-03-05,85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232
-2021-03-05,86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109
-2021-03-05,87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156
-2021-03-05,88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84
-2021-03-05,89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235
-2021-03-05,90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53
-2021-03-05,91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221
-2021-03-05,92,Angela,Rogers,arogers2j@goodreads.com,Female,98.104.132.187
-2021-03-05,93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57
-2021-03-05,94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189
-2021-03-05,95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180
-2021-03-05,96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144
-2021-03-05,97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117
-2021-03-05,98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126
-2021-03-05,99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244
-2021-03-05,100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88
diff --git a/tests/integration/simple_copy_test/seeds-merge-cols-update/expected_result.csv b/tests/integration/simple_copy_test/seeds-merge-cols-update/expected_result.csv
deleted file mode 100644
index b9e0e900b..000000000
--- a/tests/integration/simple_copy_test/seeds-merge-cols-update/expected_result.csv
+++ /dev/null
@@ -1,201 +0,0 @@
-load_date,id,first_name,last_name,email,gender,ip_address
-2021-03-05,1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
-2021-03-05,2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
-2021-03-05,3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
-2021-03-05,4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
-2021-03-05,5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
-2021-03-05,6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220
-2021-03-05,7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64
-2021-03-05,8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13
-2021-03-05,9,Gary,Day,gday8@nih.gov,Male,35.81.68.186
-2021-03-05,10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100
-2021-03-05,11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67
-2021-03-05,12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193
-2021-03-05,13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5
-2021-03-05,14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250
-2021-03-05,15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245
-2021-03-05,16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54
-2021-03-05,17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96
-2021-03-05,18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72
-2021-03-05,19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174
-2021-03-05,20,Andrew,Davis,adavisj@reddit.com,Male,9.255.67.25
-2021-03-05,21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253
-2021-03-05,22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153
-2021-03-05,23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201
-2021-03-05,24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122
-2021-03-05,25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95
-2021-03-05,26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52
-2021-03-05,27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26
-2021-03-05,28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118
-2021-03-05,29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28
-2021-03-05,30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177
-2021-03-05,31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233
-2021-03-05,32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203
-2021-03-05,33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149
-2021-03-05,34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167
-2021-03-05,35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110
-2021-03-05,36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68
-2021-03-05,37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89
-2021-03-05,38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81
-2021-03-05,39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15
-2021-03-05,40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255
-2021-03-05,41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140
-2021-03-05,42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24
-2021-03-05,43,Sean,Mason,smason16@icq.com,Male,159.219.155.249
-2021-03-05,44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218
-2021-03-05,45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198
-2021-03-05,46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18
-2021-03-05,47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238
-2021-03-05,48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61
-2021-03-05,49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21
-2021-03-05,50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209
-2021-03-05,51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87
-2021-03-05,52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142
-2021-03-05,53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126
-2021-03-05,54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212
-2021-03-05,55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194
-2021-03-05,56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22
-2021-03-05,57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60
-2021-03-05,58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50
-2021-03-05,59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222
-2021-03-05,60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115
-2021-03-05,61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155
-2021-03-05,62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94
-2021-03-05,63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106
-2021-03-05,64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68
-2021-03-05,65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41
-2021-03-05,66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109
-2021-03-05,67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77
-2021-03-05,68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194
-2021-03-05,69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135
-2021-03-05,70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87
-2021-03-05,71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44
-2021-03-05,72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182
-2021-03-05,73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241
-2021-03-05,74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24
-2021-03-05,75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214
-2021-03-05,76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199
-2021-03-05,77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41
-2021-03-05,78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255
-2021-03-05,79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144
-2021-03-05,80,Rose,King,rking27@ucoz.com,Female,212.123.168.231
-2021-03-05,81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188
-2021-03-05,82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61
-2021-03-05,83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30
-2021-03-05,84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192
-2021-03-05,85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232
-2021-03-05,86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109
-2021-03-05,87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156
-2021-03-05,88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84
-2021-03-05,89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235
-2021-03-05,90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53
-2021-03-05,91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221
-2021-03-05,92,Angela,Rogers,ascott2j@goodreads.com,Female,98.119.208.155
-2021-03-05,93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57
-2021-03-05,94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189
-2021-03-05,95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180
-2021-03-05,96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144
-2021-03-05,97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117
-2021-03-05,98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126
-2021-03-05,99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244
-2021-03-05,100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88
-2021-03-06,101,Michael,Perez,mperez0@chronoengine.com,Male,106.239.70.175
-2021-03-06,102,Shawn,Mccoy,smccoy1@reddit.com,Male,24.165.76.182
-2021-03-06,103,Kathleen,Payne,kpayne2@cargocollective.com,Female,113.207.168.106
-2021-03-06,104,Jimmy,Cooper,jcooper3@cargocollective.com,Male,198.24.63.114
-2021-03-06,105,Katherine,Rice,krice4@typepad.com,Female,36.97.186.238
-2021-03-06,106,Sarah,Ryan,sryan5@gnu.org,Female,119.117.152.40
-2021-03-06,107,Martin,Mcdonald,mmcdonald6@opera.com,Male,8.76.38.115
-2021-03-06,108,Frank,Robinson,frobinson7@wunderground.com,Male,186.14.64.194
-2021-03-06,109,Jennifer,Franklin,jfranklin8@mail.ru,Female,91.216.3.131
-2021-03-06,110,Henry,Welch,hwelch9@list-manage.com,Male,176.35.182.168
-2021-03-06,111,Fred,Snyder,fsnydera@reddit.com,Male,217.106.196.54
-2021-03-06,112,Amy,Dunn,adunnb@nba.com,Female,95.39.163.195
-2021-03-06,113,Kathleen,Meyer,kmeyerc@cdc.gov,Female,164.142.188.214
-2021-03-06,114,Steve,Ferguson,sfergusond@reverbnation.com,Male,138.22.204.251
-2021-03-06,115,Teresa,Hill,thille@dion.ne.jp,Female,82.84.228.235
-2021-03-06,116,Amanda,Harper,aharperf@mail.ru,Female,16.123.56.176
-2021-03-06,117,Kimberly,Ray,krayg@xing.com,Female,48.66.48.12
-2021-03-06,118,Johnny,Knight,jknighth@jalbum.net,Male,99.30.138.123
-2021-03-06,119,Virginia,Freeman,vfreemani@tiny.cc,Female,225.172.182.63
-2021-03-06,120,Anna,Austin,aaustinj@diigo.com,Female,62.111.227.148
-2021-03-06,121,Willie,Hill,whillk@mail.ru,Male,0.86.232.249
-2021-03-06,122,Sean,Harris,sharrisl@zdnet.com,Male,117.165.133.249
-2021-03-06,123,Mildred,Adams,madamsm@usatoday.com,Female,163.44.97.46
-2021-03-06,124,David,Graham,dgrahamn@zimbio.com,Male,78.13.246.202
-2021-03-06,125,Victor,Hunter,vhuntero@ehow.com,Male,64.156.179.139
-2021-03-06,126,Aaron,Ruiz,aruizp@weebly.com,Male,34.194.68.78
-2021-03-06,127,Benjamin,Brooks,bbrooksq@jalbum.net,Male,20.192.189.107
-2021-03-06,128,Lisa,Wilson,lwilsonr@japanpost.jp,Female,199.152.130.217
-2021-03-06,129,Benjamin,King,bkings@comsenz.com,Male,29.189.189.213
-2021-03-06,130,Christina,Williamson,cwilliamsont@boston.com,Female,194.101.52.60
-2021-03-06,131,Jane,Gonzalez,jgonzalezu@networksolutions.com,Female,109.119.12.87
-2021-03-06,132,Thomas,Owens,towensv@psu.edu,Male,84.168.213.153
-2021-03-06,133,Katherine,Moore,kmoorew@naver.com,Female,183.150.65.24
-2021-03-06,134,Jennifer,Stewart,jstewartx@yahoo.com,Female,38.41.244.58
-2021-03-06,135,Sara,Tucker,stuckery@topsy.com,Female,181.130.59.184
-2021-03-06,136,Harold,Ortiz,hortizz@vkontakte.ru,Male,198.231.63.137
-2021-03-06,137,Shirley,James,sjames10@yelp.com,Female,83.27.160.104
-2021-03-06,138,Dennis,Johnson,djohnson11@slate.com,Male,183.178.246.101
-2021-03-06,139,Louise,Weaver,lweaver12@china.com.cn,Female,1.14.110.18
-2021-03-06,140,Maria,Armstrong,marmstrong13@prweb.com,Female,181.142.1.249
-2021-03-06,141,Gloria,Cruz,gcruz14@odnoklassniki.ru,Female,178.232.140.243
-2021-03-06,142,Diana,Spencer,dspencer15@ifeng.com,Female,125.153.138.244
-2021-03-06,143,Kelly,Nguyen,knguyen16@altervista.org,Female,170.13.201.119
-2021-03-06,144,Jane,Rodriguez,jrodriguez17@biblegateway.com,Female,12.102.249.81
-2021-03-06,145,Scott,Brown,sbrown18@geocities.jp,Male,108.174.99.192
-2021-03-06,146,Norma,Cruz,ncruz19@si.edu,Female,201.112.156.197
-2021-03-06,147,Marie,Peters,mpeters1a@mlb.com,Female,231.121.197.144
-2021-03-06,148,Lillian,Carr,lcarr1b@typepad.com,Female,206.179.164.163
-2021-03-06,149,Judy,Nichols,jnichols1c@t-online.de,Female,158.190.209.194
-2021-03-06,150,Billy,Long,blong1d@yahoo.com,Male,175.20.23.160
-2021-03-06,151,Howard,Reid,hreid1e@exblog.jp,Male,118.99.196.20
-2021-03-06,152,Laura,Ferguson,lferguson1f@tuttocitta.it,Female,22.77.87.110
-2021-03-06,153,Anne,Bailey,abailey1g@geocities.com,Female,58.144.159.245
-2021-03-06,154,Rose,Morgan,rmorgan1h@ehow.com,Female,118.127.97.4
-2021-03-06,155,Nicholas,Reyes,nreyes1i@google.ru,Male,50.135.10.252
-2021-03-06,156,Joshua,Kennedy,jkennedy1j@house.gov,Male,154.6.163.209
-2021-03-06,157,Paul,Watkins,pwatkins1k@upenn.edu,Male,177.236.120.87
-2021-03-06,158,Kathryn,Kelly,kkelly1l@businessweek.com,Female,70.28.61.86
-2021-03-06,159,Adam,Armstrong,aarmstrong1m@techcrunch.com,Male,133.235.24.202
-2021-03-06,160,Norma,Wallace,nwallace1n@phoca.cz,Female,241.119.227.128
-2021-03-06,161,Timothy,Reyes,treyes1o@google.cn,Male,86.28.23.26
-2021-03-06,162,Elizabeth,Patterson,epatterson1p@sun.com,Female,139.97.159.149
-2021-03-06,163,Edward,Gomez,egomez1q@google.fr,Male,158.103.108.255
-2021-03-06,164,David,Cox,dcox1r@friendfeed.com,Male,206.80.80.58
-2021-03-06,165,Brenda,Wood,bwood1s@over-blog.com,Female,217.207.44.179
-2021-03-06,166,Adam,Walker,awalker1t@blogs.com,Male,253.211.54.93
-2021-03-06,167,Michael,Hart,mhart1u@wix.com,Male,230.206.200.22
-2021-03-06,168,Jesse,Ellis,jellis1v@google.co.uk,Male,213.254.162.52
-2021-03-06,169,Janet,Powell,jpowell1w@un.org,Female,27.192.194.86
-2021-03-06,170,Helen,Ford,hford1x@creativecommons.org,Female,52.160.102.168
-2021-03-06,171,Gerald,Carpenter,gcarpenter1y@about.me,Male,36.30.194.218
-2021-03-06,172,Kathryn,Oliver,koliver1z@army.mil,Female,202.63.103.69
-2021-03-06,173,Alan,Berry,aberry20@gov.uk,Male,246.157.112.211
-2021-03-06,174,Harry,Andrews,handrews21@ameblo.jp,Male,195.108.0.12
-2021-03-06,175,Andrea,Hall,ahall22@hp.com,Female,149.162.163.28
-2021-03-06,176,Barbara,Wells,bwells23@behance.net,Female,224.70.72.1
-2021-03-06,177,Anne,Wells,awells24@apache.org,Female,180.168.81.153
-2021-03-06,178,Harry,Harper,hharper25@rediff.com,Male,151.87.130.21
-2021-03-06,179,Jack,Ray,jray26@wufoo.com,Male,220.109.38.178
-2021-03-06,180,Phillip,Hamilton,phamilton27@joomla.org,Male,166.40.47.30
-2021-03-06,181,Shirley,Hunter,shunter28@newsvine.com,Female,97.209.140.194
-2021-03-06,182,Arthur,Daniels,adaniels29@reuters.com,Male,5.40.240.86
-2021-03-06,183,Virginia,Rodriguez,vrodriguez2a@walmart.com,Female,96.80.164.184
-2021-03-06,184,Christina,Ryan,cryan2b@hibu.com,Female,56.35.5.52
-2021-03-06,185,Theresa,Mendoza,tmendoza2c@vinaora.com,Female,243.42.0.210
-2021-03-06,186,Jason,Cole,jcole2d@ycombinator.com,Male,198.248.39.129
-2021-03-06,187,Phillip,Bryant,pbryant2e@rediff.com,Male,140.39.116.251
-2021-03-06,188,Adam,Torres,atorres2f@sun.com,Male,101.75.187.135
-2021-03-06,189,Margaret,Johnston,mjohnston2g@ucsd.edu,Female,159.30.69.149
-2021-03-06,190,Paul,Payne,ppayne2h@hhs.gov,Male,199.234.140.220
-2021-03-06,191,Todd,Willis,twillis2i@businessweek.com,Male,191.59.136.214
-2021-03-06,192,Willie,Oliver,woliver2j@noaa.gov,Male,44.212.35.197
-2021-03-06,193,Frances,Robertson,frobertson2k@go.com,Female,31.117.65.136
-2021-03-06,194,Gregory,Hawkins,ghawkins2l@joomla.org,Male,91.3.22.49
-2021-03-06,195,Lisa,Perkins,lperkins2m@si.edu,Female,145.95.31.186
-2021-03-06,196,Jacqueline,Anderson,janderson2n@cargocollective.com,Female,14.176.0.187
-2021-03-06,197,Shirley,Diaz,sdiaz2o@ucla.edu,Female,207.12.95.46
-2021-03-06,198,Nicole,Meyer,nmeyer2p@flickr.com,Female,231.79.115.13
-2021-03-06,199,Mary,Gray,mgray2q@constantcontact.com,Female,210.116.64.253
-2021-03-06,200,Jean,Mcdonald,jmcdonald2r@baidu.com,Female,122.239.235.117
diff --git a/tests/integration/simple_copy_test/seeds-merge-cols-update/seed.csv b/tests/integration/simple_copy_test/seeds-merge-cols-update/seed.csv
deleted file mode 100644
index 2a2ff674c..000000000
--- a/tests/integration/simple_copy_test/seeds-merge-cols-update/seed.csv
+++ /dev/null
@@ -1,204 +0,0 @@
-load_date,id,first_name,last_name,email,gender,ip_address
-2021-03-05,1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
-2021-03-05,2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
-2021-03-05,3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
-2021-03-05,4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
-2021-03-05,5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
-2021-03-05,6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220
-2021-03-05,7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64
-2021-03-05,8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13
-2021-03-05,9,Gary,Day,gday8@nih.gov,Male,35.81.68.186
-2021-03-05,10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100
-2021-03-05,11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67
-2021-03-05,12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193
-2021-03-05,13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5
-2021-03-05,14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250
-2021-03-05,15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245
-2021-03-05,16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54
-2021-03-05,17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96
-2021-03-05,18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72
-2021-03-05,19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174
-2021-03-05,20,Andrew,Davis,adavisj@patch.com,Male,9.255.67.25
-2021-03-05,21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253
-2021-03-05,22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153
-2021-03-05,23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201
-2021-03-05,24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122
-2021-03-05,25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95
-2021-03-05,26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52
-2021-03-05,27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26
-2021-03-05,28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118
-2021-03-05,29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28
-2021-03-05,30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177
-2021-03-05,31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233
-2021-03-05,32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203
-2021-03-05,33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149
-2021-03-05,34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167
-2021-03-05,35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110
-2021-03-05,36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68
-2021-03-05,37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89
-2021-03-05,38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81
-2021-03-05,39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15
-2021-03-05,40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255
-2021-03-05,41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140
-2021-03-05,42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24
-2021-03-05,43,Sean,Mason,smason16@icq.com,Male,159.219.155.249
-2021-03-05,44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218
-2021-03-05,45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198
-2021-03-05,46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18
-2021-03-05,47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238
-2021-03-05,48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61
-2021-03-05,49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21
-2021-03-05,50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209
-2021-03-05,51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87
-2021-03-05,52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142
-2021-03-05,53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126
-2021-03-05,54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212
-2021-03-05,55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194
-2021-03-05,56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22
-2021-03-05,57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60
-2021-03-05,58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50
-2021-03-05,59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222
-2021-03-05,60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115
-2021-03-05,61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155
-2021-03-05,62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94
-2021-03-05,63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106
-2021-03-05,64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68
-2021-03-05,65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41
-2021-03-05,66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109
-2021-03-05,67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77
-2021-03-05,68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194
-2021-03-05,69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135
-2021-03-05,70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87
-2021-03-05,71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44
-2021-03-05,72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182
-2021-03-05,73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241
-2021-03-05,74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24
-2021-03-05,75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214
-2021-03-05,76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199
-2021-03-05,77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41
-2021-03-05,78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255
-2021-03-05,79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144
-2021-03-05,80,Rose,King,rking27@ucoz.com,Female,212.123.168.231
-2021-03-05,81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188
-2021-03-05,82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61
-2021-03-05,83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30
-2021-03-05,84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192
-2021-03-05,85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232
-2021-03-05,86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109
-2021-03-05,87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156
-2021-03-05,88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84
-2021-03-05,89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235
-2021-03-05,90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53
-2021-03-05,91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221
-2021-03-05,92,Angela,Rogers,arogers2j@goodreads.com,Female,98.104.132.187
-2021-03-05,93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57
-2021-03-05,94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189
-2021-03-05,95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180
-2021-03-05,96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144
-2021-03-05,97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117
-2021-03-05,98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126
-2021-03-05,99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244
-2021-03-05,100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88
-2021-03-06,20,Andrew,Davis,adavisj@reddit.com,Male,9.255.67.25
-2021-03-06,83,Josh,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30
-2021-03-06,92,Angela,Scott,ascott2j@goodreads.com,Female,98.119.208.155
-2021-03-06,101,Michael,Perez,mperez0@chronoengine.com,Male,106.239.70.175
-2021-03-06,102,Shawn,Mccoy,smccoy1@reddit.com,Male,24.165.76.182
-2021-03-06,103,Kathleen,Payne,kpayne2@cargocollective.com,Female,113.207.168.106
-2021-03-06,104,Jimmy,Cooper,jcooper3@cargocollective.com,Male,198.24.63.114
-2021-03-06,105,Katherine,Rice,krice4@typepad.com,Female,36.97.186.238
-2021-03-06,106,Sarah,Ryan,sryan5@gnu.org,Female,119.117.152.40
-2021-03-06,107,Martin,Mcdonald,mmcdonald6@opera.com,Male,8.76.38.115
-2021-03-06,108,Frank,Robinson,frobinson7@wunderground.com,Male,186.14.64.194
-2021-03-06,109,Jennifer,Franklin,jfranklin8@mail.ru,Female,91.216.3.131
-2021-03-06,110,Henry,Welch,hwelch9@list-manage.com,Male,176.35.182.168
-2021-03-06,111,Fred,Snyder,fsnydera@reddit.com,Male,217.106.196.54
-2021-03-06,112,Amy,Dunn,adunnb@nba.com,Female,95.39.163.195
-2021-03-06,113,Kathleen,Meyer,kmeyerc@cdc.gov,Female,164.142.188.214
-2021-03-06,114,Steve,Ferguson,sfergusond@reverbnation.com,Male,138.22.204.251
-2021-03-06,115,Teresa,Hill,thille@dion.ne.jp,Female,82.84.228.235
-2021-03-06,116,Amanda,Harper,aharperf@mail.ru,Female,16.123.56.176
-2021-03-06,117,Kimberly,Ray,krayg@xing.com,Female,48.66.48.12
-2021-03-06,118,Johnny,Knight,jknighth@jalbum.net,Male,99.30.138.123
-2021-03-06,119,Virginia,Freeman,vfreemani@tiny.cc,Female,225.172.182.63
-2021-03-06,120,Anna,Austin,aaustinj@diigo.com,Female,62.111.227.148
-2021-03-06,121,Willie,Hill,whillk@mail.ru,Male,0.86.232.249
-2021-03-06,122,Sean,Harris,sharrisl@zdnet.com,Male,117.165.133.249
-2021-03-06,123,Mildred,Adams,madamsm@usatoday.com,Female,163.44.97.46
-2021-03-06,124,David,Graham,dgrahamn@zimbio.com,Male,78.13.246.202
-2021-03-06,125,Victor,Hunter,vhuntero@ehow.com,Male,64.156.179.139
-2021-03-06,126,Aaron,Ruiz,aruizp@weebly.com,Male,34.194.68.78
-2021-03-06,127,Benjamin,Brooks,bbrooksq@jalbum.net,Male,20.192.189.107
-2021-03-06,128,Lisa,Wilson,lwilsonr@japanpost.jp,Female,199.152.130.217
-2021-03-06,129,Benjamin,King,bkings@comsenz.com,Male,29.189.189.213
-2021-03-06,130,Christina,Williamson,cwilliamsont@boston.com,Female,194.101.52.60
-2021-03-06,131,Jane,Gonzalez,jgonzalezu@networksolutions.com,Female,109.119.12.87
-2021-03-06,132,Thomas,Owens,towensv@psu.edu,Male,84.168.213.153
-2021-03-06,133,Katherine,Moore,kmoorew@naver.com,Female,183.150.65.24
-2021-03-06,134,Jennifer,Stewart,jstewartx@yahoo.com,Female,38.41.244.58
-2021-03-06,135,Sara,Tucker,stuckery@topsy.com,Female,181.130.59.184
-2021-03-06,136,Harold,Ortiz,hortizz@vkontakte.ru,Male,198.231.63.137
-2021-03-06,137,Shirley,James,sjames10@yelp.com,Female,83.27.160.104
-2021-03-06,138,Dennis,Johnson,djohnson11@slate.com,Male,183.178.246.101
-2021-03-06,139,Louise,Weaver,lweaver12@china.com.cn,Female,1.14.110.18
-2021-03-06,140,Maria,Armstrong,marmstrong13@prweb.com,Female,181.142.1.249
-2021-03-06,141,Gloria,Cruz,gcruz14@odnoklassniki.ru,Female,178.232.140.243
-2021-03-06,142,Diana,Spencer,dspencer15@ifeng.com,Female,125.153.138.244
-2021-03-06,143,Kelly,Nguyen,knguyen16@altervista.org,Female,170.13.201.119
-2021-03-06,144,Jane,Rodriguez,jrodriguez17@biblegateway.com,Female,12.102.249.81
-2021-03-06,145,Scott,Brown,sbrown18@geocities.jp,Male,108.174.99.192
-2021-03-06,146,Norma,Cruz,ncruz19@si.edu,Female,201.112.156.197
-2021-03-06,147,Marie,Peters,mpeters1a@mlb.com,Female,231.121.197.144
-2021-03-06,148,Lillian,Carr,lcarr1b@typepad.com,Female,206.179.164.163
-2021-03-06,149,Judy,Nichols,jnichols1c@t-online.de,Female,158.190.209.194
-2021-03-06,150,Billy,Long,blong1d@yahoo.com,Male,175.20.23.160
-2021-03-06,151,Howard,Reid,hreid1e@exblog.jp,Male,118.99.196.20
-2021-03-06,152,Laura,Ferguson,lferguson1f@tuttocitta.it,Female,22.77.87.110
-2021-03-06,153,Anne,Bailey,abailey1g@geocities.com,Female,58.144.159.245
-2021-03-06,154,Rose,Morgan,rmorgan1h@ehow.com,Female,118.127.97.4
-2021-03-06,155,Nicholas,Reyes,nreyes1i@google.ru,Male,50.135.10.252
-2021-03-06,156,Joshua,Kennedy,jkennedy1j@house.gov,Male,154.6.163.209
-2021-03-06,157,Paul,Watkins,pwatkins1k@upenn.edu,Male,177.236.120.87
-2021-03-06,158,Kathryn,Kelly,kkelly1l@businessweek.com,Female,70.28.61.86
-2021-03-06,159,Adam,Armstrong,aarmstrong1m@techcrunch.com,Male,133.235.24.202
-2021-03-06,160,Norma,Wallace,nwallace1n@phoca.cz,Female,241.119.227.128
-2021-03-06,161,Timothy,Reyes,treyes1o@google.cn,Male,86.28.23.26
-2021-03-06,162,Elizabeth,Patterson,epatterson1p@sun.com,Female,139.97.159.149
-2021-03-06,163,Edward,Gomez,egomez1q@google.fr,Male,158.103.108.255
-2021-03-06,164,David,Cox,dcox1r@friendfeed.com,Male,206.80.80.58
-2021-03-06,165,Brenda,Wood,bwood1s@over-blog.com,Female,217.207.44.179
-2021-03-06,166,Adam,Walker,awalker1t@blogs.com,Male,253.211.54.93
-2021-03-06,167,Michael,Hart,mhart1u@wix.com,Male,230.206.200.22
-2021-03-06,168,Jesse,Ellis,jellis1v@google.co.uk,Male,213.254.162.52
-2021-03-06,169,Janet,Powell,jpowell1w@un.org,Female,27.192.194.86
-2021-03-06,170,Helen,Ford,hford1x@creativecommons.org,Female,52.160.102.168
-2021-03-06,171,Gerald,Carpenter,gcarpenter1y@about.me,Male,36.30.194.218
-2021-03-06,172,Kathryn,Oliver,koliver1z@army.mil,Female,202.63.103.69
-2021-03-06,173,Alan,Berry,aberry20@gov.uk,Male,246.157.112.211
-2021-03-06,174,Harry,Andrews,handrews21@ameblo.jp,Male,195.108.0.12
-2021-03-06,175,Andrea,Hall,ahall22@hp.com,Female,149.162.163.28
-2021-03-06,176,Barbara,Wells,bwells23@behance.net,Female,224.70.72.1
-2021-03-06,177,Anne,Wells,awells24@apache.org,Female,180.168.81.153
-2021-03-06,178,Harry,Harper,hharper25@rediff.com,Male,151.87.130.21
-2021-03-06,179,Jack,Ray,jray26@wufoo.com,Male,220.109.38.178
-2021-03-06,180,Phillip,Hamilton,phamilton27@joomla.org,Male,166.40.47.30
-2021-03-06,181,Shirley,Hunter,shunter28@newsvine.com,Female,97.209.140.194
-2021-03-06,182,Arthur,Daniels,adaniels29@reuters.com,Male,5.40.240.86
-2021-03-06,183,Virginia,Rodriguez,vrodriguez2a@walmart.com,Female,96.80.164.184
-2021-03-06,184,Christina,Ryan,cryan2b@hibu.com,Female,56.35.5.52
-2021-03-06,185,Theresa,Mendoza,tmendoza2c@vinaora.com,Female,243.42.0.210
-2021-03-06,186,Jason,Cole,jcole2d@ycombinator.com,Male,198.248.39.129
-2021-03-06,187,Phillip,Bryant,pbryant2e@rediff.com,Male,140.39.116.251
-2021-03-06,188,Adam,Torres,atorres2f@sun.com,Male,101.75.187.135
-2021-03-06,189,Margaret,Johnston,mjohnston2g@ucsd.edu,Female,159.30.69.149
-2021-03-06,190,Paul,Payne,ppayne2h@hhs.gov,Male,199.234.140.220
-2021-03-06,191,Todd,Willis,twillis2i@businessweek.com,Male,191.59.136.214
-2021-03-06,192,Willie,Oliver,woliver2j@noaa.gov,Male,44.212.35.197
-2021-03-06,193,Frances,Robertson,frobertson2k@go.com,Female,31.117.65.136
-2021-03-06,194,Gregory,Hawkins,ghawkins2l@joomla.org,Male,91.3.22.49
-2021-03-06,195,Lisa,Perkins,lperkins2m@si.edu,Female,145.95.31.186
-2021-03-06,196,Jacqueline,Anderson,janderson2n@cargocollective.com,Female,14.176.0.187
-2021-03-06,197,Shirley,Diaz,sdiaz2o@ucla.edu,Female,207.12.95.46
-2021-03-06,198,Nicole,Meyer,nmeyer2p@flickr.com,Female,231.79.115.13
-2021-03-06,199,Mary,Gray,mgray2q@constantcontact.com,Female,210.116.64.253
-2021-03-06,200,Jean,Mcdonald,jmcdonald2r@baidu.com,Female,122.239.235.117
diff --git a/tests/integration/simple_copy_test/test_simple_copy.py b/tests/integration/simple_copy_test/test_simple_copy.py
deleted file mode 100644
index 47c7665c8..000000000
--- a/tests/integration/simple_copy_test/test_simple_copy.py
+++ /dev/null
@@ -1,96 +0,0 @@
-import json
-import os
-from pytest import mark
-
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-
-class BaseTestSimpleCopy(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "simple_copy"
-
-    @staticmethod
-    def dir(path):
-        return path.lstrip('/')
-
-    @property
-    def models(self):
-        return self.dir("models")
-
-    @property
-    def project_config(self):
-        return self.seed_quote_cfg_with({
-            'profile': '{{ "tes" ~ "t" }}'
-        })
-
-    def seed_quote_cfg_with(self, extra):
-        cfg = {
-            'config-version': 2,
-            'seeds': {
-                'quote_columns': False,
-            }
-        }
-        cfg.update(extra)
-        return cfg
-
-
-class TestSimpleCopy(BaseTestSimpleCopy):
-
-    @property
-    def project_config(self):
-        return self.seed_quote_cfg_with({"seed-paths": [self.dir("seed-initial")]})
-
-    @use_profile("bigquery")
-    def test__bigquery__simple_copy(self):
-        results = self.run_dbt(["seed"])
-        self.assertEqual(len(results),  1)
-        results = self.run_dbt()
-        self.assertEqual(len(results),  7)
-
-        self.assertTablesEqual("seed", "view_model")
-        self.assertTablesEqual("seed", "incremental")
-        self.assertTablesEqual("seed", "materialized")
-        self.assertTablesEqual("seed", "get_and_ref")
-
-        self.use_default_project({"seed-paths": [self.dir("seed-update")]})
-
-        results = self.run_dbt(["seed"])
-        self.assertEqual(len(results),  1)
-        results = self.run_dbt()
-        self.assertEqual(len(results),  7)
-
-        self.assertTablesEqual("seed", "view_model")
-        self.assertTablesEqual("seed", "incremental")
-        self.assertTablesEqual("seed", "materialized")
-        self.assertTablesEqual("seed", "get_and_ref")
-
-
-class TestIncrementalMergeColumns(BaseTestSimpleCopy):
-    @property
-    def models(self):
-        return self.dir("models-merge-update")
-
-    @property
-    def project_config(self):
-        return {
-            "seeds": {
-                "quote_columns": False
-            }
-        }
-
-    def seed_and_run(self):
-        self.run_dbt(["seed"])
-        self.run_dbt(["run"])
-
-    @use_profile("bigquery")
-    def test__bigquery__incremental_merge_columns(self):
-        self.use_default_project({
-            "seed-paths": ["seeds-merge-cols-initial"]
-        })
-        self.seed_and_run()
-        self.use_default_project({
-            "seed-paths": ["seeds-merge-cols-update"]
-        })
-        self.seed_and_run()
-        self.assertTablesEqual("incremental_update_cols", "expected_result")
\ No newline at end of file
diff --git a/tests/integration/statement_test/models-bq/statement_actual.sql b/tests/integration/statement_test/models-bq/statement_actual.sql
deleted file mode 100644
index 92f9ab1ab..000000000
--- a/tests/integration/statement_test/models-bq/statement_actual.sql
+++ /dev/null
@@ -1,23 +0,0 @@
-
--- {{ ref('seed') }}
-
-{%- call statement('test_statement', fetch_result=True) -%}
-
-  select
-    count(*) as `num_records`
-
-  from {{ ref('seed') }}
-
-{%- endcall -%}
-
-{% set result = load_result('test_statement') %}
-
-{% set res_table = result['table'] %}
-{% set res_matrix = result['data'] %}
-
-{% set matrix_value = res_matrix[0][0] %}
-{% set table_value = res_table[0]['num_records'] %}
-
-select 'matrix' as source, {{ matrix_value }} as value
-union all
-select 'table' as source, {{ table_value }} as value
diff --git a/tests/integration/statement_test/seed/seed.csv b/tests/integration/statement_test/seed/seed.csv
deleted file mode 100644
index 640af6c4e..000000000
--- a/tests/integration/statement_test/seed/seed.csv
+++ /dev/null
@@ -1,101 +0,0 @@
-id,first_name,last_name,email,gender,ip_address
-1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
-2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
-3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
-4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
-5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
-6,Jacqueline,Griffin,jgriffin5@t.co,Female,16.13.192.220
-7,Wanda,Arnold,warnold6@google.nl,Female,232.116.150.64
-8,Craig,Ortiz,cortiz7@sciencedaily.com,Male,199.126.106.13
-9,Gary,Day,gday8@nih.gov,Male,35.81.68.186
-10,Rose,Wright,rwright9@yahoo.co.jp,Female,236.82.178.100
-11,Raymond,Kelley,rkelleya@fc2.com,Male,213.65.166.67
-12,Gerald,Robinson,grobinsonb@disqus.com,Male,72.232.194.193
-13,Mildred,Martinez,mmartinezc@samsung.com,Female,198.29.112.5
-14,Dennis,Arnold,darnoldd@google.com,Male,86.96.3.250
-15,Judy,Gray,jgraye@opensource.org,Female,79.218.162.245
-16,Theresa,Garza,tgarzaf@epa.gov,Female,21.59.100.54
-17,Gerald,Robertson,grobertsong@csmonitor.com,Male,131.134.82.96
-18,Philip,Hernandez,phernandezh@adobe.com,Male,254.196.137.72
-19,Julia,Gonzalez,jgonzalezi@cam.ac.uk,Female,84.240.227.174
-20,Andrew,Davis,adavisj@patch.com,Male,9.255.67.25
-21,Kimberly,Harper,kharperk@foxnews.com,Female,198.208.120.253
-22,Mark,Martin,mmartinl@marketwatch.com,Male,233.138.182.153
-23,Cynthia,Ruiz,cruizm@google.fr,Female,18.178.187.201
-24,Samuel,Carroll,scarrolln@youtu.be,Male,128.113.96.122
-25,Jennifer,Larson,jlarsono@vinaora.com,Female,98.234.85.95
-26,Ashley,Perry,aperryp@rakuten.co.jp,Female,247.173.114.52
-27,Howard,Rodriguez,hrodriguezq@shutterfly.com,Male,231.188.95.26
-28,Amy,Brooks,abrooksr@theatlantic.com,Female,141.199.174.118
-29,Louise,Warren,lwarrens@adobe.com,Female,96.105.158.28
-30,Tina,Watson,twatsont@myspace.com,Female,251.142.118.177
-31,Janice,Kelley,jkelleyu@creativecommons.org,Female,239.167.34.233
-32,Terry,Mccoy,tmccoyv@bravesites.com,Male,117.201.183.203
-33,Jeffrey,Morgan,jmorganw@surveymonkey.com,Male,78.101.78.149
-34,Louis,Harvey,lharveyx@sina.com.cn,Male,51.50.0.167
-35,Philip,Miller,pmillery@samsung.com,Male,103.255.222.110
-36,Willie,Marshall,wmarshallz@ow.ly,Male,149.219.91.68
-37,Patrick,Lopez,plopez10@redcross.org,Male,250.136.229.89
-38,Adam,Jenkins,ajenkins11@harvard.edu,Male,7.36.112.81
-39,Benjamin,Cruz,bcruz12@linkedin.com,Male,32.38.98.15
-40,Ruby,Hawkins,rhawkins13@gmpg.org,Female,135.171.129.255
-41,Carlos,Barnes,cbarnes14@a8.net,Male,240.197.85.140
-42,Ruby,Griffin,rgriffin15@bravesites.com,Female,19.29.135.24
-43,Sean,Mason,smason16@icq.com,Male,159.219.155.249
-44,Anthony,Payne,apayne17@utexas.edu,Male,235.168.199.218
-45,Steve,Cruz,scruz18@pcworld.com,Male,238.201.81.198
-46,Anthony,Garcia,agarcia19@flavors.me,Male,25.85.10.18
-47,Doris,Lopez,dlopez1a@sphinn.com,Female,245.218.51.238
-48,Susan,Nichols,snichols1b@freewebs.com,Female,199.99.9.61
-49,Wanda,Ferguson,wferguson1c@yahoo.co.jp,Female,236.241.135.21
-50,Andrea,Pierce,apierce1d@google.co.uk,Female,132.40.10.209
-51,Lawrence,Phillips,lphillips1e@jugem.jp,Male,72.226.82.87
-52,Judy,Gilbert,jgilbert1f@multiply.com,Female,196.250.15.142
-53,Eric,Williams,ewilliams1g@joomla.org,Male,222.202.73.126
-54,Ralph,Romero,rromero1h@sogou.com,Male,123.184.125.212
-55,Jean,Wilson,jwilson1i@ocn.ne.jp,Female,176.106.32.194
-56,Lori,Reynolds,lreynolds1j@illinois.edu,Female,114.181.203.22
-57,Donald,Moreno,dmoreno1k@bbc.co.uk,Male,233.249.97.60
-58,Steven,Berry,sberry1l@eepurl.com,Male,186.193.50.50
-59,Theresa,Shaw,tshaw1m@people.com.cn,Female,120.37.71.222
-60,John,Stephens,jstephens1n@nationalgeographic.com,Male,191.87.127.115
-61,Richard,Jacobs,rjacobs1o@state.tx.us,Male,66.210.83.155
-62,Andrew,Lawson,alawson1p@over-blog.com,Male,54.98.36.94
-63,Peter,Morgan,pmorgan1q@rambler.ru,Male,14.77.29.106
-64,Nicole,Garrett,ngarrett1r@zimbio.com,Female,21.127.74.68
-65,Joshua,Kim,jkim1s@edublogs.org,Male,57.255.207.41
-66,Ralph,Roberts,rroberts1t@people.com.cn,Male,222.143.131.109
-67,George,Montgomery,gmontgomery1u@smugmug.com,Male,76.75.111.77
-68,Gerald,Alvarez,galvarez1v@flavors.me,Male,58.157.186.194
-69,Donald,Olson,dolson1w@whitehouse.gov,Male,69.65.74.135
-70,Carlos,Morgan,cmorgan1x@pbs.org,Male,96.20.140.87
-71,Aaron,Stanley,astanley1y@webnode.com,Male,163.119.217.44
-72,Virginia,Long,vlong1z@spiegel.de,Female,204.150.194.182
-73,Robert,Berry,rberry20@tripadvisor.com,Male,104.19.48.241
-74,Antonio,Brooks,abrooks21@unesco.org,Male,210.31.7.24
-75,Ruby,Garcia,rgarcia22@ovh.net,Female,233.218.162.214
-76,Jack,Hanson,jhanson23@blogtalkradio.com,Male,31.55.46.199
-77,Kathryn,Nelson,knelson24@walmart.com,Female,14.189.146.41
-78,Jason,Reed,jreed25@printfriendly.com,Male,141.189.89.255
-79,George,Coleman,gcoleman26@people.com.cn,Male,81.189.221.144
-80,Rose,King,rking27@ucoz.com,Female,212.123.168.231
-81,Johnny,Holmes,jholmes28@boston.com,Male,177.3.93.188
-82,Katherine,Gilbert,kgilbert29@altervista.org,Female,199.215.169.61
-83,Joshua,Thomas,jthomas2a@ustream.tv,Male,0.8.205.30
-84,Julie,Perry,jperry2b@opensource.org,Female,60.116.114.192
-85,Richard,Perry,rperry2c@oracle.com,Male,181.125.70.232
-86,Kenneth,Ruiz,kruiz2d@wikimedia.org,Male,189.105.137.109
-87,Jose,Morgan,jmorgan2e@webnode.com,Male,101.134.215.156
-88,Donald,Campbell,dcampbell2f@goo.ne.jp,Male,102.120.215.84
-89,Debra,Collins,dcollins2g@uol.com.br,Female,90.13.153.235
-90,Jesse,Johnson,jjohnson2h@stumbleupon.com,Male,225.178.125.53
-91,Elizabeth,Stone,estone2i@histats.com,Female,123.184.126.221
-92,Angela,Rogers,arogers2j@goodreads.com,Female,98.104.132.187
-93,Emily,Dixon,edixon2k@mlb.com,Female,39.190.75.57
-94,Albert,Scott,ascott2l@tinypic.com,Male,40.209.13.189
-95,Barbara,Peterson,bpeterson2m@ow.ly,Female,75.249.136.180
-96,Adam,Greene,agreene2n@fastcompany.com,Male,184.173.109.144
-97,Earl,Sanders,esanders2o@hc360.com,Male,247.34.90.117
-98,Angela,Brooks,abrooks2p@mtv.com,Female,10.63.249.126
-99,Harold,Foster,hfoster2q@privacy.gov.au,Male,139.214.40.244
-100,Carl,Meyer,cmeyer2r@disqus.com,Male,204.117.7.88
diff --git a/tests/integration/statement_test/seed/statement_expected.csv b/tests/integration/statement_test/seed/statement_expected.csv
deleted file mode 100644
index cf9d9af15..000000000
--- a/tests/integration/statement_test/seed/statement_expected.csv
+++ /dev/null
@@ -1,3 +0,0 @@
-source,value
-matrix,100
-table,100
diff --git a/tests/integration/statement_test/test_statements.py b/tests/integration/statement_test/test_statements.py
deleted file mode 100644
index 620ca239b..000000000
--- a/tests/integration/statement_test/test_statements.py
+++ /dev/null
@@ -1,36 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-
-class TestStatementsBigquery(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "statements"
-
-    @staticmethod
-    def dir(path):
-        return path.lstrip("/")
-
-    @property
-    def models(self):
-        return self.dir("models-bq")
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seeds': {
-                'quote_columns': False,
-            }
-        }
-
-    @use_profile("bigquery")
-    def test_bigquery_statements(self):
-        self.use_default_project({"seed-paths": [self.dir("seed")]})
-
-        results = self.run_dbt(["seed"])
-        self.assertEqual(len(results), 2)
-        results = self.run_dbt()
-        self.assertEqual(len(results), 1)
-
-        self.assertTablesEqual("statement_actual", "statement_expected")

From f9532a5679849e3fd1a5bbdeefbd4cfd8267943d Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Thu, 23 Mar 2023 13:07:36 -0700
Subject: [PATCH 591/860] See about adding dev requirements to test runner
 (#623)

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .github/workflows/integration.yml | 1 +
 1 file changed, 1 insertion(+)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index ce403235d..7be34358f 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -90,6 +90,7 @@ jobs:
             bigquery:
               - 'dbt/**'
               - 'tests/**'
+              - 'dev-requirements.txt'
       - name: Generate integration test matrix
         id: generate-matrix
         uses: actions/github-script@v4

From 98f302649f3611875590dacfb0f30f2bba4f7fea Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 24 Mar 2023 11:23:54 -0700
Subject: [PATCH 592/860] Update pre-commit requirement from ~=2.21 to ~=3.2
 (#617)

Updates the requirements on [pre-commit](https://github.com/pre-commit/pre-commit) to permit the latest version.
- [Release notes](https://github.com/pre-commit/pre-commit/releases)
- [Changelog](https://github.com/pre-commit/pre-commit/blob/main/CHANGELOG.md)
- [Commits](https://github.com/pre-commit/pre-commit/commits/v3.2.0)

---
updated-dependencies:
- dependency-name: pre-commit
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mila Page <67295367+VersusFacit@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index 7f1eb9a67..c8aacf182 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -16,7 +16,7 @@ ipdb~=0.13.13
 mypy==1.1.1  # patch updates have historically introduced breaking changes
 pip-tools~=6.12
 pre-commit~=2.21;python_version=="3.7"
-pre-commit~=3.1;python_version>="3.8"
+pre-commit~=3.2;python_version>="3.8"
 pre-commit-hooks~=4.4
 pytest~=7.2
 pytest-csv~=3.0

From c6a7c8d6fd2723e4e27742beb1c702bb72d88093 Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Fri, 24 Mar 2023 14:39:51 -0700
Subject: [PATCH 593/860] Fix test (#628)

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 tests/unit/test_bigquery_adapter.py | 29 ++++++++++++++---------------
 1 file changed, 14 insertions(+), 15 deletions(-)

diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 1277d64f4..28aca8db8 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -1,4 +1,3 @@
-from unicodedata import name
 import agate
 import decimal
 import json
@@ -13,8 +12,6 @@
 
 import dbt.dataclass_schema
 
-import dbt.flags as flags
-
 from dbt.adapters.bigquery import BigQueryCredentials
 from dbt.adapters.bigquery import BigQueryAdapter
 from dbt.adapters.bigquery import BigQueryRelation
@@ -27,7 +24,6 @@
 from dbt.logger import GLOBAL_LOGGER as logger  # noqa
 from dbt.context.providers import RuntimeConfigObject
 
-import google.cloud.bigquery
 from google.cloud.bigquery import AccessEntry
 
 from .utils import config_from_parts_or_dicts, inject_adapter, TestAdapterConversions
@@ -121,7 +117,7 @@ def setUp(self):
                             'execution_config' : {
                                 'service_account': 'dbt@dummy-project.iam.gserviceaccount.com',
                                 'subnetwork_uri': 'dataproc',
-                                'network_tags': [ "foo", "bar" ]
+                                'network_tags': ["foo", "bar"]
                             }
                         },
                         'labels': {
@@ -197,7 +193,7 @@ def test_acquire_connection_oauth_no_project_validations(self, mock_open_connect
         except dbt.exceptions.DbtValidationError as e:
             self.fail('got DbtValidationError: {}'.format(str(e)))
 
-        except BaseException as e:
+        except BaseException:
             raise
 
         mock_open_connection.assert_not_called()
@@ -214,7 +210,7 @@ def test_acquire_connection_oauth_validations(self, mock_open_connection):
         except dbt.exceptions.DbtValidationError as e:
             self.fail('got DbtValidationError: {}'.format(str(e)))
 
-        except BaseException as e:
+        except BaseException:
             raise
 
         mock_open_connection.assert_not_called()
@@ -233,7 +229,7 @@ def test_acquire_connection_dataproc_serverless(self, mock_open_connection, mock
         except dbt.exceptions.ValidationException as e:
             self.fail('got ValidationException: {}'.format(str(e)))
 
-        except BaseException as e:
+        except BaseException:
             raise
 
         mock_open_connection.assert_not_called()
@@ -250,7 +246,7 @@ def test_acquire_connection_service_account_validations(self, mock_open_connecti
         except dbt.exceptions.DbtValidationError as e:
             self.fail('got DbtValidationError: {}'.format(str(e)))
 
-        except BaseException as e:
+        except BaseException:
             raise
 
         mock_open_connection.assert_not_called()
@@ -267,7 +263,7 @@ def test_acquire_connection_oauth_token_validations(self, mock_open_connection):
         except dbt.exceptions.DbtValidationError as e:
             self.fail('got DbtValidationError: {}'.format(str(e)))
 
-        except BaseException as e:
+        except BaseException:
             raise
 
         mock_open_connection.assert_not_called()
@@ -284,7 +280,7 @@ def test_acquire_connection_oauth_credentials_validations(self, mock_open_connec
         except dbt.exceptions.DbtValidationError as e:
             self.fail('got DbtValidationError: {}'.format(str(e)))
 
-        except BaseException as e:
+        except BaseException:
             raise
 
         mock_open_connection.assert_not_called()
@@ -303,7 +299,7 @@ def test_acquire_connection_impersonated_service_account_validations(
         except dbt.exceptions.DbtValidationError as e:
             self.fail('got DbtValidationError: {}'.format(str(e)))
 
-        except BaseException as e:
+        except BaseException:
             raise
 
         mock_open_connection.assert_not_called()
@@ -395,6 +391,7 @@ def setUp(self):
         self.relation_cls = self._relation_patch.start()
 
         self.mock_connection_manager = self.conn_manager_cls.return_value
+        self.mock_connection_manager.get_if_exists().name = "mock_conn_name"
         self.conn_manager_cls.TYPE = "bigquery"
         self.relation_cls.get_default_quote_policy.side_effect = (
             BigQueryRelation.get_default_quote_policy
@@ -1053,8 +1050,10 @@ def test_grant_access_to_calls_update_with_valid_access_entry(self):
         a_different_entity = BigQueryRelation.from_dict(
             {
                 "type": None,
-                "path": {"database": "another-test-project",
-                 "schema": "test_schema_2", "identifier": "my_view"},
+                "path": {
+                    "database": "another-test-project",
+                    "schema": "test_schema_2", "identifier": "my_view"
+                },
                 "quote_policy": {"identifier": True},
             }
         )
@@ -1086,4 +1085,4 @@ def test_sanitize_label_length(label_length):
         random.choice(string.ascii_uppercase + string.digits)
         for i in range(label_length)
     )
-    assert len(_sanitize_label(random_string)) <= _VALIDATE_LABEL_LENGTH_LIMIT
\ No newline at end of file
+    assert len(_sanitize_label(random_string)) <= _VALIDATE_LABEL_LENGTH_LIMIT

From f8ace3880e15550587270ed8357d9d0d33823d2c Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 24 Mar 2023 15:24:31 -0700
Subject: [PATCH 594/860] Update types-protobuf requirement from ~=4.0 to
 ~=4.22 (#597)

Updates the requirements on [types-protobuf](https://github.com/python/typeshed) to permit the latest version.
- [Release notes](https://github.com/python/typeshed/releases)
- [Commits](https://github.com/python/typeshed/commits)

---
updated-dependencies:
- dependency-name: types-protobuf
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mila Page <67295367+VersusFacit@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index c8aacf182..4a982392e 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -28,6 +28,6 @@ tox~=3.0;python_version=="3.7"
 tox~=4.4;python_version>="3.8"
 types-pytz~=2022.7
 types-requests~=2.28
-types-protobuf~=4.0
+types-protobuf~=4.22
 twine~=4.0
 wheel~=0.40

From 500867bb7de9c56e330680a9adc1546c27915d2f Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Tue, 28 Mar 2023 11:52:33 -0500
Subject: [PATCH 595/860] add test for enforcing contracts on incremental
 materializations (#608)

* add test for enforcing contracts on incremental materializations

* cleanup formatting

* fix test files

* add incremental model

* reset dev reqs
---
 tests/functional/adapter/test_constraints.py | 85 +++++++++++++++++---
 1 file changed, 76 insertions(+), 9 deletions(-)

diff --git a/tests/functional/adapter/test_constraints.py b/tests/functional/adapter/test_constraints.py
index ae3d874d2..266b97b0d 100644
--- a/tests/functional/adapter/test_constraints.py
+++ b/tests/functional/adapter/test_constraints.py
@@ -3,19 +3,26 @@
 from dbt.tests.adapter.constraints.test_constraints import (
     BaseTableConstraintsColumnsEqual,
     BaseViewConstraintsColumnsEqual,
-    BaseConstraintsRuntimeEnforcement
+    BaseIncrementalConstraintsColumnsEqual,
+    BaseConstraintsRuntimeDdlEnforcement,
+    BaseConstraintsRollback,
+    BaseIncrementalConstraintsRuntimeDdlEnforcement,
+    BaseIncrementalConstraintsRollback,
 )
 from dbt.tests.adapter.constraints.fixtures import (
     my_model_sql,
+    my_incremental_model_sql,
     my_model_wrong_order_sql,
-    my_model_wrong_name_sql,
     my_model_view_wrong_order_sql,
+    my_model_incremental_wrong_order_sql,
+    my_model_wrong_name_sql,
     my_model_view_wrong_name_sql,
+    my_model_incremental_wrong_name_sql,
     model_schema_yml,
 )
 
 _expected_sql_bigquery = """
-create or replace table {0} (
+create or replace table <model_identifier> (
     id integer not null,
     color string,
     date_day string
@@ -65,7 +72,10 @@ def data_types(self, int_type, string_type):
         ]
 
 
-class TestBigQueryTableConstraintsColumnsEqual(BigQueryColumnEqualSetup, BaseTableConstraintsColumnsEqual):
+class TestBigQueryTableConstraintsColumnsEqual(
+    BigQueryColumnEqualSetup,
+    BaseTableConstraintsColumnsEqual
+):
     @pytest.fixture(scope="class")
     def models(self):
         return {
@@ -75,7 +85,10 @@ def models(self):
         }
 
 
-class TestBigQueryViewConstraintsColumnsEqual(BigQueryColumnEqualSetup, BaseViewConstraintsColumnsEqual):
+class TestBigQueryViewConstraintsColumnsEqual(
+    BigQueryColumnEqualSetup,
+    BaseViewConstraintsColumnsEqual
+):
     @pytest.fixture(scope="class")
     def models(self):
         return {
@@ -85,19 +98,73 @@ def models(self):
         }
 
 
-class TestBigQueryConstraintsRuntimeEnforcement(BaseConstraintsRuntimeEnforcement):
+class TestBigQueryIncrementalConstraintsColumnsEqual(
+    BigQueryColumnEqualSetup,
+    BaseIncrementalConstraintsColumnsEqual
+):
     @pytest.fixture(scope="class")
     def models(self):
         return {
-            "my_model.sql": my_model_sql,
+            "my_model_wrong_order.sql": my_model_incremental_wrong_order_sql,
+            "my_model_wrong_name.sql": my_model_incremental_wrong_name_sql,
+            "constraints_schema.yml": constraints_yml,
+        }
+
+
+class TestBigQueryTableConstraintsRuntimeDdlEnforcement(
+    BaseConstraintsRuntimeDdlEnforcement
+):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model.sql": my_model_wrong_order_sql,
             "constraints_schema.yml": constraints_yml,
         }
 
     @pytest.fixture(scope="class")
     def expected_sql(self, project):
-        relation = relation_from_name(project.adapter, "my_model")
-        return _expected_sql_bigquery.format(relation)
+        return _expected_sql_bigquery
+
+
+class TestBigQueryTableConstraintsRollback(
+    BaseConstraintsRollback
+):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model.sql": my_model_sql,
+            "constraints_schema.yml": constraints_yml,
+        }
 
     @pytest.fixture(scope="class")
     def expected_error_messages(self):
         return ["Required field id cannot be null"]
+
+class TestBigQueryIncrementalConstraintsRuntimeDdlEnforcement(
+    BaseIncrementalConstraintsRuntimeDdlEnforcement
+):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model.sql": my_model_incremental_wrong_order_sql,
+            "constraints_schema.yml": constraints_yml,
+        }
+
+    @pytest.fixture(scope="class")
+    def expected_sql(self, project):
+        return _expected_sql_bigquery
+
+
+class TestBigQueryIncrementalConstraintsRollback(
+    BaseIncrementalConstraintsRollback
+):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model.sql": my_incremental_model_sql,
+            "constraints_schema.yml": constraints_yml,
+        }
+
+    @pytest.fixture(scope="class")
+    def expected_error_messages(self):
+        return ["Required field id cannot be null"]
\ No newline at end of file

From 3d2f2b188a8c1dfe90220c38fac82dd0c2ef5445 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Tue, 28 Mar 2023 17:39:46 -0500
Subject: [PATCH 596/860] update to retrieve contract enforced from dict (#634)

* update to retrieve contract enforced from dict

* change ref

* Update Under the Hood-20230223-105149.yaml

* revert branch dependency
---
 .changes/1.5.0/Under the Hood-20230223-105149.yaml      | 6 ------
 .changes/unreleased/Under the Hood-20230223-105149.yaml | 6 ++++++
 dbt/include/bigquery/macros/adapters.sql                | 6 ++++--
 3 files changed, 10 insertions(+), 8 deletions(-)
 delete mode 100644 .changes/1.5.0/Under the Hood-20230223-105149.yaml
 create mode 100644 .changes/unreleased/Under the Hood-20230223-105149.yaml

diff --git a/.changes/1.5.0/Under the Hood-20230223-105149.yaml b/.changes/1.5.0/Under the Hood-20230223-105149.yaml
deleted file mode 100644
index d06d2e60e..000000000
--- a/.changes/1.5.0/Under the Hood-20230223-105149.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Rename constraints_enabled to contract
-time: 2023-02-23T10:51:49.737457-05:00
-custom:
-  Author: gshank
-  Issue: "548"
diff --git a/.changes/unreleased/Under the Hood-20230223-105149.yaml b/.changes/unreleased/Under the Hood-20230223-105149.yaml
new file mode 100644
index 000000000..ca315cc1d
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20230223-105149.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Treat contract config as a python object
+time: 2023-02-23T10:51:49.737457-05:00
+custom:
+  Author: gshank emmyoop
+  Issue: 548 633
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index 4c9f00dbf..dc1335db9 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -52,7 +52,8 @@
     {{ sql_header if sql_header is not none }}
 
     create or replace table {{ relation }}
-      {% if config.get('contract', False) %}
+      {%- set contract_config = config.get('contract') -%}
+      {%- if contract_config.enforced -%}
         {{ get_assert_columns_equivalent(compiled_code) }}
         {{ get_columns_spec_ddl() }}
         {%- set compiled_code = get_select_subquery(compiled_code) %}
@@ -91,7 +92,8 @@
 
   create or replace view {{ relation }}
   {{ bigquery_view_options(config, model) }}
-  {% if config.get('contract', False) -%}
+  {%- set contract_config = config.get('contract') -%}
+  {%- if contract_config.enforced -%}
     {{ get_assert_columns_equivalent(sql) }}
   {%- endif %}
   as {{ sql }};

From 3487ebcbbd8acd90c8a9f3d71c6ec9fe4e474ac9 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 28 Mar 2023 20:36:12 -0700
Subject: [PATCH 597/860] Update pytz requirement from ~=2022.7 to ~=2023.2
 (#632)

Updates the requirements on [pytz](https://github.com/stub42/pytz) to permit the latest version.
- [Release notes](https://github.com/stub42/pytz/releases)
- [Commits](https://github.com/stub42/pytz/compare/release_2022.7...release_2023.2)

---
updated-dependencies:
- dependency-name: pytz
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mila Page <67295367+VersusFacit@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index 4a982392e..048e121b0 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -23,7 +23,7 @@ pytest-csv~=3.0
 pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
 pytest-xdist~=3.2
-pytz~=2022.7
+pytz~=2023.2
 tox~=3.0;python_version=="3.7"
 tox~=4.4;python_version>="3.8"
 types-pytz~=2022.7

From 13167576f3a96065149fca6de3fa1867cfdafe65 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 28 Mar 2023 20:53:12 -0700
Subject: [PATCH 598/860] Update types-pytz requirement from ~=2022.7 to
 ~=2023.2 (#631)

Updates the requirements on [types-pytz](https://github.com/python/typeshed) to permit the latest version.
- [Release notes](https://github.com/python/typeshed/releases)
- [Commits](https://github.com/python/typeshed/commits)

---
updated-dependencies:
- dependency-name: types-pytz
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mila Page <67295367+VersusFacit@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index 048e121b0..d0ee86a79 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -26,7 +26,7 @@ pytest-xdist~=3.2
 pytz~=2023.2
 tox~=3.0;python_version=="3.7"
 tox~=4.4;python_version>="3.8"
-types-pytz~=2022.7
+types-pytz~=2023.2
 types-requests~=2.28
 types-protobuf~=4.22
 twine~=4.0

From b9af046936d508e4f5604936864aa1272b12de42 Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Thu, 30 Mar 2023 20:01:32 +0000
Subject: [PATCH 599/860] Bumping version to 1.5.0b4 and generate changelog

---
 .bumpversion.cfg                              |  2 +-
 .changes/1.5.0-b4.md                          | 18 ++++++++++++++++
 .../Features-20230228-094234.yaml             |  0
 .../Features-20230314-171221.yaml             |  0
 .../Features-20230315-120554.yaml             |  0
 .../Fixes-20230322-162200.yaml                |  0
 .../Under the Hood-20230223-105149.yaml       |  0
 CHANGELOG.md                                  | 21 ++++++++++++++++++-
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 10 files changed, 41 insertions(+), 4 deletions(-)
 create mode 100644 .changes/1.5.0-b4.md
 rename .changes/{unreleased => 1.5.0}/Features-20230228-094234.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Features-20230314-171221.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Features-20230315-120554.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Fixes-20230322-162200.yaml (100%)
 rename .changes/{unreleased => 1.5.0}/Under the Hood-20230223-105149.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index fadf60275..dc5b81465 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.5.0b3
+current_version = 1.5.0b4
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.5.0-b4.md b/.changes/1.5.0-b4.md
new file mode 100644
index 000000000..2262d374d
--- /dev/null
+++ b/.changes/1.5.0-b4.md
@@ -0,0 +1,18 @@
+## dbt-bigquery 1.5.0-b4 - March 30, 2023
+
+### Features
+
+- Adding `bytes_billed` to `BigQueryAdapterResponse`  ([#560](https://github.com/dbt-labs/dbt-bigquery/issues/560))
+- Modify addapter to support unified constraint fields ([#567](https://github.com/dbt-labs/dbt-bigquery/issues/567))
+- Modify create_table_as to use contract column order ([#579](https://github.com/dbt-labs/dbt-bigquery/issues/579))
+
+### Fixes
+
+- Use _make_ref_key_dict instead of _make_ref_key_msg ([#621](https://github.com/dbt-labs/dbt-bigquery/issues/621))
+
+### Under the Hood
+
+- Treat contract config as a python object ([#548](https://github.com/dbt-labs/dbt-bigquery/issues/548), [#633](https://github.com/dbt-labs/dbt-bigquery/issues/633))
+
+### Contributors
+- [@bruno-szdl](https://github.com/bruno-szdl) ([#560](https://github.com/dbt-labs/dbt-bigquery/issues/560))
diff --git a/.changes/unreleased/Features-20230228-094234.yaml b/.changes/1.5.0/Features-20230228-094234.yaml
similarity index 100%
rename from .changes/unreleased/Features-20230228-094234.yaml
rename to .changes/1.5.0/Features-20230228-094234.yaml
diff --git a/.changes/unreleased/Features-20230314-171221.yaml b/.changes/1.5.0/Features-20230314-171221.yaml
similarity index 100%
rename from .changes/unreleased/Features-20230314-171221.yaml
rename to .changes/1.5.0/Features-20230314-171221.yaml
diff --git a/.changes/unreleased/Features-20230315-120554.yaml b/.changes/1.5.0/Features-20230315-120554.yaml
similarity index 100%
rename from .changes/unreleased/Features-20230315-120554.yaml
rename to .changes/1.5.0/Features-20230315-120554.yaml
diff --git a/.changes/unreleased/Fixes-20230322-162200.yaml b/.changes/1.5.0/Fixes-20230322-162200.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230322-162200.yaml
rename to .changes/1.5.0/Fixes-20230322-162200.yaml
diff --git a/.changes/unreleased/Under the Hood-20230223-105149.yaml b/.changes/1.5.0/Under the Hood-20230223-105149.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20230223-105149.yaml
rename to .changes/1.5.0/Under the Hood-20230223-105149.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index a71fcd8bf..a1e6f96bf 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,6 +5,26 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.5.0-b4 - March 30, 2023
+
+### Features
+
+- Adding `bytes_billed` to `BigQueryAdapterResponse`  ([#560](https://github.com/dbt-labs/dbt-bigquery/issues/560))
+- Modify addapter to support unified constraint fields ([#567](https://github.com/dbt-labs/dbt-bigquery/issues/567))
+- Modify create_table_as to use contract column order ([#579](https://github.com/dbt-labs/dbt-bigquery/issues/579))
+
+### Fixes
+
+- Use _make_ref_key_dict instead of _make_ref_key_msg ([#621](https://github.com/dbt-labs/dbt-bigquery/issues/621))
+
+### Under the Hood
+
+- Treat contract config as a python object ([#548](https://github.com/dbt-labs/dbt-bigquery/issues/548), [#633](https://github.com/dbt-labs/dbt-bigquery/issues/633))
+
+### Contributors
+- [@bruno-szdl](https://github.com/bruno-szdl) ([#560](https://github.com/dbt-labs/dbt-bigquery/issues/560))
+
+
 ## dbt-bigquery 1.5.0-b3 - March 16, 2023
 
 ### Features
@@ -26,7 +46,6 @@
 - [@patkearns10](https://github.com/patkearns10) ([#562](https://github.com/dbt-labs/dbt-bigquery/issues/562))
 - [@torkjel](https://github.com/torkjel) ([#530](https://github.com/dbt-labs/dbt-bigquery/issues/530))
 
-
 ## dbt-bigquery 1.5.0-b2 - March 02, 2023
 
 ### Under the Hood
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 649c005ac..4a7a8147e 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.5.0b3"
+version = "1.5.0b4"
diff --git a/setup.py b/setup.py
index 3186aa435..b401f311e 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.5.0b3"
+package_version = "1.5.0b4"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From 43b7f32d1964926260aea7591876f6e9f07afaa1 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen <jeremy@dbtlabs.com>
Date: Tue, 11 Apr 2023 18:46:45 +0200
Subject: [PATCH 600/860] Update `bigquery__format_column` per dbt-core#7319
 (#655)

* Test dbt-core#7319

* Update bigquery__format_column

* Changelog entry

* Reset dev-requirements to main branch
---
 .changes/unreleased/Under the Hood-20230411-143129.yaml    | 6 ++++++
 dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql | 4 +++-
 2 files changed, 9 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Under the Hood-20230411-143129.yaml

diff --git a/.changes/unreleased/Under the Hood-20230411-143129.yaml b/.changes/unreleased/Under the Hood-20230411-143129.yaml
new file mode 100644
index 000000000..2d41e7cff
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20230411-143129.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Update bigquery__format_column macro to support prettier ContractError message"
+time: 2023-04-11T14:31:29.378726+02:00
+custom:
+  Author: jtcohen6
+  Issue: "656"
diff --git a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
index 091209ae3..dcf8af903 100644
--- a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
+++ b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
@@ -27,5 +27,7 @@
 {% endmacro %}
 
 {% macro bigquery__format_column(column) -%}
-  {{ return(column.column.lower() ~ " " ~ column.data_type) }}
+  {% set data_type = column.data_type %}
+  {% set formatted = column.column.lower() ~ " " ~ data_type %}
+  {{ return({'name': column.name, 'data_type': data_type, 'formatted': formatted}) }}
 {%- endmacro -%}

From 4a5d648219d6bce00f7e3530bc4b1ffca2eb862d Mon Sep 17 00:00:00 2001
From: Peter Webb <peter.webb@dbtlabs.com>
Date: Tue, 11 Apr 2023 13:59:24 -0400
Subject: [PATCH 601/860] Model-Level Constraints (#648)

* CT-2221: Temporary change to dev-reqs

* CT-2221: Add BigQuery adapter support for model-level constraints

* CT-2221: Add changelog entry

* CT-2221: Fixed unit tests. Added support for NOT ENFORCED primary key syntax.

* CT-2221: Add support for foreign keys as well.

* CT-2221: Address review issue.

* CT-2221: Fix some integration test failures

* CT-2221: Re-add macro override which should not have been removed.
---
 .../unreleased/Features-20230406-104433.yaml  |  6 +++
 dbt/adapters/bigquery/impl.py                 | 26 +++++++++++
 dbt/include/bigquery/macros/adapters.sql      |  2 +-
 .../macros/utils/get_columns_spec_ddl.sql     | 28 ------------
 dev-requirements.txt                          |  4 +-
 tests/functional/adapter/test_constraints.py  | 44 ++++++++++++++++---
 6 files changed, 73 insertions(+), 37 deletions(-)
 create mode 100644 .changes/unreleased/Features-20230406-104433.yaml

diff --git a/.changes/unreleased/Features-20230406-104433.yaml b/.changes/unreleased/Features-20230406-104433.yaml
new file mode 100644
index 000000000..4ec1d8120
--- /dev/null
+++ b/.changes/unreleased/Features-20230406-104433.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Add support for model-level constraints
+time: 2023-04-06T10:44:33.045896-04:00
+custom:
+  Author: peterallenwebb
+  Issue: "569"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 158cbc151..e7e41a0c3 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -1,6 +1,8 @@
 from dataclasses import dataclass
 import threading
 from typing import Dict, List, Optional, Any, Set, Union, Type
+
+from dbt.contracts.graph.nodes import ColumnLevelConstraint, ModelLevelConstraint, ConstraintType
 from dbt.dataclass_schema import dbtClassMixin, ValidationError
 
 import dbt.deprecations
@@ -907,3 +909,27 @@ def python_submission_helpers(self) -> Dict[str, Type[PythonJobHelper]]:
             "cluster": ClusterDataprocHelper,
             "serverless": ServerlessDataProcHelper,
         }
+
+    @classmethod
+    def render_column_constraint(cls, constraint: ColumnLevelConstraint) -> str:
+        if constraint.type == ConstraintType.not_null:
+            return super().render_column_constraint(constraint)
+        elif (
+            constraint.type == ConstraintType.primary_key
+            or constraint.type == ConstraintType.foreign_key
+        ):
+            c = super().render_column_constraint(constraint)
+            return f"{c} not enforced"
+        else:
+            return ""
+
+    @classmethod
+    def render_model_constraint(cls, constraint: ModelLevelConstraint) -> Optional[str]:
+        if (
+            constraint.type == ConstraintType.primary_key
+            or constraint.type == ConstraintType.foreign_key
+        ):
+            c = super().render_model_constraint(constraint)
+            return f"{c} not enforced" if c else None
+        else:
+            return None
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index dc1335db9..bb7fb8b1a 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -55,7 +55,7 @@
       {%- set contract_config = config.get('contract') -%}
       {%- if contract_config.enforced -%}
         {{ get_assert_columns_equivalent(compiled_code) }}
-        {{ get_columns_spec_ddl() }}
+        {{ get_table_columns_and_constraints() }}
         {%- set compiled_code = get_select_subquery(compiled_code) %}
       {% endif %}
     {{ partition_by(partition_config) }}
diff --git a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
index dcf8af903..d078e8ad9 100644
--- a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
+++ b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
@@ -1,31 +1,3 @@
-{% macro bigquery__get_columns_spec_ddl() %}
-  {# loop through user_provided_columns to create DDL with data types and constraints #}
-    {%- set ns = namespace(at_least_one_check=False, at_least_one_pk=False) -%}
-    {%- set user_provided_columns = model['columns'] -%}
-    (
-    {% for i in user_provided_columns %}
-      {%- set col = user_provided_columns[i] -%}
-      {%- set constraints = col['constraints'] -%}
-      {{ col['name'] }} {{ col['data_type'] }}
-      {%- for c in constraints -%}
-        {%- if c.type == "check" -%}
-          {%- set ns.at_least_one_check = True -%}
-        {%- elif c.type == "primary_key" -%}
-          {%- set ns.at_least_one_pk = True -%}
-        {%- else %} {{ adapter.render_raw_column_constraint(c) }}
-        {%- endif -%}
-      {%- endfor -%}
-      {{ "," if not loop.last }}
-    {% endfor -%}
-    )
-  {%- if ns.at_least_one_check -%}
-      {{exceptions.warn("We noticed you have check constraints in your configs. These are not compatible with BigQuery and will be ignored.")}}
-  {%- endif -%}
-  {%- if ns.at_least_one_pk -%}
-    {{exceptions.warn("We noticed you have primary key constraints in your configs. These are not compatible with BigQuery and will be ignored.")}}
-  {%- endif -%}
-{% endmacro %}
-
 {% macro bigquery__format_column(column) -%}
   {% set data_type = column.data_type %}
   {% set formatted = column.column.lower() ~ " " ~ data_type %}
diff --git a/dev-requirements.txt b/dev-requirements.txt
index d0ee86a79..fc12cd0fd 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -1,7 +1,7 @@
 # install latest changes in dbt-core
 # TODO: how to automate switching from develop to version branches?
-git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
-git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
+git+https://github.com/dbt-labs/dbt-core.git@paw/ct-1922-model-level-constraints#egg=dbt-core&subdirectory=core
+git+https://github.com/dbt-labs/dbt-core.git@paw/ct-1922-model-level-constraints#egg=dbt-tests-adapter&subdirectory=tests/adapter
 
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor
diff --git a/tests/functional/adapter/test_constraints.py b/tests/functional/adapter/test_constraints.py
index 266b97b0d..27666f064 100644
--- a/tests/functional/adapter/test_constraints.py
+++ b/tests/functional/adapter/test_constraints.py
@@ -7,7 +7,7 @@
     BaseConstraintsRuntimeDdlEnforcement,
     BaseConstraintsRollback,
     BaseIncrementalConstraintsRuntimeDdlEnforcement,
-    BaseIncrementalConstraintsRollback,
+    BaseIncrementalConstraintsRollback, BaseModelConstraintsRuntimeEnforcement,
 )
 from dbt.tests.adapter.constraints.fixtures import (
     my_model_sql,
@@ -18,12 +18,12 @@
     my_model_wrong_name_sql,
     my_model_view_wrong_name_sql,
     my_model_incremental_wrong_name_sql,
-    model_schema_yml,
+    model_schema_yml, constrained_model_schema_yml,
 )
 
 _expected_sql_bigquery = """
 create or replace table <model_identifier> (
-    id integer not null,
+    id integer not null primary key not enforced,
     color string,
     date_day string
 )
@@ -43,8 +43,8 @@
 # Different on BigQuery:
 # - does not support a data type named 'text' (TODO handle this via type translation/aliasing!)
 # - raises an explicit error, if you try to set a primary key constraint, because it's not enforced
-constraints_yml = model_schema_yml.replace("text", "string").replace("primary key", "")
-
+constraints_yml = model_schema_yml.replace("text", "string")
+model_constraints_yml = constrained_model_schema_yml.replace("text", "string")
 
 class BigQueryColumnEqualSetup:
     @pytest.fixture
@@ -167,4 +167,36 @@ def models(self):
 
     @pytest.fixture(scope="class")
     def expected_error_messages(self):
-        return ["Required field id cannot be null"]
\ No newline at end of file
+        return ["Required field id cannot be null"]
+
+
+class TestBigQueryModelConstraintsRuntimeEnforcement(BaseModelConstraintsRuntimeEnforcement):
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model.sql": my_incremental_model_sql,
+            "constraints_schema.yml": model_constraints_yml,
+        }
+
+    @pytest.fixture(scope="class")
+    def expected_sql(self):
+        return """
+create or replace table <model_identifier> (
+    id integer not null,
+    color string,
+    date_day string,
+    primary key (id) not enforced
+)
+OPTIONS()
+as (
+    select id,
+    color, 
+    date_day from 
+  ( 
+    select 1 as id, 
+    'blue' as color, 
+    '2019-01-01' as date_day
+  ) as model_subq
+);
+"""
\ No newline at end of file

From d7ab34734c5285c52962611043146ac2126f9a7c Mon Sep 17 00:00:00 2001
From: Peter Webb <peter.webb@dbtlabs.com>
Date: Tue, 11 Apr 2023 14:59:07 -0400
Subject: [PATCH 602/860] Repair accidental change to dev-requirements (#658)

---
 .changes/unreleased/Fixes-20230411-143515.yaml | 6 ++++++
 dev-requirements.txt                           | 4 ++--
 2 files changed, 8 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230411-143515.yaml

diff --git a/.changes/unreleased/Fixes-20230411-143515.yaml b/.changes/unreleased/Fixes-20230411-143515.yaml
new file mode 100644
index 000000000..c88b53094
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230411-143515.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Repair accidental change to dev-requirements
+time: 2023-04-11T14:35:15.369296-04:00
+custom:
+  Author: peterallenwebb
+  Issue: "657"
diff --git a/dev-requirements.txt b/dev-requirements.txt
index fc12cd0fd..d0ee86a79 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -1,7 +1,7 @@
 # install latest changes in dbt-core
 # TODO: how to automate switching from develop to version branches?
-git+https://github.com/dbt-labs/dbt-core.git@paw/ct-1922-model-level-constraints#egg=dbt-core&subdirectory=core
-git+https://github.com/dbt-labs/dbt-core.git@paw/ct-1922-model-level-constraints#egg=dbt-tests-adapter&subdirectory=tests/adapter
+git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
+git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
 
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor

From 55cb5ca800faa3ff4e9e1ddfa5e087ad36f97804 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Tue, 11 Apr 2023 14:46:53 -0500
Subject: [PATCH 603/860] update for generalized constraint warnings (#646)

* generalize constrint warnings

* fix tests

* put back text override

* add extra spaces

* consolidate logic

* fixed method signature

* update dev requirements
---
 .changes/1.5.0/Features-20221220-193731.yaml |  7 ++---
 dbt/adapters/bigquery/impl.py                | 32 ++++++++++++--------
 tests/functional/adapter/test_constraints.py |  3 +-
 3 files changed, 24 insertions(+), 18 deletions(-)

diff --git a/.changes/1.5.0/Features-20221220-193731.yaml b/.changes/1.5.0/Features-20221220-193731.yaml
index dbd16ae0f..c9aa5c03d 100644
--- a/.changes/1.5.0/Features-20221220-193731.yaml
+++ b/.changes/1.5.0/Features-20221220-193731.yaml
@@ -1,7 +1,6 @@
 kind: Features
-body: 'dbt-constraints support for BigQuery as per dbt-core issue #1358'
+body: 'Support for data type constraints in BigQuery'
 time: 2022-12-20T19:37:31.982821+01:00
 custom:
-  Author: victoriapm
-  Issue: "444"
-  PR: "426"
+  Author: victoriapm, emmyoop
+  Issue: 444 568
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index e7e41a0c3..9016dfb83 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -12,6 +12,7 @@
 from dbt import ui  # type: ignore
 from dbt.adapters.base import (
     BaseAdapter,
+    ConstraintSupport,
     available,
     RelationType,
     BaseRelation,
@@ -32,7 +33,9 @@
 )
 from dbt.adapters.bigquery.connections import BigQueryAdapterResponse
 from dbt.contracts.graph.manifest import Manifest
-from dbt.events import AdapterLogger
+from dbt.events import (
+    AdapterLogger,
+)
 from dbt.events.functions import fire_event
 from dbt.events.types import SchemaCreation, SchemaDrop
 from dbt.utils import filter_null_values
@@ -168,6 +171,14 @@ class BigQueryAdapter(BaseAdapter):
 
     AdapterSpecificConfigs = BigqueryConfig
 
+    CONSTRAINT_SUPPORT = {
+        ConstraintType.check: ConstraintSupport.NOT_SUPPORTED,
+        ConstraintType.not_null: ConstraintSupport.ENFORCED,
+        ConstraintType.unique: ConstraintSupport.NOT_SUPPORTED,
+        ConstraintType.primary_key: ConstraintSupport.ENFORCED,
+        ConstraintType.foreign_key: ConstraintSupport.ENFORCED,
+    }
+
     ###
     # Implementations of abstract methods
     ###
@@ -911,25 +922,22 @@ def python_submission_helpers(self) -> Dict[str, Type[PythonJobHelper]]:
         }
 
     @classmethod
-    def render_column_constraint(cls, constraint: ColumnLevelConstraint) -> str:
-        if constraint.type == ConstraintType.not_null:
-            return super().render_column_constraint(constraint)
-        elif (
+    def render_column_constraint(cls, constraint: ColumnLevelConstraint) -> Optional[str]:
+        c = super().render_column_constraint(constraint)
+        if (
             constraint.type == ConstraintType.primary_key
             or constraint.type == ConstraintType.foreign_key
         ):
-            c = super().render_column_constraint(constraint)
-            return f"{c} not enforced"
-        else:
-            return ""
+            return f"{c} not enforced" if c else None
+        return c
 
     @classmethod
     def render_model_constraint(cls, constraint: ModelLevelConstraint) -> Optional[str]:
+        c = super().render_model_constraint(constraint)
         if (
             constraint.type == ConstraintType.primary_key
             or constraint.type == ConstraintType.foreign_key
         ):
-            c = super().render_model_constraint(constraint)
             return f"{c} not enforced" if c else None
-        else:
-            return None
+
+        return c
diff --git a/tests/functional/adapter/test_constraints.py b/tests/functional/adapter/test_constraints.py
index 27666f064..e0d30cf6f 100644
--- a/tests/functional/adapter/test_constraints.py
+++ b/tests/functional/adapter/test_constraints.py
@@ -42,7 +42,6 @@
 
 # Different on BigQuery:
 # - does not support a data type named 'text' (TODO handle this via type translation/aliasing!)
-# - raises an explicit error, if you try to set a primary key constraint, because it's not enforced
 constraints_yml = model_schema_yml.replace("text", "string")
 model_constraints_yml = constrained_model_schema_yml.replace("text", "string")
 
@@ -199,4 +198,4 @@ def expected_sql(self):
     '2019-01-01' as date_day
   ) as model_subq
 );
-"""
\ No newline at end of file
+"""

From a10e2b0dbe6e6d8521a2bb74aa59d550323b95dc Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Tue, 11 Apr 2023 13:24:00 -0700
Subject: [PATCH 604/860] add steps to drop existing tables on --full-refresh
 (#653)

* add steps to drop existing tables on --full-refresh

* cleanup pr and add changelog entry

---------

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20230408-035117.yaml | 7 +++++++
 dbt/include/bigquery/macros/adapters.sql       | 6 ++++++
 2 files changed, 13 insertions(+)
 create mode 100644 .changes/unreleased/Fixes-20230408-035117.yaml

diff --git a/.changes/unreleased/Fixes-20230408-035117.yaml b/.changes/unreleased/Fixes-20230408-035117.yaml
new file mode 100644
index 000000000..56799823b
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230408-035117.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: ' add full refresh capabilities to tabular bigquery python models to accommodate
+  schema changes'
+time: 2023-04-08T03:51:17.167349-07:00
+custom:
+  Author: versusfacit
+  Issue: "653"
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index bb7fb8b1a..ed9359bee 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -73,6 +73,12 @@
     TODO: Deep dive into spark sessions to see if we can reuse a single session for an entire
     dbt invocation.
      --#}
+
+    {#-- when a user wants to change the schema of an existing relation, they must intentionally drop the table in the dataset --#}
+    {%- set old_relation = adapter.get_relation(database=relation.database, schema=relation.schema, identifier=relation.identifier) -%}
+    {%- if (old_relation.is_table and (should_full_refresh())) -%}
+      {% do adapter.drop_relation(relation) %}
+    {%- endif -%}
     {{ py_write_table(compiled_code=compiled_code, target_relation=relation.quote(database=False, schema=False, identifier=False)) }}
   {%- else -%}
     {% do exceptions.raise_compiler_error("bigquery__create_table_as macro didn't get supported language, it got %s" % language) %}

From 3cb64f2f2d99d9f1a049e41851d3f610715abe41 Mon Sep 17 00:00:00 2001
From: Florian Eiden <florian.eiden@dbtlabs.com>
Date: Tue, 11 Apr 2023 22:53:28 +0200
Subject: [PATCH 605/860] Ignore nulls in partitions to be processed (#552)

Co-authored-by: Mila Page <67295367+VersusFacit@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20230222-133301.yaml              | 6 ++++++
 .../incremental_strategy/insert_overwrite.sql               | 3 ++-
 2 files changed, 8 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20230222-133301.yaml

diff --git a/.changes/unreleased/Fixes-20230222-133301.yaml b/.changes/unreleased/Fixes-20230222-133301.yaml
new file mode 100644
index 000000000..7c613a109
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230222-133301.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Align partitions to be updated in incremental insert_overwrite to _dbt_max_partition wrt to nulls
+time: 2023-02-22T13:33:01.607225-08:00
+custom:
+  Author: patkearns10
+  Issue: "544"
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
index 63df65e2a..3153f49d0 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
@@ -134,7 +134,8 @@
       -- 2. define partitions to update
       set (dbt_partitions_for_replacement) = (
           select as struct
-              array_agg(distinct {{ partition_by.render_wrapped() }})
+              -- IGNORE NULLS: this needs to be aligned to _dbt_max_partition, which ignores null
+              array_agg(distinct {{ partition_by.render_wrapped() }} IGNORE NULLS)
           from {{ tmp_relation }}
       );
 

From 4673e40f36079fb547ae2a75fb352e2d0eb209ff Mon Sep 17 00:00:00 2001
From: Neelesh Salian <nssalian@users.noreply.github.com>
Date: Wed, 12 Apr 2023 07:58:29 -0700
Subject: [PATCH 606/860] [ADAP-17]: Test conversion for simple snapshot into a
 functional test (#651)

* Testing with the changes for BQ.WIP

* Fix for tests

* Nit and remove old file

* Fix nits and use string

* remove branch dependency

* Delete the integration dir after porting over the last test

* Restore to varchar

* Naming for update records expression

* Clean up
---
 .../functional/adapter/test_simple_snaphot.py | 105 ++
 tests/integration/__init__.py                 |   0
 tests/integration/base.py                     | 929 ------------------
 .../add_column_to_source_bq.sql               |  56 --
 .../check_snapshots_test_current.sql          |  51 -
 .../check-snapshots/check_cols_cycle.sql      |  33 -
 .../custom-snapshot-macros/custom.sql         |  18 -
 .../invalidate_bigquery.sql                   |  12 -
 .../macros/test_no_overlaps.sql               |  85 --
 .../simple_snapshot_test/models/.gitkeep      |   0
 .../models/ref_snapshot.sql                   |   1 -
 .../simple_snapshot_test/models/schema.yml    |   5 -
 .../integration/simple_snapshot_test/seed.sql | 220 -----
 .../simple_snapshot_test/seed_bq.sql          |  81 --
 .../simple_snapshot_test/seeds/seed.csv       |   4 -
 .../seeds/seed_newcol.csv                     |   4 -
 .../test-check-col-snapshots-bq/snapshot.sql  |  29 -
 .../snapshot.sql                              |   9 -
 .../test-check-col-snapshots/snapshot.sql     |  28 -
 .../test-snapshots-bq/snapshot.sql            |  19 -
 .../test-snapshots-checkall/snapshot.sql      |   4 -
 .../test-snapshots-pg/snapshot.sql            |  19 -
 .../snapshot.sql                              |  41 -
 .../test-snapshots-select/snapshot.sql        |  44 -
 .../test_simple_snapshot.py                   | 564 -----------
 .../test_snapshot_check_cols.py               |  40 -
 .../simple_snapshot_test/update_bq.sql        |  78 --
 27 files changed, 105 insertions(+), 2374 deletions(-)
 create mode 100644 tests/functional/adapter/test_simple_snaphot.py
 delete mode 100644 tests/integration/__init__.py
 delete mode 100644 tests/integration/base.py
 delete mode 100644 tests/integration/simple_snapshot_test/add_column_to_source_bq.sql
 delete mode 100644 tests/integration/simple_snapshot_test/check-snapshots-expected/check_snapshots_test_current.sql
 delete mode 100644 tests/integration/simple_snapshot_test/check-snapshots/check_cols_cycle.sql
 delete mode 100644 tests/integration/simple_snapshot_test/custom-snapshot-macros/custom.sql
 delete mode 100644 tests/integration/simple_snapshot_test/invalidate_bigquery.sql
 delete mode 100644 tests/integration/simple_snapshot_test/macros/test_no_overlaps.sql
 delete mode 100644 tests/integration/simple_snapshot_test/models/.gitkeep
 delete mode 100644 tests/integration/simple_snapshot_test/models/ref_snapshot.sql
 delete mode 100644 tests/integration/simple_snapshot_test/models/schema.yml
 delete mode 100644 tests/integration/simple_snapshot_test/seed.sql
 delete mode 100644 tests/integration/simple_snapshot_test/seed_bq.sql
 delete mode 100644 tests/integration/simple_snapshot_test/seeds/seed.csv
 delete mode 100644 tests/integration/simple_snapshot_test/seeds/seed_newcol.csv
 delete mode 100644 tests/integration/simple_snapshot_test/test-check-col-snapshots-bq/snapshot.sql
 delete mode 100644 tests/integration/simple_snapshot_test/test-check-col-snapshots-noconfig/snapshot.sql
 delete mode 100644 tests/integration/simple_snapshot_test/test-check-col-snapshots/snapshot.sql
 delete mode 100644 tests/integration/simple_snapshot_test/test-snapshots-bq/snapshot.sql
 delete mode 100644 tests/integration/simple_snapshot_test/test-snapshots-checkall/snapshot.sql
 delete mode 100644 tests/integration/simple_snapshot_test/test-snapshots-pg/snapshot.sql
 delete mode 100644 tests/integration/simple_snapshot_test/test-snapshots-select-noconfig/snapshot.sql
 delete mode 100644 tests/integration/simple_snapshot_test/test-snapshots-select/snapshot.sql
 delete mode 100644 tests/integration/simple_snapshot_test/test_simple_snapshot.py
 delete mode 100644 tests/integration/simple_snapshot_test/test_snapshot_check_cols.py
 delete mode 100644 tests/integration/simple_snapshot_test/update_bq.sql

diff --git a/tests/functional/adapter/test_simple_snaphot.py b/tests/functional/adapter/test_simple_snaphot.py
new file mode 100644
index 000000000..045da5d44
--- /dev/null
+++ b/tests/functional/adapter/test_simple_snaphot.py
@@ -0,0 +1,105 @@
+import pytest
+
+from dbt.tests.util import run_dbt
+
+from dbt.tests.adapter.simple_snapshot.test_snapshot import BaseSimpleSnapshotBase, BaseSnapshotCheck
+
+SNAPSHOT_TIMESTAMP_SQL = """
+{% snapshot snapshot %}
+    {{ config(
+        target_database=database,
+        target_schema=schema,
+        unique_key='id',
+        strategy='timestamp',
+        updated_at='updated_at_ts',
+        invalidate_hard_deletes=True,
+    ) }}
+    select *, timestamp(updated_at) as updated_at_ts from {{ ref('fact') }}
+{% endsnapshot %}
+"""
+
+
+class TestSnapshot(BaseSimpleSnapshotBase):
+    # Not importing the base case because the test_updates* tests need modification for updating intervals
+    @pytest.fixture(scope="class")
+    def snapshots(self):
+        # Using the snapshot defined in the class itself rather than the base case
+        # Reason: dbt-bigquery:#3710: UNION ALL issue when running snapshots with invalidate_hard_deletes=True
+        return {"snapshot.sql": SNAPSHOT_TIMESTAMP_SQL}
+
+    def test_updates_are_captured_by_snapshot(self, project):
+        """
+        Update the last 5 records. Show that all ids are current, but the last 5 reflect updates.
+        """
+        date_add_expression = "date_add(updated_at, interval 1 day)"
+        self.update_fact_records(
+            {
+                "updated_at": date_add_expression
+            },
+            "id between 16 and 20"
+        )
+        run_dbt(["snapshot"])
+        self._assert_results(
+            ids_with_current_snapshot_records=range(1, 21),
+            ids_with_closed_out_snapshot_records=range(16, 21),
+        )
+
+    def test_inserts_are_captured_by_snapshot(self, project):
+        """
+        Insert 10 records. Show that there are 30 records in `snapshot`, all of which are current.
+        """
+        self.insert_fact_records("id between 21 and 30")
+        run_dbt(["snapshot"])
+        self._assert_results(
+            ids_with_current_snapshot_records=range(1, 31), ids_with_closed_out_snapshot_records=[]
+        )
+
+    def test_deletes_are_captured_by_snapshot(self, project):
+        """
+        Hard delete the last five records. Show that there are now only 15 current records and 5 expired records.
+        """
+        self.delete_fact_records("id between 16 and 20")
+        run_dbt(["snapshot"])
+        self._assert_results(
+            ids_with_current_snapshot_records=range(1, 16),
+            ids_with_closed_out_snapshot_records=range(16, 21),
+        )
+
+    def test_revives_are_captured_by_snapshot(self, project):
+        """
+        Delete the last five records and run snapshot to collect that information, then revive 3 of those records.
+        Show that there are now 18 current records and 5 expired records.
+        """
+        self.delete_fact_records("id between 16 and 20")
+        run_dbt(["snapshot"])
+        self.insert_fact_records("id between 16 and 18")
+        run_dbt(["snapshot"])
+        self._assert_results(
+            ids_with_current_snapshot_records=range(1, 19),
+            ids_with_closed_out_snapshot_records=range(16, 21),
+        )
+
+    def test_new_column_captured_by_snapshot(self, project):
+        """
+        Add a column to `fact` and populate the last 10 records with a non-null value.
+        Show that all ids are current, but the last 10 reflect updates and the first 10 don't
+        i.e. if the column is added, but not updated, the record doesn't reflect that it's updated
+        """
+        self.add_fact_column("full_name", "varchar(200) default null")
+        date_add_expression = "date_add(date(updated_at), interval 1 day)"
+        self.update_fact_records(
+            {
+                "full_name": "first_name || ' ' || last_name",
+                "updated_at": date_add_expression,
+            },
+            "id between 11 and 20",
+        )
+        run_dbt(["snapshot"])
+        self._assert_results(
+            ids_with_current_snapshot_records=range(1, 21),
+            ids_with_closed_out_snapshot_records=range(11, 21),
+        )
+
+
+class TestSnapshotCheck(BaseSnapshotCheck):
+    pass
diff --git a/tests/integration/__init__.py b/tests/integration/__init__.py
deleted file mode 100644
index e69de29bb..000000000
diff --git a/tests/integration/base.py b/tests/integration/base.py
deleted file mode 100644
index f8062f802..000000000
--- a/tests/integration/base.py
+++ /dev/null
@@ -1,929 +0,0 @@
-import json
-import os
-import random
-import shutil
-import sys
-import tempfile
-import traceback
-import unittest
-import warnings
-from io import StringIO
-from contextlib import contextmanager
-from datetime import datetime
-from functools import wraps
-
-import pytest
-import yaml
-from unittest.mock import patch
-
-import dbt.main as dbt
-from dbt import flags
-from dbt.deprecations import reset_deprecations
-from dbt.adapters.factory import get_adapter, reset_adapters, register_adapter
-from dbt.clients.jinja import template_cache
-from dbt.config import RuntimeConfig
-from dbt.context import providers
-from dbt.logger import log_manager
-from dbt.events.functions import (
-    capture_stdout_logs, fire_event, setup_event_logger, stop_capture_stdout_logs
-)
-from dbt.events import AdapterLogger
-from dbt.contracts.graph.manifest import Manifest
-
-
-logger = AdapterLogger("Bigquery")
-INITIAL_ROOT = os.getcwd()
-
-
-def normalize(path):
-    """On windows, neither is enough on its own:
-
-    >>> normcase('C:\\documents/ALL CAPS/subdir\\..')
-    'c:\\documents\\all caps\\subdir\\..'
-    >>> normpath('C:\\documents/ALL CAPS/subdir\\..')
-    'C:\\documents\\ALL CAPS'
-    >>> normpath(normcase('C:\\documents/ALL CAPS/subdir\\..'))
-    'c:\\documents\\all caps'
-    """
-    return os.path.normcase(os.path.normpath(path))
-
-
-class Normalized:
-    def __init__(self, value):
-        self.value = value
-
-    def __repr__(self):
-        return f'Normalized({self.value!r})'
-
-    def __str__(self):
-        return f'Normalized({self.value!s})'
-
-    def __eq__(self, other):
-        return normalize(self.value) == normalize(other)
-
-
-class FakeArgs:
-    def __init__(self):
-        self.threads = 1
-        self.defer = False
-        self.full_refresh = False
-        self.models = None
-        self.select = None
-        self.exclude = None
-        self.single_threaded = False
-        self.selector_name = None
-        self.state = None
-        self.defer = None
-
-
-class TestArgs:
-    def __init__(self, kwargs):
-        self.which = 'run'
-        self.single_threaded = False
-        self.profiles_dir = None
-        self.project_dir = None
-        self.__dict__.update(kwargs)
-
-
-def _profile_from_test_name(test_name):
-    adapter_names = ('bigquery',)
-    adapters_in_name = sum(x in test_name for x in adapter_names)
-    if adapters_in_name != 1:
-        raise ValueError(
-            'test names must have exactly 1 profile choice embedded, {} has {}'
-            .format(test_name, adapters_in_name)
-        )
-
-    for adapter_name in adapter_names:
-        if adapter_name in test_name:
-            return adapter_name
-
-    raise ValueError(
-        'could not find adapter name in test name {}'.format(test_name)
-    )
-
-
-def _pytest_test_name():
-    return os.environ['PYTEST_CURRENT_TEST'].split()[0]
-
-
-def _pytest_get_test_root():
-    test_path = _pytest_test_name().split('::')[0]
-    relative_to = INITIAL_ROOT
-    head = os.path.relpath(test_path, relative_to)
-
-    path_parts = []
-    while head:
-        head, tail = os.path.split(head)
-        path_parts.append(tail)
-    path_parts.reverse()
-    # dbt tests are all of the form 'tests/integration/suite_name'
-    target = os.path.join(*path_parts[:3])  # TODO: try to not hard code this
-    return os.path.join(relative_to, target)
-
-
-def _really_makedirs(path):
-    while not os.path.exists(path):
-        try:
-            os.makedirs(path)
-        except EnvironmentError:
-            raise
-
-
-class DBTIntegrationTest(unittest.TestCase):
-    CREATE_SCHEMA_STATEMENT = 'CREATE SCHEMA {}'
-    DROP_SCHEMA_STATEMENT = 'DROP SCHEMA IF EXISTS {} CASCADE'
-
-    _randint = random.randint(0, 9999)
-    _runtime_timedelta = (datetime.utcnow() - datetime(1970, 1, 1, 0, 0, 0))
-    _runtime = (
-        (int(_runtime_timedelta.total_seconds() * 1e6)) +
-        _runtime_timedelta.microseconds
-    )
-
-    prefix = f'test{_runtime}{_randint:04}'
-    setup_alternate_db = False
-
-    def bigquery_profile(self):
-        credentials_json_str = os.getenv('BIGQUERY_TEST_SERVICE_ACCOUNT_JSON').replace("'", '')
-        credentials = json.loads(credentials_json_str)
-        project_id = credentials.get('project_id')
-
-        return {
-            'config': {
-                'send_anonymous_usage_stats': False
-            },
-            'test': {
-                'outputs': {
-                    'default2': {
-                        'type': 'bigquery',
-                        'method': 'service-account-json',
-                        'threads': 1,
-                        'project': project_id,
-                        'keyfile_json': credentials,
-                        'schema': self.unique_schema(),
-                    },
-                    'alternate': {
-                        'type': 'bigquery',
-                        'method': 'service-account-json',
-                        'threads': 1,
-                        'project': project_id,
-                        'keyfile_json': credentials,
-                        'schema': self.unique_schema(),
-                        'execution_project': self.alternative_database,
-                    },
-                },
-                'target': 'default2'
-            }
-        }
-
-    @property
-    def packages_config(self):
-        return None
-
-    @property
-    def selectors_config(self):
-        return None
-
-    def unique_schema(self):
-        schema = self.schema
-
-        to_return = "{}_{}".format(self.prefix, schema)
-
-        return to_return.lower()
-
-    @property
-    def default_database(self):
-        database = self.config.credentials.database
-        return database
-
-    @property
-    def alternative_database(self):
-        return os.environ['BIGQUERY_TEST_ALT_DATABASE']
-
-    def get_profile(self, adapter_type):
-        if adapter_type == 'bigquery':
-            return self.bigquery_profile()
-        else:
-            raise ValueError('invalid adapter type {}'.format(adapter_type))
-
-    def _pick_profile(self):
-        test_name = self.id().split('.')[-1]
-        return _profile_from_test_name(test_name)
-
-    def _symlink_test_folders(self):
-        for entry in os.listdir(self.test_original_source_path):
-            src = os.path.join(self.test_original_source_path, entry)
-            tst = os.path.join(self.test_root_dir, entry)
-            if os.path.isdir(src) or src.endswith('.sql'):
-                # symlink all sql files and all directories.
-                os.symlink(src, tst)
-        os.symlink(self._logs_dir, os.path.join(self.test_root_dir, 'logs'))
-
-    @property
-    def test_root_realpath(self):
-        if sys.platform == 'darwin':
-            return os.path.realpath(self.test_root_dir)
-        else:
-            return self.test_root_dir
-
-    def _generate_test_root_dir(self):
-        return normalize(tempfile.mkdtemp(prefix='dbt-int-test-'))
-
-    def setUp(self):
-        # Logbook warnings are ignored so we don't have to fork logbook to support python 3.10.
-        # This _only_ works for tests in `test/integration`.
-        warnings.filterwarnings(
-            "ignore",
-            category=DeprecationWarning,
-            module="logbook"
-        )
-        self.dbt_core_install_root = os.path.dirname(dbt.__file__)
-        log_manager.reset_handlers()
-        self.initial_dir = INITIAL_ROOT
-        os.chdir(self.initial_dir)
-        # before we go anywhere, collect the initial path info
-        self._logs_dir = os.path.join(self.initial_dir, 'logs', self.prefix)
-        setup_event_logger(self._logs_dir)
-        _really_makedirs(self._logs_dir)
-        self.test_original_source_path = _pytest_get_test_root()
-        self.test_root_dir = self._generate_test_root_dir()
-
-        os.chdir(self.test_root_dir)
-        try:
-            self._symlink_test_folders()
-        except Exception as exc:
-            msg = '\n\t'.join((
-                'Failed to symlink test folders!',
-                'initial_dir={0.initial_dir}',
-                'test_original_source_path={0.test_original_source_path}',
-                'test_root_dir={0.test_root_dir}'
-            )).format(self)
-            logger.exception(msg)
-
-            # if logging isn't set up, I still really want this message.
-            print(msg)
-            traceback.print_exc()
-
-            raise
-
-        self._created_schemas = set()
-        reset_deprecations()
-        template_cache.clear()
-
-        self.use_profile(self._pick_profile())
-        self.use_default_project()
-        self.set_packages()
-        self.set_selectors()
-        self.load_config()
-
-    def use_default_project(self, overrides=None):
-        # create a dbt_project.yml
-        base_project_config = {
-            'name': 'test',
-            'version': '1.0',
-            'config-version': 2,
-            'test-paths': [],
-            'model-paths': [self.models],
-            'profile': 'test',
-        }
-
-        project_config = {}
-        project_config.update(base_project_config)
-        project_config.update(self.project_config)
-        project_config.update(overrides or {})
-
-        with open("dbt_project.yml", 'w') as f:
-            yaml.safe_dump(project_config, f, default_flow_style=True)
-
-    def use_profile(self, adapter_type):
-        self.adapter_type = adapter_type
-
-        profile_config = {}
-        default_profile_config = self.get_profile(adapter_type)
-
-        profile_config.update(default_profile_config)
-        profile_config.update(self.profile_config)
-
-        if not os.path.exists(self.test_root_dir):
-            os.makedirs(self.test_root_dir)
-
-        flags.PROFILES_DIR = self.test_root_dir
-        profiles_path = os.path.join(self.test_root_dir, 'profiles.yml')
-        with open(profiles_path, 'w') as f:
-            yaml.safe_dump(profile_config, f, default_flow_style=True)
-        self._profile_config = profile_config
-
-    def set_packages(self):
-        if self.packages_config is not None:
-            with open('packages.yml', 'w') as f:
-                yaml.safe_dump(self.packages_config, f, default_flow_style=True)
-
-    def set_selectors(self):
-        if self.selectors_config is not None:
-            with open('selectors.yml', 'w') as f:
-                yaml.safe_dump(self.selectors_config, f, default_flow_style=True)
-
-    def load_config(self):
-        # we've written our profile and project. Now we want to instantiate a
-        # fresh adapter for the tests.
-        # it's important to use a different connection handle here so
-        # we don't look into an incomplete transaction
-        kwargs = {
-            'profile': None,
-            'profiles_dir': self.test_root_dir,
-            'target': None,
-        }
-
-        config = RuntimeConfig.from_args(TestArgs(kwargs))
-
-        register_adapter(config)
-        adapter = get_adapter(config)
-        adapter.cleanup_connections()
-        self.adapter_type = adapter.type()
-        self.adapter = adapter
-        self.config = config
-
-        self._drop_schemas()
-        self._create_schemas()
-
-    def quote_as_configured(self, value, quote_key):
-        return self.adapter.quote_as_configured(value, quote_key)
-
-    def tearDown(self):
-        # get any current run adapter and clean up its connections before we
-        # reset them. It'll probably be different from ours because
-        # handle_and_check() calls reset_adapters().
-        register_adapter(self.config)
-        adapter = get_adapter(self.config)
-        if adapter is not self.adapter:
-            adapter.cleanup_connections()
-        if not hasattr(self, 'adapter'):
-            self.adapter = adapter
-
-        self._drop_schemas()
-
-        self.adapter.cleanup_connections()
-        reset_adapters()
-        os.chdir(INITIAL_ROOT)
-        try:
-            shutil.rmtree(self.test_root_dir)
-        except EnvironmentError:
-            logger.exception('Could not clean up after test - {} not removable'
-                             .format(self.test_root_dir))
-
-    def _get_schema_fqn(self, database, schema):
-        schema_fqn = self.quote_as_configured(schema, 'schema')
-        return schema_fqn
-
-    def _create_schema_named(self, database, schema):
-        relation = self.adapter.Relation.create(database=database, schema=schema)
-        self.adapter.create_schema(relation)
-
-    def _drop_schema_named(self, database, schema):
-        relation = self.adapter.Relation.create(database=database, schema=schema)
-        self.adapter.drop_schema(relation)
-
-    def _create_schemas(self):
-        schema = self.unique_schema()
-        with self.adapter.connection_named('__test'):
-            self._create_schema_named(self.default_database, schema)
-
-    def _drop_schemas(self):
-        with self.adapter.connection_named('__test'):
-            schema = self.unique_schema()
-            self._drop_schema_named(self.default_database, schema)
-            if self.setup_alternate_db and self.alternative_database:
-                self._drop_schema_named(self.alternative_database, schema)
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-        }
-
-    @property
-    def profile_config(self):
-        return {}
-
-    def run_dbt(self, args=None, expect_pass=True, profiles_dir=True):
-        res, success = self.run_dbt_and_check(args=args, profiles_dir=profiles_dir)
-        self.assertEqual(
-            success, expect_pass,
-            "dbt exit state did not match expected")
-
-        return res
-
-
-    def run_dbt_and_capture(self, *args, **kwargs):
-        try:
-            stringbuf = StringIO()
-            capture_stdout_logs(stringbuf)
-            res = self.run_dbt(*args, **kwargs)
-            stdout = stringbuf.getvalue()
-
-        finally:
-            stop_capture_stdout_logs()
-
-        return res, stdout
-
-    def run_dbt_and_check(self, args=None, profiles_dir=True):
-        log_manager.reset_handlers()
-        if args is None:
-            args = ["run"]
-
-        final_args = []
-
-        if os.getenv('DBT_TEST_SINGLE_THREADED') in ('y', 'Y', '1'):
-            final_args.append('--single-threaded')
-
-        final_args.extend(args)
-
-        if profiles_dir:
-            final_args.extend(['--profiles-dir', self.test_root_dir])
-        final_args.append('--log-cache-events')
-
-        logger.info("Invoking dbt with {}".format(final_args))
-        return dbt.handle_and_check(final_args)
-
-    def run_sql_file(self, path, kwargs=None):
-        with open(path, 'r') as f:
-            statements = f.read().split(";")
-            for statement in statements:
-                self.run_sql(statement, kwargs=kwargs)
-
-    def transform_sql(self, query, kwargs=None):
-        to_return = query
-
-        base_kwargs = {
-            'schema': self.unique_schema(),
-            'database': self.adapter.quote(self.default_database),
-        }
-        if kwargs is None:
-            kwargs = {}
-        base_kwargs.update(kwargs)
-
-        to_return = to_return.format(**base_kwargs)
-
-        return to_return
-
-    def run_sql_bigquery(self, sql, fetch):
-        """Run an SQL query on a bigquery adapter. No cursors, transactions,
-        etc. to worry about"""
-
-        do_fetch = fetch != 'None'
-        _, res = self.adapter.execute(sql, fetch=do_fetch)
-
-        # convert dataframe to matrix-ish repr
-        if fetch == 'one':
-            return res[0]
-        else:
-            return list(res)
-
-    def run_sql(self, query, fetch='None', kwargs=None, connection_name=None):
-        if connection_name is None:
-            connection_name = '__test'
-
-        if query.strip() == "":
-            return
-
-        sql = self.transform_sql(query, kwargs=kwargs)
-
-        with self.get_connection(connection_name) as conn:
-            logger.debug('test connection "{}" executing: {}'.format(conn.name, sql))
-            return self.run_sql_bigquery(sql, fetch)
-
-    def _ilike(self, target, value):
-        return "{} ilike '{}'".format(target, value)
-
-    def get_many_table_columns_bigquery(self, tables, schema, database=None):
-        result = []
-        for table in tables:
-            relation = self._make_relation(table, schema, database)
-            columns = self.adapter.get_columns_in_relation(relation)
-            for col in columns:
-                result.append((table, col.column, col.dtype, col.char_size))
-        return result
-
-    def get_many_table_columns(self, tables, schema, database=None):
-        result = self.get_many_table_columns_bigquery(tables, schema, database)
-        result.sort(key=lambda x: '{}.{}'.format(x[0], x[1]))
-        return result
-
-    def filter_many_columns(self, column):
-        if len(column) == 3:
-            table_name, column_name, data_type = column
-            char_size = None
-        else:
-            table_name, column_name, data_type, char_size = column
-        return (table_name, column_name, data_type, char_size)
-
-    @contextmanager
-    def get_connection(self, name=None):
-        """Create a test connection context where all executed macros, etc will
-        get self.adapter as the adapter.
-
-        This allows tests to run normal adapter macros as if reset_adapters()
-        were not called by handle_and_check (for asserts, etc)
-        """
-        if name is None:
-            name = '__test'
-        with patch.object(providers, 'get_adapter', return_value=self.adapter):
-            with self.adapter.connection_named(name):
-                conn = self.adapter.connections.get_thread_connection()
-                yield conn
-
-    def get_relation_columns(self, relation):
-        with self.get_connection():
-            columns = self.adapter.get_columns_in_relation(relation)
-
-        return sorted(((c.name, c.dtype, c.char_size) for c in columns),
-                      key=lambda x: x[0])
-
-    def get_table_columns(self, table, schema=None, database=None):
-        schema = self.unique_schema() if schema is None else schema
-        database = self.default_database if database is None else database
-        relation = self.adapter.Relation.create(
-            database=database,
-            schema=schema,
-            identifier=table,
-            type='table',
-            quote_policy=self.config.quoting
-        )
-        return self.get_relation_columns(relation)
-
-    def get_table_columns_as_dict(self, tables, schema=None):
-        col_matrix = self.get_many_table_columns(tables, schema)
-        res = {}
-        for row in col_matrix:
-            table_name = row[0]
-            col_def = row[1:]
-            if table_name not in res:
-                res[table_name] = []
-            res[table_name].append(col_def)
-        return res
-
-    def get_models_in_schema(self, schema=None):
-        schema = self.unique_schema() if schema is None else schema
-        sql = """
-                select table_name,
-                        case when table_type = 'BASE TABLE' then 'table'
-                             when table_type = 'VIEW' then 'view'
-                             else table_type
-                        end as materialization
-                from information_schema.tables
-                where {}
-                order by table_name
-                """
-
-        sql = sql.format(self._ilike('table_schema', schema))
-        result = self.run_sql(sql, fetch='all')
-
-        return {model_name: materialization for (model_name, materialization) in result}
-
-    def _assertTablesEqualSql(self, relation_a, relation_b, columns=None):
-        if columns is None:
-            columns = self.get_relation_columns(relation_a)
-        column_names = [c[0] for c in columns]
-
-        sql = self.adapter.get_rows_different_sql(
-            relation_a, relation_b, column_names
-        )
-
-        return sql
-
-    def assertTablesEqual(self, table_a, table_b,
-                          table_a_schema=None, table_b_schema=None,
-                          table_a_db=None, table_b_db=None):
-        if table_a_schema is None:
-            table_a_schema = self.unique_schema()
-
-        if table_b_schema is None:
-            table_b_schema = self.unique_schema()
-
-        if table_a_db is None:
-            table_a_db = self.default_database
-
-        if table_b_db is None:
-            table_b_db = self.default_database
-
-        relation_a = self._make_relation(table_a, table_a_schema, table_a_db)
-        relation_b = self._make_relation(table_b, table_b_schema, table_b_db)
-
-        self._assertTableColumnsEqual(relation_a, relation_b)
-
-        sql = self._assertTablesEqualSql(relation_a, relation_b)
-        result = self.run_sql(sql, fetch='one')
-
-        self.assertEqual(
-            result[0],
-            0,
-            'row_count_difference nonzero: ' + sql
-        )
-        self.assertEqual(
-            result[1],
-            0,
-            'num_mismatched nonzero: ' + sql
-        )
-
-    def _make_relation(self, identifier, schema=None, database=None):
-        if schema is None:
-            schema = self.unique_schema()
-        if database is None:
-            database = self.default_database
-        return self.adapter.Relation.create(
-            database=database,
-            schema=schema,
-            identifier=identifier,
-            quote_policy=self.config.quoting
-        )
-
-    def get_many_relation_columns(self, relations):
-        """Returns a dict of (datbase, schema) -> (dict of (table_name -> list of columns))
-        """
-        schema_fqns = {}
-        for rel in relations:
-            this_schema = schema_fqns.setdefault((rel.database, rel.schema), [])
-            this_schema.append(rel.identifier)
-
-        column_specs = {}
-        for key, tables in schema_fqns.items():
-            database, schema = key
-            columns = self.get_many_table_columns(tables, schema, database=database)
-            table_columns = {}
-            for col in columns:
-                table_columns.setdefault(col[0], []).append(col[1:])
-            for rel_name, columns in table_columns.items():
-                key = (database, schema, rel_name)
-                column_specs[key] = columns
-
-        return column_specs
-
-    def assertManyRelationsEqual(self, relations, default_schema=None, default_database=None):
-        if default_schema is None:
-            default_schema = self.unique_schema()
-        if default_database is None:
-            default_database = self.default_database
-
-        specs = []
-        for relation in relations:
-            if not isinstance(relation, (tuple, list)):
-                relation = [relation]
-
-            assert len(relation) <= 3
-
-            if len(relation) == 3:
-                relation = self._make_relation(*relation)
-            elif len(relation) == 2:
-                relation = self._make_relation(relation[0], relation[1], default_database)
-            elif len(relation) == 1:
-                relation = self._make_relation(relation[0], default_schema, default_database)
-            else:
-                raise ValueError('relation must be a sequence of 1, 2, or 3 values')
-
-            specs.append(relation)
-
-        with self.get_connection():
-            column_specs = self.get_many_relation_columns(specs)
-
-        # make sure everyone has equal column definitions
-        first_columns = None
-        for relation in specs:
-            key = (relation.database, relation.schema, relation.identifier)
-            # get a good error here instead of a hard-to-diagnose KeyError
-            self.assertIn(key, column_specs, f'No columns found for {key}')
-            columns = column_specs[key]
-            if first_columns is None:
-                first_columns = columns
-            else:
-                self.assertEqual(
-                    first_columns, columns,
-                    '{} did not match {}'.format(str(specs[0]), str(relation))
-                )
-
-        # make sure everyone has the same data. if we got here, everyone had
-        # the same column specs!
-        first_relation = None
-        for relation in specs:
-            if first_relation is None:
-                first_relation = relation
-            else:
-                sql = self._assertTablesEqualSql(first_relation, relation,
-                                                 columns=first_columns)
-                result = self.run_sql(sql, fetch='one')
-
-                self.assertEqual(
-                    result[0],
-                    0,
-                    'row_count_difference nonzero: ' + sql
-                )
-                self.assertEqual(
-                    result[1],
-                    0,
-                    'num_mismatched nonzero: ' + sql
-                )
-
-    def assertManyTablesEqual(self, *args):
-        schema = self.unique_schema()
-
-        all_tables = []
-        for table_equivalencies in args:
-            all_tables += list(table_equivalencies)
-
-        all_cols = self.get_table_columns_as_dict(all_tables, schema)
-
-        for table_equivalencies in args:
-            first_table = table_equivalencies[0]
-            first_relation = self._make_relation(first_table)
-
-            # assert that all tables have the same columns
-            base_result = all_cols[first_table]
-            self.assertTrue(len(base_result) > 0)
-
-            for other_table in table_equivalencies[1:]:
-                other_result = all_cols[other_table]
-                self.assertTrue(len(other_result) > 0)
-                self.assertEqual(base_result, other_result)
-
-                other_relation = self._make_relation(other_table)
-                sql = self._assertTablesEqualSql(first_relation,
-                                                 other_relation,
-                                                 columns=base_result)
-                result = self.run_sql(sql, fetch='one')
-
-                self.assertEqual(
-                    result[0],
-                    0,
-                    'row_count_difference nonzero: ' + sql
-                )
-                self.assertEqual(
-                    result[1],
-                    0,
-                    'num_mismatched nonzero: ' + sql
-                )
-
-
-    def _assertTableRowCountsEqual(self, relation_a, relation_b):
-        cmp_query = """
-            with table_a as (
-
-                select count(*) as num_rows from {}
-
-            ), table_b as (
-
-                select count(*) as num_rows from {}
-
-            )
-
-            select table_a.num_rows - table_b.num_rows as difference
-            from table_a, table_b
-
-        """.format(str(relation_a), str(relation_b))
-
-        res = self.run_sql(cmp_query, fetch='one')
-
-        self.assertEqual(int(res[0]), 0, "Row count of table {} doesn't match row count of table {}. ({} rows different)".format(
-                relation_a.identifier,
-                relation_b.identifier,
-                res[0]
-            )
-        )
-
-    def assertTableDoesNotExist(self, table, schema=None, database=None):
-        columns = self.get_table_columns(table, schema, database)
-
-        self.assertEqual(
-            len(columns),
-            0
-        )
-
-    def assertTableDoesExist(self, table, schema=None, database=None):
-        columns = self.get_table_columns(table, schema, database)
-
-        self.assertGreater(
-            len(columns),
-            0
-        )
-
-    def _assertTableColumnsEqual(self, relation_a, relation_b):
-        table_a_result = self.get_relation_columns(relation_a)
-        table_b_result = self.get_relation_columns(relation_b)
-
-        text_types = {'text', 'character varying', 'character', 'varchar'}
-
-        self.assertEqual(len(table_a_result), len(table_b_result))
-        for a_column, b_column in zip(table_a_result, table_b_result):
-            a_name, a_type, a_size = a_column
-            b_name, b_type, b_size = b_column
-            self.assertEqual(a_name, b_name,
-                '{} vs {}: column "{}" != "{}"'.format(
-                    relation_a, relation_b, a_name, b_name
-                ))
-
-            self.assertEqual(a_type, b_type,
-                '{} vs {}: column "{}" has type "{}" != "{}"'.format(
-                    relation_a, relation_b, a_name, a_type, b_type
-                ))
-
-            self.assertEqual(a_size, b_size,
-                '{} vs {}: column "{}" has size "{}" != "{}"'.format(
-                    relation_a, relation_b, a_name, a_size, b_size
-                ))
-
-    def assertEquals(self, *args, **kwargs):
-        # assertEquals is deprecated. This makes the warnings less chatty
-        self.assertEqual(*args, **kwargs)
-
-    def assertBetween(self, timestr, start, end=None):
-        datefmt = '%Y-%m-%dT%H:%M:%S.%fZ'
-        if end is None:
-            end = datetime.utcnow()
-
-        parsed = datetime.strptime(timestr, datefmt)
-
-        self.assertLessEqual(start, parsed,
-            'parsed date {} happened before {}'.format(
-                parsed,
-                start.strftime(datefmt))
-        )
-        self.assertGreaterEqual(end, parsed,
-            'parsed date {} happened after {}'.format(
-                parsed,
-                end.strftime(datefmt))
-        )
-
-
-def use_profile(profile_name):
-    """A decorator to declare a test method as using a particular profile.
-    Handles both setting the nose attr and calling self.use_profile.
-
-    Use like this:
-
-    class TestSomething(DBIntegrationTest):
-        @use_profile('postgres')
-        def test_postgres_thing(self):
-            self.assertEqual(self.adapter_type, 'postgres')
-
-        @use_profile('snowflake')
-        def test_snowflake_thing(self):
-            self.assertEqual(self.adapter_type, 'snowflake')
-    """
-    def outer(wrapped):
-        @getattr(pytest.mark, 'profile_'+profile_name)
-        @wraps(wrapped)
-        def func(self, *args, **kwargs):
-            return wrapped(self, *args, **kwargs)
-        # sanity check at import time
-        assert _profile_from_test_name(wrapped.__name__) == profile_name
-        return func
-    return outer
-
-
-class AnyFloat:
-    """Any float. Use this in assertEqual() calls to assert that it is a float.
-    """
-    def __eq__(self, other):
-        return isinstance(other, float)
-
-
-class AnyString:
-    """Any string. Use this in assertEqual() calls to assert that it is a string.
-    """
-    def __eq__(self, other):
-        return isinstance(other, str)
-
-
-class AnyStringWith:
-    def __init__(self, contains=None):
-        self.contains = contains
-
-    def __eq__(self, other):
-        if not isinstance(other, str):
-            return False
-
-        if self.contains is None:
-            return True
-
-        return self.contains in other
-
-    def __repr__(self):
-        return 'AnyStringWith<{!r}>'.format(self.contains)
-
-
-def bigquery_rate_limiter(err, *args):
-    msg = str(err)
-    if 'too many table update operations for this table' in msg:
-        time.sleep(1)
-        return True
-    return False
-
-
-def get_manifest():
-    path = './target/partial_parse.msgpack'
-    if os.path.exists(path):
-        with open(path, 'rb') as fp:
-            manifest_mp = fp.read()
-        manifest: Manifest = Manifest.from_msgpack(manifest_mp)
-        return manifest
-    else:
-        return None
diff --git a/tests/integration/simple_snapshot_test/add_column_to_source_bq.sql b/tests/integration/simple_snapshot_test/add_column_to_source_bq.sql
deleted file mode 100644
index e1babb82c..000000000
--- a/tests/integration/simple_snapshot_test/add_column_to_source_bq.sql
+++ /dev/null
@@ -1,56 +0,0 @@
-
-create or replace table {schema}.seed as (
-
-    select *,
-        [
-            struct(
-                1 as field_1,
-                2 as field_2
-            ),
-            struct(
-                3 as field_1,
-                4 as field_2
-            )
-        ] as repeated_nested_field,
-
-        struct(
-            1 as field_1,
-            2 as field_2
-        ) as nested_field,
-
-        [
-            1,
-            2
-        ] as repeated_field
-
-    from {schema}.seed
-
-);
-
-create or replace table {schema}.snapshot_expected as (
-
-    select *,
-        [
-            struct(
-                1 as field_1,
-                2 as field_2
-            ),
-            struct(
-                3 as field_1,
-                4 as field_2
-            )
-        ] as repeated_nested_field,
-
-        struct(
-            1 as field_1,
-            2 as field_2
-        ) as nested_field,
-
-        [
-            1,
-            2
-        ] as repeated_field
-
-    from {schema}.snapshot_expected
-
-);
diff --git a/tests/integration/simple_snapshot_test/check-snapshots-expected/check_snapshots_test_current.sql b/tests/integration/simple_snapshot_test/check-snapshots-expected/check_snapshots_test_current.sql
deleted file mode 100644
index 414afb472..000000000
--- a/tests/integration/simple_snapshot_test/check-snapshots-expected/check_snapshots_test_current.sql
+++ /dev/null
@@ -1,51 +0,0 @@
-
-
-with query as (
-
-    -- check that the current value for id=1 is red
-    select case when (
-        select count(*)
-        from {{ ref('check_cols_cycle') }}
-        where id = 1 and color = 'red' and dbt_valid_to is null
-    ) = 1 then 0 else 1 end as failures
-
-    union all
-
-    -- check that the previous 'red' value for id=1 is invalidated
-    select case when (
-        select count(*)
-        from {{ ref('check_cols_cycle') }}
-        where id = 1 and color = 'red' and dbt_valid_to is not null
-    ) = 1 then 0 else 1 end as failures
-
-    union all
-
-    -- check that there's only one current record for id=2
-    select case when (
-        select count(*)
-        from {{ ref('check_cols_cycle') }}
-        where id = 2 and color = 'pink' and dbt_valid_to is null
-    ) = 1 then 0 else 1 end as failures
-
-    union all
-
-    -- check that the previous value for id=2 is represented
-    select case when (
-        select count(*)
-        from {{ ref('check_cols_cycle') }}
-        where id = 2 and color = 'green' and dbt_valid_to is not null
-    ) = 1 then 0 else 1 end as failures
-
-    union all
-
-    -- check that there are 5 records total in the table
-    select case when (
-        select count(*)
-        from {{ ref('check_cols_cycle') }}
-    ) = 5 then 0 else 1 end as failures
-
-)
-
-select *
-from query
-where failures = 1
diff --git a/tests/integration/simple_snapshot_test/check-snapshots/check_cols_cycle.sql b/tests/integration/simple_snapshot_test/check-snapshots/check_cols_cycle.sql
deleted file mode 100644
index 8b36f35a1..000000000
--- a/tests/integration/simple_snapshot_test/check-snapshots/check_cols_cycle.sql
+++ /dev/null
@@ -1,33 +0,0 @@
-
-{% snapshot check_cols_cycle %}
-
-    {{
-        config(
-            target_database=database,
-            target_schema=schema,
-            unique_key='id',
-            strategy='check',
-            check_cols=['color']
-        )
-    }}
-
-    {% if var('version') == 1 %}
-
-        select 1 as id, 'red' as color union all
-        select 2 as id, 'green' as color
-
-    {% elif var('version') == 2 %}
-
-        select 1 as id, 'blue' as color union all
-        select 2 as id, 'green' as color
-
-    {% elif var('version') == 3 %}
-
-        select 1 as id, 'red' as color union all
-        select 2 as id, 'pink' as color
-
-    {% else %}
-        {% do exceptions.raise_compiler_error("Got bad version: " ~ var('version')) %}
-    {% endif %}
-
-{% endsnapshot %}
diff --git a/tests/integration/simple_snapshot_test/custom-snapshot-macros/custom.sql b/tests/integration/simple_snapshot_test/custom-snapshot-macros/custom.sql
deleted file mode 100644
index 4347088e4..000000000
--- a/tests/integration/simple_snapshot_test/custom-snapshot-macros/custom.sql
+++ /dev/null
@@ -1,18 +0,0 @@
-{# A "custom" strategy that's really just the timestamp one #}
-{% macro snapshot_custom_strategy(node, snapshotted_rel, current_rel, config, target_exists) %}
-    {% set primary_key = config['unique_key'] %}
-    {% set updated_at = config['updated_at'] %}
-
-    {% set row_changed_expr -%}
-        ({{ snapshotted_rel }}.{{ updated_at }} < {{ current_rel }}.{{ updated_at }})
-    {%- endset %}
-
-    {% set scd_id_expr = snapshot_hash_arguments([primary_key, updated_at]) %}
-
-    {% do return({
-        "unique_key": primary_key,
-        "updated_at": updated_at,
-        "row_changed": row_changed_expr,
-        "scd_id": scd_id_expr
-    }) %}
-{% endmacro %}
diff --git a/tests/integration/simple_snapshot_test/invalidate_bigquery.sql b/tests/integration/simple_snapshot_test/invalidate_bigquery.sql
deleted file mode 100644
index d4641d451..000000000
--- a/tests/integration/simple_snapshot_test/invalidate_bigquery.sql
+++ /dev/null
@@ -1,12 +0,0 @@
-
--- update records 11 - 21. Change email and updated_at field
-update {database}.{schema}.seed set
-    updated_at = timestamp_add(updated_at, interval 1 hour),
-    email      = case when id = 20 then 'pfoxj@creativecommons.org' else concat('new_', email) end
-where id >= 10 and id <= 20;
-
-
--- invalidate records 11 - 21
-update {database}.{schema}.snapshot_expected set
-    dbt_valid_to   = timestamp_add(updated_at, interval 1 hour)
-where id >= 10 and id <= 20;
diff --git a/tests/integration/simple_snapshot_test/macros/test_no_overlaps.sql b/tests/integration/simple_snapshot_test/macros/test_no_overlaps.sql
deleted file mode 100644
index 6d432193c..000000000
--- a/tests/integration/simple_snapshot_test/macros/test_no_overlaps.sql
+++ /dev/null
@@ -1,85 +0,0 @@
-{% macro get_snapshot_unique_id() -%}
-    {{ return(adapter.dispatch('get_snapshot_unique_id')()) }}
-{%- endmacro %}
-
-{% macro default__get_snapshot_unique_id() -%}
-  {% do return("id || '-' || first_name") %}
-{%- endmacro %}
-
-
-{% macro bigquery__get_snapshot_unique_id() -%}
-    {%- do return('concat(cast(id as string), "-", first_name)') -%}
-{%- endmacro %}
-
-{#
-    mostly copy+pasted from dbt_utils, but I removed some parameters and added
-    a query that calls get_snapshot_unique_id
-#}
-{% test mutually_exclusive_ranges(model) %}
-
-with base as (
-    select {{ get_snapshot_unique_id() }} as dbt_unique_id,
-    *
-    from {{ model }}
-),
-window_functions as (
-
-    select
-        dbt_valid_from as lower_bound,
-        coalesce(dbt_valid_to, '2099-1-1T00:00:01') as upper_bound,
-
-        lead(dbt_valid_from) over (
-            partition by dbt_unique_id
-            order by dbt_valid_from
-        ) as next_lower_bound,
-
-        row_number() over (
-            partition by dbt_unique_id
-            order by dbt_valid_from desc
-        ) = 1 as is_last_record
-
-    from base
-
-),
-
-calc as (
-    -- We want to return records where one of our assumptions fails, so we'll use
-    -- the `not` function with `and` statements so we can write our assumptions nore cleanly
-    select
-        *,
-
-        -- For each record: lower_bound should be < upper_bound.
-        -- Coalesce it to return an error on the null case (implicit assumption
-        -- these columns are not_null)
-        coalesce(
-            lower_bound < upper_bound,
-            is_last_record
-        ) as lower_bound_less_than_upper_bound,
-
-        -- For each record: upper_bound {{ allow_gaps_operator }} the next lower_bound.
-        -- Coalesce it to handle null cases for the last record.
-        coalesce(
-            upper_bound = next_lower_bound,
-            is_last_record,
-            false
-        ) as upper_bound_equal_to_next_lower_bound
-
-    from window_functions
-
-),
-
-validation_errors as (
-
-    select
-        *
-    from calc
-
-    where not(
-        -- THE FOLLOWING SHOULD BE TRUE --
-        lower_bound_less_than_upper_bound
-        and upper_bound_equal_to_next_lower_bound
-    )
-)
-
-select * from validation_errors
-{% endtest %}
diff --git a/tests/integration/simple_snapshot_test/models/.gitkeep b/tests/integration/simple_snapshot_test/models/.gitkeep
deleted file mode 100644
index e69de29bb..000000000
diff --git a/tests/integration/simple_snapshot_test/models/ref_snapshot.sql b/tests/integration/simple_snapshot_test/models/ref_snapshot.sql
deleted file mode 100644
index c453929ce..000000000
--- a/tests/integration/simple_snapshot_test/models/ref_snapshot.sql
+++ /dev/null
@@ -1 +0,0 @@
-select * from {{ ref('snapshot_actual') }}
diff --git a/tests/integration/simple_snapshot_test/models/schema.yml b/tests/integration/simple_snapshot_test/models/schema.yml
deleted file mode 100644
index 259e55b95..000000000
--- a/tests/integration/simple_snapshot_test/models/schema.yml
+++ /dev/null
@@ -1,5 +0,0 @@
-version: 2
-snapshots:
-  - name: snapshot_actual
-    tests:
-      - mutually_exclusive_ranges
diff --git a/tests/integration/simple_snapshot_test/seed.sql b/tests/integration/simple_snapshot_test/seed.sql
deleted file mode 100644
index 8f3422e36..000000000
--- a/tests/integration/simple_snapshot_test/seed.sql
+++ /dev/null
@@ -1,220 +0,0 @@
-create table {database}.{schema}.seed (
-	id INTEGER,
-	first_name VARCHAR(50),
-	last_name VARCHAR(50),
-	email VARCHAR(50),
-	gender VARCHAR(50),
-	ip_address VARCHAR(20),
-	updated_at TIMESTAMP WITHOUT TIME ZONE
-);
-
-create table {database}.{schema}.snapshot_expected (
-	id INTEGER,
-	first_name VARCHAR(50),
-	last_name VARCHAR(50),
-	email VARCHAR(50),
-	gender VARCHAR(50),
-	ip_address VARCHAR(20),
-
-	-- snapshotting fields
-	updated_at TIMESTAMP WITHOUT TIME ZONE,
-	dbt_valid_from TIMESTAMP WITHOUT TIME ZONE,
-	dbt_valid_to   TIMESTAMP WITHOUT TIME ZONE,
-	dbt_scd_id     VARCHAR(32),
-	dbt_updated_at TIMESTAMP WITHOUT TIME ZONE
-);
-
-
--- seed inserts
-insert into {database}.{schema}.seed (id, first_name, last_name, email, gender, ip_address, updated_at) values
-(1, 'Judith', 'Kennedy', 'jkennedy0@phpbb.com', 'Female', '54.60.24.128', '2015-12-24 12:19:28'),
-(2, 'Arthur', 'Kelly', 'akelly1@eepurl.com', 'Male', '62.56.24.215', '2015-10-28 16:22:15'),
-(3, 'Rachel', 'Moreno', 'rmoreno2@msu.edu', 'Female', '31.222.249.23', '2016-04-05 02:05:30'),
-(4, 'Ralph', 'Turner', 'rturner3@hp.com', 'Male', '157.83.76.114', '2016-08-08 00:06:51'),
-(5, 'Laura', 'Gonzales', 'lgonzales4@howstuffworks.com', 'Female', '30.54.105.168', '2016-09-01 08:25:38'),
-(6, 'Katherine', 'Lopez', 'klopez5@yahoo.co.jp', 'Female', '169.138.46.89', '2016-08-30 18:52:11'),
-(7, 'Jeremy', 'Hamilton', 'jhamilton6@mozilla.org', 'Male', '231.189.13.133', '2016-07-17 02:09:46'),
-(8, 'Heather', 'Rose', 'hrose7@goodreads.com', 'Female', '87.165.201.65', '2015-12-29 22:03:56'),
-(9, 'Gregory', 'Kelly', 'gkelly8@trellian.com', 'Male', '154.209.99.7', '2016-03-24 21:18:16'),
-(10, 'Rachel', 'Lopez', 'rlopez9@themeforest.net', 'Female', '237.165.82.71', '2016-08-20 15:44:49'),
-(11, 'Donna', 'Welch', 'dwelcha@shutterfly.com', 'Female', '103.33.110.138', '2016-02-27 01:41:48'),
-(12, 'Russell', 'Lawrence', 'rlawrenceb@qq.com', 'Male', '189.115.73.4', '2016-06-11 03:07:09'),
-(13, 'Michelle', 'Montgomery', 'mmontgomeryc@scientificamerican.com', 'Female', '243.220.95.82', '2016-06-18 16:27:19'),
-(14, 'Walter', 'Castillo', 'wcastillod@pagesperso-orange.fr', 'Male', '71.159.238.196', '2016-10-06 01:55:44'),
-(15, 'Robin', 'Mills', 'rmillse@vkontakte.ru', 'Female', '172.190.5.50', '2016-10-31 11:41:21'),
-(16, 'Raymond', 'Holmes', 'rholmesf@usgs.gov', 'Male', '148.153.166.95', '2016-10-03 08:16:38'),
-(17, 'Gary', 'Bishop', 'gbishopg@plala.or.jp', 'Male', '161.108.182.13', '2016-08-29 19:35:20'),
-(18, 'Anna', 'Riley', 'arileyh@nasa.gov', 'Female', '253.31.108.22', '2015-12-11 04:34:27'),
-(19, 'Sarah', 'Knight', 'sknighti@foxnews.com', 'Female', '222.220.3.177', '2016-09-26 00:49:06'),
-(20, 'Phyllis', 'Fox', null, 'Female', '163.191.232.95', '2016-08-21 10:35:19');
-
-
--- populate snapshot table
-insert into {database}.{schema}.snapshot_expected (
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    dbt_valid_from,
-    dbt_valid_to,
-    dbt_updated_at,
-    dbt_scd_id
-)
-
-select
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    -- fields added by snapshotting
-    updated_at as dbt_valid_from,
-    null::timestamp as dbt_valid_to,
-    updated_at as dbt_updated_at,
-    md5(id || '-' || first_name || '|' || updated_at::text) as dbt_scd_id
-from {database}.{schema}.seed;
-
-create table {database}.{schema}.snapshot_castillo_expected (
-    id INTEGER,
-    first_name VARCHAR(50),
-    last_name VARCHAR(50),
-    email VARCHAR(50),
-    gender VARCHAR(50),
-    ip_address VARCHAR(20),
-
-    -- snapshotting fields
-    "1-updated_at" TIMESTAMP WITHOUT TIME ZONE,
-    dbt_valid_from TIMESTAMP WITHOUT TIME ZONE,
-    dbt_valid_to   TIMESTAMP WITHOUT TIME ZONE,
-    dbt_scd_id     VARCHAR(32),
-    dbt_updated_at TIMESTAMP WITHOUT TIME ZONE
-
-);
-
--- one entry
-insert into {database}.{schema}.snapshot_castillo_expected (
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    "1-updated_at",
-    dbt_valid_from,
-    dbt_valid_to,
-    dbt_updated_at,
-    dbt_scd_id
-)
-
-select
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    -- fields added by snapshotting
-    updated_at as dbt_valid_from,
-    null::timestamp as dbt_valid_to,
-    updated_at as dbt_updated_at,
-    md5(id || '-' || first_name || '|' || updated_at::text) as dbt_scd_id
-from {database}.{schema}.seed where last_name = 'Castillo';
-
-create table {database}.{schema}.snapshot_alvarez_expected (
-    id INTEGER,
-    first_name VARCHAR(50),
-    last_name VARCHAR(50),
-    email VARCHAR(50),
-    gender VARCHAR(50),
-    ip_address VARCHAR(20),
-
-    -- snapshotting fields
-    updated_at TIMESTAMP WITHOUT TIME ZONE,
-    dbt_valid_from TIMESTAMP WITHOUT TIME ZONE,
-    dbt_valid_to   TIMESTAMP WITHOUT TIME ZONE,
-    dbt_scd_id     VARCHAR(32),
-    dbt_updated_at TIMESTAMP WITHOUT TIME ZONE
-);
-
--- 0 entries
-insert into {database}.{schema}.snapshot_alvarez_expected (
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    dbt_valid_from,
-    dbt_valid_to,
-    dbt_updated_at,
-    dbt_scd_id
-)
-
-select
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    -- fields added by snapshotting
-    updated_at as dbt_valid_from,
-    null::timestamp as dbt_valid_to,
-    updated_at as dbt_updated_at,
-    md5(id || '-' || first_name || '|' || updated_at::text) as dbt_scd_id
-from {database}.{schema}.seed where last_name = 'Alvarez';
-
-create table {database}.{schema}.snapshot_kelly_expected (
-    id INTEGER,
-    first_name VARCHAR(50),
-    last_name VARCHAR(50),
-    email VARCHAR(50),
-    gender VARCHAR(50),
-    ip_address VARCHAR(20),
-
-    -- snapshotting fields
-    updated_at TIMESTAMP WITHOUT TIME ZONE,
-    dbt_valid_from TIMESTAMP WITHOUT TIME ZONE,
-    dbt_valid_to   TIMESTAMP WITHOUT TIME ZONE,
-    dbt_scd_id     VARCHAR(32),
-    dbt_updated_at TIMESTAMP WITHOUT TIME ZONE
-);
-
-
--- 2 entries
-insert into {database}.{schema}.snapshot_kelly_expected (
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    dbt_valid_from,
-    dbt_valid_to,
-    dbt_updated_at,
-    dbt_scd_id
-)
-
-select
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    -- fields added by snapshotting
-    updated_at as dbt_valid_from,
-    null::timestamp as dbt_valid_to,
-    updated_at as dbt_updated_at,
-    md5(id || '-' || first_name || '|' || updated_at::text) as dbt_scd_id
-from {database}.{schema}.seed where last_name = 'Kelly';
diff --git a/tests/integration/simple_snapshot_test/seed_bq.sql b/tests/integration/simple_snapshot_test/seed_bq.sql
deleted file mode 100644
index 5ea93fee4..000000000
--- a/tests/integration/simple_snapshot_test/seed_bq.sql
+++ /dev/null
@@ -1,81 +0,0 @@
-create table {database}.{schema}.seed (
-	id INT64,
-	first_name STRING,
-	last_name STRING,
-	email STRING,
-	gender STRING,
-	ip_address STRING,
-	updated_at TIMESTAMP
-);
-
-create table {database}.{schema}.snapshot_expected (
-	id INT64,
-	first_name STRING,
-	last_name STRING,
-	email STRING,
-	gender STRING,
-	ip_address STRING,
-
-	-- snapshotting fields
-	updated_at TIMESTAMP,
-	dbt_valid_from TIMESTAMP,
-	dbt_valid_to   TIMESTAMP,
-	dbt_scd_id     STRING,
-	dbt_updated_at TIMESTAMP
-);
-
-
--- seed inserts
-insert {database}.{schema}.seed (id, first_name, last_name, email, gender, ip_address, updated_at) values
-(1, 'Judith', 'Kennedy', 'jkennedy0@phpbb.com', 'Female', '54.60.24.128', '2015-12-24 12:19:28'),
-(2, 'Arthur', 'Kelly', 'akelly1@eepurl.com', 'Male', '62.56.24.215', '2015-10-28 16:22:15'),
-(3, 'Rachel', 'Moreno', 'rmoreno2@msu.edu', 'Female', '31.222.249.23', '2016-04-05 02:05:30'),
-(4, 'Ralph', 'Turner', 'rturner3@hp.com', 'Male', '157.83.76.114', '2016-08-08 00:06:51'),
-(5, 'Laura', 'Gonzales', 'lgonzales4@howstuffworks.com', 'Female', '30.54.105.168', '2016-09-01 08:25:38'),
-(6, 'Katherine', 'Lopez', 'klopez5@yahoo.co.jp', 'Female', '169.138.46.89', '2016-08-30 18:52:11'),
-(7, 'Jeremy', 'Hamilton', 'jhamilton6@mozilla.org', 'Male', '231.189.13.133', '2016-07-17 02:09:46'),
-(8, 'Heather', 'Rose', 'hrose7@goodreads.com', 'Female', '87.165.201.65', '2015-12-29 22:03:56'),
-(9, 'Gregory', 'Kelly', 'gkelly8@trellian.com', 'Male', '154.209.99.7', '2016-03-24 21:18:16'),
-(10, 'Rachel', 'Lopez', 'rlopez9@themeforest.net', 'Female', '237.165.82.71', '2016-08-20 15:44:49'),
-(11, 'Donna', 'Welch', 'dwelcha@shutterfly.com', 'Female', '103.33.110.138', '2016-02-27 01:41:48'),
-(12, 'Russell', 'Lawrence', 'rlawrenceb@qq.com', 'Male', '189.115.73.4', '2016-06-11 03:07:09'),
-(13, 'Michelle', 'Montgomery', 'mmontgomeryc@scientificamerican.com', 'Female', '243.220.95.82', '2016-06-18 16:27:19'),
-(14, 'Walter', 'Castillo', 'wcastillod@pagesperso-orange.fr', 'Male', '71.159.238.196', '2016-10-06 01:55:44'),
-(15, 'Robin', 'Mills', 'rmillse@vkontakte.ru', 'Female', '172.190.5.50', '2016-10-31 11:41:21'),
-(16, 'Raymond', 'Holmes', 'rholmesf@usgs.gov', 'Male', '148.153.166.95', '2016-10-03 08:16:38'),
-(17, 'Gary', 'Bishop', 'gbishopg@plala.or.jp', 'Male', '161.108.182.13', '2016-08-29 19:35:20'),
-(18, 'Anna', 'Riley', 'arileyh@nasa.gov', 'Female', '253.31.108.22', '2015-12-11 04:34:27'),
-(19, 'Sarah', 'Knight', 'sknighti@foxnews.com', 'Female', '222.220.3.177', '2016-09-26 00:49:06'),
-(20, 'Phyllis', 'Fox', null, 'Female', '163.191.232.95', '2016-08-21 10:35:19');
-
-
--- populate snapshot table
-insert {database}.{schema}.snapshot_expected (
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    dbt_valid_from,
-    dbt_valid_to,
-    dbt_updated_at,
-    dbt_scd_id
-)
-
-select
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    -- fields added by snapshotting
-    updated_at as dbt_valid_from,
-    cast(null as timestamp) as dbt_valid_to,
-    updated_at as dbt_updated_at,
-    to_hex(md5(concat(cast(id as string), '-', first_name, '|', cast(updated_at as string)))) as dbt_scd_id
-from {database}.{schema}.seed;
-
diff --git a/tests/integration/simple_snapshot_test/seeds/seed.csv b/tests/integration/simple_snapshot_test/seeds/seed.csv
deleted file mode 100644
index 9da8d46ff..000000000
--- a/tests/integration/simple_snapshot_test/seeds/seed.csv
+++ /dev/null
@@ -1,4 +0,0 @@
-id,first_name
-1,Judith
-2,Arthur
-3,Rachel
diff --git a/tests/integration/simple_snapshot_test/seeds/seed_newcol.csv b/tests/integration/simple_snapshot_test/seeds/seed_newcol.csv
deleted file mode 100644
index 005517bda..000000000
--- a/tests/integration/simple_snapshot_test/seeds/seed_newcol.csv
+++ /dev/null
@@ -1,4 +0,0 @@
-id,first_name,last_name
-1,Judith,Kennedy
-2,Arthur,Kelly
-3,Rachel,Moreno
diff --git a/tests/integration/simple_snapshot_test/test-check-col-snapshots-bq/snapshot.sql b/tests/integration/simple_snapshot_test/test-check-col-snapshots-bq/snapshot.sql
deleted file mode 100644
index 9c8459756..000000000
--- a/tests/integration/simple_snapshot_test/test-check-col-snapshots-bq/snapshot.sql
+++ /dev/null
@@ -1,29 +0,0 @@
-{% snapshot snapshot_actual %}
-    {# this used to be check_cols=('email',), which ought to be totally valid,
-    but is not because type systems are hard. #}
-    {{
-        config(
-            target_project=var('target_database', database),
-            target_dataset=var('target_schema', schema),
-            unique_key='concat(cast(id as string) , "-", first_name)',
-            strategy='check',
-            check_cols=['email'],
-        )
-    }}
-    select * from `{{target.database}}`.`{{schema}}`.seed
-{% endsnapshot %}
-
-
-{# This should be exactly the same #}
-{% snapshot snapshot_checkall %}
-    {{
-        config(
-            target_project=var('target_database', database),
-            target_dataset=var('target_schema', schema),
-            unique_key='concat(cast(id as string) , "-", first_name)',
-            strategy='check',
-            check_cols='all',
-        )
-    }}
-    select * from `{{target.database}}`.`{{schema}}`.seed
-{% endsnapshot %}
diff --git a/tests/integration/simple_snapshot_test/test-check-col-snapshots-noconfig/snapshot.sql b/tests/integration/simple_snapshot_test/test-check-col-snapshots-noconfig/snapshot.sql
deleted file mode 100644
index daf4cf312..000000000
--- a/tests/integration/simple_snapshot_test/test-check-col-snapshots-noconfig/snapshot.sql
+++ /dev/null
@@ -1,9 +0,0 @@
-{% snapshot snapshot_actual %}
-    select * from {{target.database}}.{{schema}}.seed
-{% endsnapshot %}
-
-{# This should be exactly the same #}
-{% snapshot snapshot_checkall %}
-	{{ config(check_cols='all') }}
-    select * from {{target.database}}.{{schema}}.seed
-{% endsnapshot %}
diff --git a/tests/integration/simple_snapshot_test/test-check-col-snapshots/snapshot.sql b/tests/integration/simple_snapshot_test/test-check-col-snapshots/snapshot.sql
deleted file mode 100644
index dd85ed753..000000000
--- a/tests/integration/simple_snapshot_test/test-check-col-snapshots/snapshot.sql
+++ /dev/null
@@ -1,28 +0,0 @@
-{% snapshot snapshot_actual %}
-
-    {{
-        config(
-            target_database=var('target_database', database),
-            target_schema=schema,
-            unique_key='id || ' ~ "'-'" ~ ' || first_name',
-            strategy='check',
-            check_cols=['email'],
-        )
-    }}
-    select * from {{target.database}}.{{schema}}.seed
-
-{% endsnapshot %}
-
-{# This should be exactly the same #}
-{% snapshot snapshot_checkall %}
-    {{
-        config(
-            target_database=var('target_database', database),
-            target_schema=schema,
-            unique_key='id || ' ~ "'-'" ~ ' || first_name',
-            strategy='check',
-            check_cols='all',
-        )
-    }}
-    select * from {{target.database}}.{{schema}}.seed
-{% endsnapshot %}
diff --git a/tests/integration/simple_snapshot_test/test-snapshots-bq/snapshot.sql b/tests/integration/simple_snapshot_test/test-snapshots-bq/snapshot.sql
deleted file mode 100644
index 7ffdedbcc..000000000
--- a/tests/integration/simple_snapshot_test/test-snapshots-bq/snapshot.sql
+++ /dev/null
@@ -1,19 +0,0 @@
-{% snapshot snapshot_actual %}
-
-    {{
-        config(
-            target_project=var('target_database', database),
-            target_dataset=var('target_schema', schema),
-            unique_key='concat(cast(id as string) , "-", first_name)',
-            strategy='timestamp',
-            updated_at='updated_at',
-        )
-    }}
-
-    {% if var('invalidate_hard_deletes', 'false') | as_bool %}
-        {{ config(invalidate_hard_deletes=True) }}
-    {% endif %}
-
-    select * from `{{target.database}}`.`{{schema}}`.seed
-
-{% endsnapshot %}
diff --git a/tests/integration/simple_snapshot_test/test-snapshots-checkall/snapshot.sql b/tests/integration/simple_snapshot_test/test-snapshots-checkall/snapshot.sql
deleted file mode 100644
index b9cd002ca..000000000
--- a/tests/integration/simple_snapshot_test/test-snapshots-checkall/snapshot.sql
+++ /dev/null
@@ -1,4 +0,0 @@
-{% snapshot my_snapshot %}
-    {{ config(check_cols='all', unique_key='id', strategy='check', target_database=database, target_schema=schema) }}
-    select * from {{ ref(var('seed_name', 'seed')) }}
-{% endsnapshot %}
diff --git a/tests/integration/simple_snapshot_test/test-snapshots-pg/snapshot.sql b/tests/integration/simple_snapshot_test/test-snapshots-pg/snapshot.sql
deleted file mode 100644
index ae5aac087..000000000
--- a/tests/integration/simple_snapshot_test/test-snapshots-pg/snapshot.sql
+++ /dev/null
@@ -1,19 +0,0 @@
-{% snapshot snapshot_actual %}
-
-    {{
-        config(
-            target_database=var('target_database', database),
-            target_schema=var('target_schema', schema),
-            unique_key='id || ' ~ "'-'" ~ ' || first_name',
-            strategy='timestamp',
-            updated_at='updated_at',
-        )
-    }}
-
-    {% if var('invalidate_hard_deletes', 'false') | as_bool %}
-        {{ config(invalidate_hard_deletes=True) }}
-    {% endif %}
-
-    select * from {{target.database}}.{{target.schema}}.seed
-
-{% endsnapshot %}
diff --git a/tests/integration/simple_snapshot_test/test-snapshots-select-noconfig/snapshot.sql b/tests/integration/simple_snapshot_test/test-snapshots-select-noconfig/snapshot.sql
deleted file mode 100644
index a62218b2c..000000000
--- a/tests/integration/simple_snapshot_test/test-snapshots-select-noconfig/snapshot.sql
+++ /dev/null
@@ -1,41 +0,0 @@
-{% snapshot snapshot_actual %}
-
-    {{
-        config(
-            target_database=var('target_database', database),
-            target_schema=var('target_schema', schema),
-        )
-    }}
-    select * from {{target.database}}.{{target.schema}}.seed
-
-{% endsnapshot %}
-
-{% snapshot snapshot_castillo %}
-
-    {{
-        config(
-            target_database=var('target_database', database),
-            updated_at='"1-updated_at"',
-        )
-    }}
-    select id,first_name,last_name,email,gender,ip_address,updated_at as "1-updated_at" from {{target.database}}.{{schema}}.seed where last_name = 'Castillo'
-
-{% endsnapshot %}
-
-{% snapshot snapshot_alvarez %}
-
-    {{
-        config(
-            target_database=var('target_database', database),
-        )
-    }}
-    select * from {{target.database}}.{{schema}}.seed where last_name = 'Alvarez'
-
-{% endsnapshot %}
-
-
-{% snapshot snapshot_kelly %}
-    {# This has no target_database set, which is allowed! #}
-    select * from {{target.database}}.{{schema}}.seed where last_name = 'Kelly'
-
-{% endsnapshot %}
diff --git a/tests/integration/simple_snapshot_test/test-snapshots-select/snapshot.sql b/tests/integration/simple_snapshot_test/test-snapshots-select/snapshot.sql
deleted file mode 100644
index 06245f36f..000000000
--- a/tests/integration/simple_snapshot_test/test-snapshots-select/snapshot.sql
+++ /dev/null
@@ -1,44 +0,0 @@
-{% snapshot snapshot_castillo %}
-
-    {{
-        config(
-            target_database=var('target_database', database),
-            target_schema=schema,
-            unique_key='id || ' ~ "'-'" ~ ' || first_name',
-            strategy='timestamp',
-            updated_at='"1-updated_at"',
-        )
-    }}
-    select id,first_name,last_name,email,gender,ip_address,updated_at as "1-updated_at" from {{target.database}}.{{schema}}.seed where last_name = 'Castillo'
-
-{% endsnapshot %}
-
-{% snapshot snapshot_alvarez %}
-
-    {{
-        config(
-            target_database=var('target_database', database),
-            target_schema=schema,
-            unique_key='id || ' ~ "'-'" ~ ' || first_name',
-            strategy='timestamp',
-            updated_at='updated_at',
-        )
-    }}
-    select * from {{target.database}}.{{schema}}.seed where last_name = 'Alvarez'
-
-{% endsnapshot %}
-
-
-{% snapshot snapshot_kelly %}
-    {# This has no target_database set, which is allowed! #}
-    {{
-        config(
-            target_schema=schema,
-            unique_key='id || ' ~ "'-'" ~ ' || first_name',
-            strategy='timestamp',
-            updated_at='updated_at',
-        )
-    }}
-    select * from {{target.database}}.{{schema}}.seed where last_name = 'Kelly'
-
-{% endsnapshot %}
diff --git a/tests/integration/simple_snapshot_test/test_simple_snapshot.py b/tests/integration/simple_snapshot_test/test_simple_snapshot.py
deleted file mode 100644
index 191dde72c..000000000
--- a/tests/integration/simple_snapshot_test/test_simple_snapshot.py
+++ /dev/null
@@ -1,564 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-from datetime import datetime
-import pytz
-
-
-class BaseSimpleSnapshotTest(DBTIntegrationTest):
-    NUM_SNAPSHOT_MODELS = 1
-
-    @property
-    def schema(self):
-        return "simple_snapshot"
-
-    @property
-    def models(self):
-        return "models"
-
-    def run_snapshot(self):
-        return self.run_dbt(['snapshot'])
-
-    def dbt_run_seed_snapshot(self):
-        self.run_sql_file('seed.sql')
-
-        results = self.run_snapshot()
-        self.assertEqual(len(results),  self.NUM_SNAPSHOT_MODELS)
-
-    def assert_case_tables_equal(self, actual, expected):
-        # this does something different on snowflake, but here it's just assertTablesEqual
-        self.assertTablesEqual(actual, expected)
-
-    def assert_expected(self):
-        self.run_dbt(['test'])
-        self.assert_case_tables_equal('snapshot_actual', 'snapshot_expected')
-
-
-class TestSimpleSnapshotFiles(BaseSimpleSnapshotTest):
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds'],
-            "snapshot-paths": ['test-snapshots-pg'],
-            'macro-paths': ['macros'],
-        }
-
-
-class TestSimpleColumnSnapshotFiles(DBTIntegrationTest):
-
-    @property
-    def schema(self):
-        return "simple_snapshot"
-
-    @property
-    def models(self):
-        return "models-checkall"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds'],
-            'macro-paths': ['custom-snapshot-macros', 'macros'],
-            'snapshot-paths': ['test-snapshots-checkall'],
-            'seeds': {
-                'quote_columns': False,
-            }
-        }
-
-    def _run_snapshot_test(self):
-        self.run_dbt(['seed'])
-        self.run_dbt(['snapshot'])
-        database = self.adapter.quote(self.default_database)
-        results = self.run_sql(
-            'select * from {}.{}.my_snapshot'.format(database, self.unique_schema()),
-            fetch='all'
-        )
-        self.assertEqual(len(results), 3)
-        for result in results:
-            self.assertEqual(len(result), 6)
-
-        self.run_dbt(['snapshot', '--vars', '{seed_name: seed_newcol}'])
-        results = self.run_sql(
-            'select * from {}.{}.my_snapshot where last_name is not NULL'.format(database, self.unique_schema()),
-            fetch='all'
-        )
-        self.assertEqual(len(results), 3)
-
-        for result in results:
-            # new column
-            self.assertEqual(len(result), 7)
-            self.assertIsNotNone(result[-1])
-
-        results = self.run_sql(
-            'select * from {}.{}.my_snapshot where last_name is NULL'.format(database, self.unique_schema()),
-            fetch='all'
-        )
-        self.assertEqual(len(results), 3)
-        for result in results:
-            # new column
-            self.assertEqual(len(result), 7)
-
-    @use_profile('bigquery')
-    def test_bigquery_renamed_source(self):
-        self._run_snapshot_test()
-
-
-class TestSimpleSnapshotFileSelects(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "simple_snapshot"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds'],
-            "snapshot-paths": ['test-snapshots-select',
-                               'test-snapshots-pg'],
-            'macro-paths': ['macros'],
-        }
-
-
-class TestConfiguredSnapshotFileSelects(TestSimpleSnapshotFileSelects):
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds'],
-            "snapshot-paths": ['test-snapshots-select-noconfig'],
-            "snapshots": {
-                "test": {
-                    "target_schema": self.unique_schema(),
-                    "unique_key": "id || '-' || first_name",
-                    'strategy': 'timestamp',
-                    'updated_at': 'updated_at',
-                },
-            },
-            'macro-paths': ['macros'],
-        }
-
-
-class TestSimpleSnapshotFilesBigquery(DBTIntegrationTest):
-    @property
-    def schema(self):
-        return "simple_snapshot"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            "snapshot-paths": ['test-snapshots-bq'],
-            'macro-paths': ['macros'],
-        }
-
-    def assert_expected(self):
-        self.run_dbt(['test'])
-        self.assertTablesEqual('snapshot_actual', 'snapshot_expected')
-
-    @use_profile('bigquery')
-    def test__bigquery__simple_snapshot(self):
-        self.run_sql_file("seed_bq.sql")
-
-        self.run_dbt(["snapshot"])
-
-        self.assert_expected()
-
-        self.run_sql_file("invalidate_bigquery.sql")
-        self.run_sql_file("update_bq.sql")
-
-        self.run_dbt(["snapshot"])
-
-        self.assert_expected()
-
-    @use_profile('bigquery')
-    def test__bigquery__snapshot_with_new_field(self):
-
-        self.run_sql_file("seed_bq.sql")
-
-        self.run_dbt(["snapshot"])
-
-        self.assertTablesEqual("snapshot_expected", "snapshot_actual")
-
-        self.run_sql_file("invalidate_bigquery.sql")
-        self.run_sql_file("update_bq.sql")
-
-        # This adds new fields to the source table, and updates the expected snapshot output accordingly
-        self.run_sql_file("add_column_to_source_bq.sql")
-
-        self.run_dbt(["snapshot"])
-
-        # A more thorough test would assert that snapshotted == expected, but BigQuery does not support the
-        # "EXCEPT DISTINCT" operator on nested fields! Instead, just check that schemas are congruent.
-
-        expected_cols = self.get_table_columns(
-            database=self.default_database,
-            schema=self.unique_schema(),
-            table='snapshot_expected'
-        )
-        snapshotted_cols = self.get_table_columns(
-            database=self.default_database,
-            schema=self.unique_schema(),
-            table='snapshot_actual'
-        )
-
-        self.assertTrue(len(expected_cols) > 0, "source table does not exist -- bad test")
-        self.assertEqual(len(expected_cols), len(snapshotted_cols), "actual and expected column lengths are different")
-
-        for (expected_col, actual_col) in zip(expected_cols, snapshotted_cols):
-            expected_name, expected_type, _ = expected_col
-            actual_name, actual_type, _ = actual_col
-            self.assertTrue(expected_name is not None)
-            self.assertTrue(expected_type is not None)
-
-            self.assertEqual(expected_name, actual_name, "names are different")
-            self.assertEqual(expected_type, actual_type, "data types are different")
-
-
-class TestCrossDBSnapshotFiles(DBTIntegrationTest):
-    setup_alternate_db = True
-
-    @property
-    def schema(self):
-        return "simple_snapshot"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        paths = ['test-snapshots-bq']
-        return {
-            'config-version': 2,
-            'snapshot-paths': paths,
-            'macro-paths': ['macros'],
-        }
-
-    def run_snapshot(self):
-        return self.run_dbt(['snapshot', '--vars', '{{"target_database": {}}}'.format(self.alternative_database)])
-
-    @use_profile('bigquery')
-    def test__bigquery__cross_snapshot(self):
-        self.run_sql_file("seed_bq.sql")
-
-        self.run_snapshot()
-
-        self.assertTablesEqual("snapshot_expected", "snapshot_actual", table_b_db=self.alternative_database)
-
-        self.run_sql_file("invalidate_bigquery.sql")
-        self.run_sql_file("update_bq.sql")
-
-        self.run_snapshot()
-
-        self.assertTablesEqual("snapshot_expected", "snapshot_actual", table_b_db=self.alternative_database)
-
-
-class TestCheckCols(TestSimpleSnapshotFiles):
-    NUM_SNAPSHOT_MODELS = 2
-
-    def _assertTablesEqualSql(self, relation_a, relation_b, columns=None):
-        # When building the equality tests, only test columns that don't start
-        # with 'dbt_', because those are time-sensitive
-        if columns is None:
-            columns = [c for c in self.get_relation_columns(relation_a) if not c[0].lower().startswith('dbt_')]
-        return super()._assertTablesEqualSql(relation_a, relation_b, columns=columns)
-
-    def assert_expected(self):
-        super().assert_expected()
-        self.assert_case_tables_equal('snapshot_checkall', 'snapshot_expected')
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds'],
-            "snapshot-paths": ['test-check-col-snapshots'],
-            'macro-paths': ['macros'],
-        }
-
-
-class TestConfiguredCheckCols(TestCheckCols):
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds'],
-            "snapshot-paths": ['test-check-col-snapshots-noconfig'],
-            "snapshots": {
-                "test": {
-                    "target_schema": self.unique_schema(),
-                    "unique_key": "id || '-' || first_name",
-                    "strategy": "check",
-                    "check_cols": ["email"],
-                },
-            },
-            'macro-paths': ['macros'],
-        }
-
-
-class TestUpdatedAtCheckCols(TestCheckCols):
-     def _assertTablesEqualSql(self, relation_a, relation_b, columns=None):
-         revived_records = self.run_sql(
-             '''
-             select
-                 id,
-                 updated_at,
-                 dbt_valid_from
-             from {}
-             '''.format(relation_b),
-             fetch='all'
-         )
-
-         for result in revived_records:
-             # result is a tuple, the updated_at is second and dbt_valid_from is latest
-             self.assertIsInstance(result[1], datetime)
-             self.assertIsInstance(result[2], datetime)
-             self.assertEqual(result[1].replace(tzinfo=pytz.UTC), result[2].replace(tzinfo=pytz.UTC))
-
-         if columns is None:
-             columns = [c for c in self.get_relation_columns(relation_a) if not c[0].lower().startswith('dbt_')]
-         return super()._assertTablesEqualSql(relation_a, relation_b, columns=columns)
-
-     def assert_expected(self):
-         super().assert_expected()
-         self.assertTablesEqual('snapshot_checkall', 'snapshot_expected')
-
-
-     @property
-     def project_config(self):
-         return {
-             'config-version': 2,
-            'seed-paths': ['seeds'],
-             "snapshot-paths": ['test-check-col-snapshots-noconfig'],
-             "snapshots": {
-                 "test": {
-                     "target_schema": self.unique_schema(),
-                     "unique_key": "id || '-' || first_name",
-                     "strategy": "check",
-                     "check_cols" : "all",
-                     "updated_at": "updated_at",
-                 },
-             },
-             'macro-paths': ['macros'],
-         }
-
-
-class TestCheckColsBigquery(TestSimpleSnapshotFilesBigquery):
-    def _assertTablesEqualSql(self, relation_a, relation_b, columns=None):
-        # When building the equality tests, only test columns that don't start
-        # with 'dbt_', because those are time-sensitive
-        if columns is None:
-            columns = [c for c in self.get_relation_columns(relation_a) if not c[0].lower().startswith('dbt_')]
-        return super()._assertTablesEqualSql(relation_a, relation_b, columns=columns)
-
-    def assert_expected(self):
-        super().assert_expected()
-        self.assertTablesEqual('snapshot_checkall', 'snapshot_expected')
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds'],
-            "snapshot-paths": ['test-check-col-snapshots-bq'],
-            'macro-paths': ['macros'],
-        }
-
-    @use_profile('bigquery')
-    def test__bigquery__snapshot_with_new_field(self):
-        self.use_default_project()
-        self.use_profile('bigquery')
-
-        self.run_sql_file("seed_bq.sql")
-
-        self.run_dbt(["snapshot"])
-
-        self.assertTablesEqual("snapshot_expected", "snapshot_actual")
-        self.assertTablesEqual("snapshot_expected", "snapshot_checkall")
-
-        self.run_sql_file("invalidate_bigquery.sql")
-        self.run_sql_file("update_bq.sql")
-
-        # This adds new fields to the source table, and updates the expected snapshot output accordingly
-        self.run_sql_file("add_column_to_source_bq.sql")
-
-        # check_cols='all' will replace the changed field
-        self.run_dbt(['snapshot'])
-
-        # A more thorough test would assert that snapshotted == expected, but BigQuery does not support the
-        # "EXCEPT DISTINCT" operator on nested fields! Instead, just check that schemas are congruent.
-
-        expected_cols = self.get_table_columns(
-            database=self.default_database,
-            schema=self.unique_schema(),
-            table='snapshot_expected'
-        )
-        snapshotted_cols = self.get_table_columns(
-            database=self.default_database,
-            schema=self.unique_schema(),
-            table='snapshot_actual'
-        )
-        snapshotted_all_cols = self.get_table_columns(
-            database=self.default_database,
-            schema=self.unique_schema(),
-            table='snapshot_checkall'
-        )
-
-        self.assertTrue(len(expected_cols) > 0, "source table does not exist -- bad test")
-        self.assertEqual(len(expected_cols), len(snapshotted_cols), "actual and expected column lengths are different")
-        self.assertEqual(len(expected_cols), len(snapshotted_all_cols))
-
-        for (expected_col, actual_col) in zip(expected_cols, snapshotted_cols):
-            expected_name, expected_type, _ = expected_col
-            actual_name, actual_type, _ = actual_col
-            self.assertTrue(expected_name is not None)
-            self.assertTrue(expected_type is not None)
-
-            self.assertEqual(expected_name, actual_name, "names are different")
-            self.assertEqual(expected_type, actual_type, "data types are different")
-
-
-class TestSnapshotHardDelete(DBTIntegrationTest):
-    # These tests uses the same seed data, containing 20 records of which we hard delete the last 10.
-    # These deleted records set the dbt_valid_to to time the snapshot was ran.
-    NUM_SNAPSHOT_MODELS = 1
-
-    @property
-    def schema(self):
-        return "simple_snapshot"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        paths = ['test-snapshots-bq']
-
-        return {
-            'config-version': 2,
-            'seed-paths': ['seeds'],
-            "snapshot-paths": paths,
-            'macro-paths': ['macros'],
-        }
-
-    @use_profile('bigquery')
-    def test__bigquery__snapshot_hard_delete(self):
-        self.run_sql_file('seed_bq.sql')
-        self._test_snapshot_hard_delete()
-
-    def _test_snapshot_hard_delete(self):
-        self._snapshot()
-
-        self.assertTablesEqual("snapshot_expected", "snapshot_actual")
-
-        self._invalidated_snapshot_datetime = None
-        self._revived_snapshot_datetime = None
-
-        self._delete_records()
-        self._snapshot_and_assert_invalidated()
-        self._revive_records()
-        self._snapshot_and_assert_revived()
-
-    def _snapshot(self):
-        begin_snapshot_datetime = datetime.now(pytz.UTC)
-        results = self.run_dbt(['snapshot', '--vars', '{invalidate_hard_deletes: true}'])
-        self.assertEqual(len(results), self.NUM_SNAPSHOT_MODELS)
-
-        return begin_snapshot_datetime
-
-    def _delete_records(self):
-        database = self.adapter.quote(self.default_database)
-
-        self.run_sql(
-            'delete from {}.{}.seed where id >= 10;'.format(database, self.unique_schema())
-        )
-
-    def _snapshot_and_assert_invalidated(self):
-        self._invalidated_snapshot_datetime = self._snapshot()
-
-        database = self.adapter.quote(self.default_database)
-
-        snapshotted = self.run_sql(
-            '''
-            select
-                id,
-                dbt_valid_to
-            from {}.{}.snapshot_actual
-            order by id
-            '''.format(database, self.unique_schema()),
-            fetch='all'
-        )
-
-        self.assertEqual(len(snapshotted), 20)
-        for result in snapshotted[10:]:
-            # result is a tuple, the dbt_valid_to column is the latest
-            self.assertIsInstance(result[-1], datetime)
-            self.assertGreaterEqual(result[-1].replace(tzinfo=pytz.UTC), self._invalidated_snapshot_datetime)
-
-    def _revive_records(self):
-        database = self.adapter.quote(self.default_database)
-
-        revival_timestamp = datetime.now(pytz.UTC).strftime(r'%Y-%m-%d %H:%M:%S')
-        self.run_sql(
-            '''
-            insert into {}.{}.seed (id, first_name, last_name, email, gender, ip_address, updated_at) values
-            (10, 'Rachel', 'Lopez', 'rlopez9@themeforest.net', 'Female', '237.165.82.71', '{}'),
-            (11, 'Donna', 'Welch', 'dwelcha@shutterfly.com', 'Female', '103.33.110.138', '{}')
-            '''.format(database, self.unique_schema(), revival_timestamp, revival_timestamp)
-        )
-
-    def _snapshot_and_assert_revived(self):
-        self._revived_snapshot_datetime = self._snapshot()
-
-        database = self.adapter.quote(self.default_database)
-
-        # records which weren't revived (id != 10, 11)
-        invalidated_records = self.run_sql(
-            '''
-            select
-                id,
-                dbt_valid_to
-            from {}.{}.snapshot_actual
-            where dbt_valid_to is not null
-            order by id
-            '''.format(database, self.unique_schema()),
-            fetch='all'
-        )
-
-        self.assertEqual(len(invalidated_records), 11)
-        for result in invalidated_records:
-            # result is a tuple, the dbt_valid_to column is the latest
-            self.assertIsInstance(result[1], datetime)
-            self.assertGreaterEqual(result[1].replace(tzinfo=pytz.UTC), self._invalidated_snapshot_datetime)
-
-        # records which weren't revived (id != 10, 11)
-        revived_records = self.run_sql(
-            '''
-            select
-                id,
-                dbt_valid_from,
-                dbt_valid_to
-            from {}.{}.snapshot_actual
-            where dbt_valid_to is null
-            and id IN (10, 11)
-            '''.format(database, self.unique_schema()),
-            fetch='all'
-        )
-
-        self.assertEqual(len(revived_records), 2)
-        for result in revived_records:
-            # result is a tuple, the dbt_valid_from is second and dbt_valid_to is latest
-            self.assertIsInstance(result[1], datetime)
-            # there are milliseconds (part of microseconds in datetime objects) in the
-            # invalidated_snapshot_datetime and not in result datetime so set the microseconds to 0
-            self.assertGreaterEqual(result[1].replace(tzinfo=pytz.UTC), self._invalidated_snapshot_datetime.replace(microsecond=0))
-            self.assertIsNone(result[2])
diff --git a/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py b/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py
deleted file mode 100644
index ad027e0d8..000000000
--- a/tests/integration/simple_snapshot_test/test_snapshot_check_cols.py
+++ /dev/null
@@ -1,40 +0,0 @@
-from tests.integration.base import DBTIntegrationTest, use_profile
-
-
-class TestSimpleSnapshotFiles(DBTIntegrationTest):
-    NUM_SNAPSHOT_MODELS = 1
-
-    @property
-    def schema(self):
-        return "simple_snapshot"
-
-    @property
-    def models(self):
-        return "models"
-
-    @property
-    def project_config(self):
-        return {
-            'config-version': 2,
-            "snapshot-paths": ['check-snapshots'],
-            "test-paths": ['check-snapshots-expected'],
-            "model-paths": [],
-        }
-
-    def test_snapshot_check_cols_cycle(self):
-        results = self.run_dbt(["snapshot", '--vars', 'version: 1'])
-        self.assertEqual(len(results), 1)
-
-        results = self.run_dbt(["snapshot", '--vars', 'version: 2'])
-        self.assertEqual(len(results), 1)
-
-        results = self.run_dbt(["snapshot", '--vars', 'version: 3'])
-        self.assertEqual(len(results), 1)
-
-    def assert_expected(self):
-        self.run_dbt(['test', '--select', 'test_type:singular', '--vars', 'version: 3'])
-
-    @use_profile('bigquery')
-    def test__bigquery__simple_snapshot(self):
-        self.test_snapshot_check_cols_cycle()
-        self.assert_expected()
diff --git a/tests/integration/simple_snapshot_test/update_bq.sql b/tests/integration/simple_snapshot_test/update_bq.sql
deleted file mode 100644
index 5c972d8af..000000000
--- a/tests/integration/simple_snapshot_test/update_bq.sql
+++ /dev/null
@@ -1,78 +0,0 @@
--- insert v2 of the 11 - 21 records
-
-insert {database}.{schema}.snapshot_expected (
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    dbt_valid_from,
-    dbt_valid_to,
-    dbt_updated_at,
-    dbt_scd_id
-)
-
-select
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    -- fields added by snapshotting
-    updated_at as dbt_valid_from,
-    cast(null as timestamp) as dbt_valid_to,
-    updated_at as dbt_updated_at,
-    to_hex(md5(concat(cast(id as string), '-', first_name, '|', cast(updated_at as string)))) as dbt_scd_id
-from {database}.{schema}.seed
-where id >= 10 and id <= 20;
-
-
--- insert 10 new records
-insert into {database}.{schema}.seed (id, first_name, last_name, email, gender, ip_address, updated_at) values
-(21, 'Judy', 'Robinson', 'jrobinsonk@blogs.com', 'Female', '208.21.192.232', '2016-09-18 08:27:38'),
-(22, 'Kevin', 'Alvarez', 'kalvarezl@buzzfeed.com', 'Male', '228.106.146.9', '2016-07-29 03:07:37'),
-(23, 'Barbara', 'Carr', 'bcarrm@pen.io', 'Female', '106.165.140.17', '2015-09-24 13:27:23'),
-(24, 'William', 'Watkins', 'wwatkinsn@guardian.co.uk', 'Male', '78.155.84.6', '2016-03-08 19:13:08'),
-(25, 'Judy', 'Cooper', 'jcoopero@google.com.au', 'Female', '24.149.123.184', '2016-10-05 20:49:33'),
-(26, 'Shirley', 'Castillo', 'scastillop@samsung.com', 'Female', '129.252.181.12', '2016-06-20 21:12:21'),
-(27, 'Justin', 'Harper', 'jharperq@opera.com', 'Male', '131.172.103.218', '2016-05-21 22:56:46'),
-(28, 'Marie', 'Medina', 'mmedinar@nhs.uk', 'Female', '188.119.125.67', '2015-10-08 13:44:33'),
-(29, 'Kelly', 'Edwards', 'kedwardss@phoca.cz', 'Female', '47.121.157.66', '2015-09-15 06:33:37'),
-(30, 'Carl', 'Coleman', 'ccolemant@wikipedia.org', 'Male', '82.227.154.83', '2016-05-26 16:46:40');
-
-
--- add these new records to the snapshot table
-insert {database}.{schema}.snapshot_expected (
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    dbt_valid_from,
-    dbt_valid_to,
-    dbt_updated_at,
-    dbt_scd_id
-)
-
-select
-    id,
-    first_name,
-    last_name,
-    email,
-    gender,
-    ip_address,
-    updated_at,
-    -- fields added by snapshotting
-    updated_at as dbt_valid_from,
-    cast(null as timestamp) as dbt_valid_to,
-    updated_at as dbt_updated_at,
-    to_hex(md5(concat(cast(id as string), '-', first_name, '|', cast(updated_at as string)))) as dbt_scd_id
-from {database}.{schema}.seed
-where id > 20;
-

From bf54414bd6cf74f73128e7a802298492102a1466 Mon Sep 17 00:00:00 2001
From: FishtownBuildBot <77737458+FishtownBuildBot@users.noreply.github.com>
Date: Mon, 17 Apr 2023 15:56:38 -0500
Subject: [PATCH 607/860] Cleanup main after cutting new 1.5.latest branch
 (#666)

* Bumping version to 1.5.0rc1

* Clean up changelog on main

* linter stuff

---------

Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .bumpversion.cfg                              |  2 +-
 .changes/1.5.0-b1.md                          | 21 ------
 .changes/1.5.0-b2.md                          |  5 --
 .changes/1.5.0-b3.md                          | 20 ------
 .changes/1.5.0-b4.md                          | 18 -----
 .changes/1.5.0/Features-20220826-115320.yaml  |  7 --
 .changes/1.5.0/Features-20221220-193731.yaml  |  6 --
 .changes/1.5.0/Features-20230223-145508.yaml  |  6 --
 .changes/1.5.0/Features-20230228-094234.yaml  |  6 --
 .changes/1.5.0/Features-20230303-132509.yaml  |  6 --
 .changes/1.5.0/Features-20230314-171221.yaml  |  6 --
 .changes/1.5.0/Features-20230315-120554.yaml  |  6 --
 .changes/1.5.0/Fixes-20221213-102005.yaml     |  7 --
 .changes/1.5.0/Fixes-20230125-174159.yaml     |  7 --
 .changes/1.5.0/Fixes-20230202-010332.yaml     |  8 ---
 .changes/1.5.0/Fixes-20230213-203317.yaml     |  6 --
 .changes/1.5.0/Fixes-20230216-140756.yaml     |  6 --
 .changes/1.5.0/Fixes-20230227-110426.yaml     |  6 --
 .changes/1.5.0/Fixes-20230309-181313.yaml     |  6 --
 .changes/1.5.0/Fixes-20230315-130504.yaml     |  6 --
 .changes/1.5.0/Fixes-20230322-162200.yaml     |  6 --
 .../1.5.0/Under the Hood-20221209-161550.yaml |  7 --
 .../1.5.0/Under the Hood-20230130-170118.yaml |  7 --
 .../1.5.0/Under the Hood-20230223-105149.yaml |  6 --
 .../unreleased/Features-20230406-104433.yaml  |  6 --
 .../unreleased/Fixes-20230222-133301.yaml     |  6 --
 .../unreleased/Fixes-20230408-035117.yaml     |  7 --
 .../unreleased/Fixes-20230411-143515.yaml     |  6 --
 .../Under the Hood-20230411-143129.yaml       |  6 --
 CHANGELOG.md                                  | 69 -------------------
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 32 files changed, 3 insertions(+), 288 deletions(-)
 delete mode 100644 .changes/1.5.0-b1.md
 delete mode 100644 .changes/1.5.0-b2.md
 delete mode 100644 .changes/1.5.0-b3.md
 delete mode 100644 .changes/1.5.0-b4.md
 delete mode 100644 .changes/1.5.0/Features-20220826-115320.yaml
 delete mode 100644 .changes/1.5.0/Features-20221220-193731.yaml
 delete mode 100644 .changes/1.5.0/Features-20230223-145508.yaml
 delete mode 100644 .changes/1.5.0/Features-20230228-094234.yaml
 delete mode 100644 .changes/1.5.0/Features-20230303-132509.yaml
 delete mode 100644 .changes/1.5.0/Features-20230314-171221.yaml
 delete mode 100644 .changes/1.5.0/Features-20230315-120554.yaml
 delete mode 100644 .changes/1.5.0/Fixes-20221213-102005.yaml
 delete mode 100644 .changes/1.5.0/Fixes-20230125-174159.yaml
 delete mode 100644 .changes/1.5.0/Fixes-20230202-010332.yaml
 delete mode 100644 .changes/1.5.0/Fixes-20230213-203317.yaml
 delete mode 100644 .changes/1.5.0/Fixes-20230216-140756.yaml
 delete mode 100644 .changes/1.5.0/Fixes-20230227-110426.yaml
 delete mode 100644 .changes/1.5.0/Fixes-20230309-181313.yaml
 delete mode 100644 .changes/1.5.0/Fixes-20230315-130504.yaml
 delete mode 100644 .changes/1.5.0/Fixes-20230322-162200.yaml
 delete mode 100644 .changes/1.5.0/Under the Hood-20221209-161550.yaml
 delete mode 100644 .changes/1.5.0/Under the Hood-20230130-170118.yaml
 delete mode 100644 .changes/1.5.0/Under the Hood-20230223-105149.yaml
 delete mode 100644 .changes/unreleased/Features-20230406-104433.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230222-133301.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230408-035117.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230411-143515.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20230411-143129.yaml

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index dc5b81465..5d390e604 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.5.0b4
+current_version = 1.5.0rc1
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.5.0-b1.md b/.changes/1.5.0-b1.md
deleted file mode 100644
index 4afeff282..000000000
--- a/.changes/1.5.0-b1.md
+++ /dev/null
@@ -1,21 +0,0 @@
-## dbt-bigquery 1.5.0-b1 - February 22, 2023
-
-### Features
-
-- add option to specify excluded columns on merge ([#5260](https://github.com/dbt-labs/dbt-bigquery/issues/5260))
-- dbt-constraints support for BigQuery as per dbt-core issue #1358 ([#444](https://github.com/dbt-labs/dbt-bigquery/issues/444))
-
-### Fixes
-
-- Make BQ access_grant updates thread safe ([#266](https://github.com/dbt-labs/dbt-bigquery/issues/266))
-- Use IEC standard abbreviations (GiB, TiB, etc) ([#477](https://github.com/dbt-labs/dbt-bigquery/issues/477))
-- Pin dataproc serverless spark runtime to `1.1` ([#531](https://github.com/dbt-labs/dbt-bigquery/issues/531))
-
-### Under the Hood
-
-- Remove manual retry to get python job status on cluster ([#422](https://github.com/dbt-labs/dbt-bigquery/issues/422))
-- remove tox call to integration tests ([#459](https://github.com/dbt-labs/dbt-bigquery/issues/459))
-
-### Contributors
-- [@dave-connors-3](https://github.com/dave-connors-3) ([#5260](https://github.com/dbt-labs/dbt-bigquery/issues/5260))
-- [@victoriapm](https://github.com/victoriapm) ([#444](https://github.com/dbt-labs/dbt-bigquery/issues/444))
diff --git a/.changes/1.5.0-b2.md b/.changes/1.5.0-b2.md
deleted file mode 100644
index a214d458c..000000000
--- a/.changes/1.5.0-b2.md
+++ /dev/null
@@ -1,5 +0,0 @@
-## dbt-bigquery 1.5.0-b2 - March 02, 2023
-
-### Under the Hood
-
-- Rename constraints_enabled to contract ([#548](https://github.com/dbt-labs/dbt-bigquery/issues/548))
diff --git a/.changes/1.5.0-b3.md b/.changes/1.5.0-b3.md
deleted file mode 100644
index f2415ad97..000000000
--- a/.changes/1.5.0-b3.md
+++ /dev/null
@@ -1,20 +0,0 @@
-## dbt-bigquery 1.5.0-b3 - March 16, 2023
-
-### Features
-
-- Enforce contracts on models materialized as tables and views ([#529](https://github.com/dbt-labs/dbt-bigquery/issues/529), [#555](https://github.com/dbt-labs/dbt-bigquery/issues/555))
-- add dataproc serverless config to profile ([#530](https://github.com/dbt-labs/dbt-bigquery/issues/530))
-
-### Fixes
-
-- Fix time ingestion partitioning option regression when combined using `require_partition_filter` option on incremental run ([#483](https://github.com/dbt-labs/dbt-bigquery/issues/483))
-- Allow policy_tags to be removed ([#349](https://github.com/dbt-labs/dbt-bigquery/issues/349))
-- Update connections.py with retry logic for 502 BadGateway ([#562](https://github.com/dbt-labs/dbt-bigquery/issues/562))
-- Fix for Python incremental model regression ([#581](https://github.com/dbt-labs/dbt-bigquery/issues/581))
-- Fix failing test by removing no erroneous asserts. ([#605](https://github.com/dbt-labs/dbt-bigquery/issues/605))
-
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#483](https://github.com/dbt-labs/dbt-bigquery/issues/483))
-- [@garsir](https://github.com/garsir) ([#349](https://github.com/dbt-labs/dbt-bigquery/issues/349))
-- [@patkearns10](https://github.com/patkearns10) ([#562](https://github.com/dbt-labs/dbt-bigquery/issues/562))
-- [@torkjel](https://github.com/torkjel) ([#530](https://github.com/dbt-labs/dbt-bigquery/issues/530))
diff --git a/.changes/1.5.0-b4.md b/.changes/1.5.0-b4.md
deleted file mode 100644
index 2262d374d..000000000
--- a/.changes/1.5.0-b4.md
+++ /dev/null
@@ -1,18 +0,0 @@
-## dbt-bigquery 1.5.0-b4 - March 30, 2023
-
-### Features
-
-- Adding `bytes_billed` to `BigQueryAdapterResponse`  ([#560](https://github.com/dbt-labs/dbt-bigquery/issues/560))
-- Modify addapter to support unified constraint fields ([#567](https://github.com/dbt-labs/dbt-bigquery/issues/567))
-- Modify create_table_as to use contract column order ([#579](https://github.com/dbt-labs/dbt-bigquery/issues/579))
-
-### Fixes
-
-- Use _make_ref_key_dict instead of _make_ref_key_msg ([#621](https://github.com/dbt-labs/dbt-bigquery/issues/621))
-
-### Under the Hood
-
-- Treat contract config as a python object ([#548](https://github.com/dbt-labs/dbt-bigquery/issues/548), [#633](https://github.com/dbt-labs/dbt-bigquery/issues/633))
-
-### Contributors
-- [@bruno-szdl](https://github.com/bruno-szdl) ([#560](https://github.com/dbt-labs/dbt-bigquery/issues/560))
diff --git a/.changes/1.5.0/Features-20220826-115320.yaml b/.changes/1.5.0/Features-20220826-115320.yaml
deleted file mode 100644
index 27bca84fc..000000000
--- a/.changes/1.5.0/Features-20220826-115320.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: add option to specify excluded columns on merge
-time: 2022-08-26T11:53:20.194981-05:00
-custom:
-  Author: dave-connors-3
-  Issue: "5260"
-  PR: "227"
diff --git a/.changes/1.5.0/Features-20221220-193731.yaml b/.changes/1.5.0/Features-20221220-193731.yaml
deleted file mode 100644
index c9aa5c03d..000000000
--- a/.changes/1.5.0/Features-20221220-193731.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: 'Support for data type constraints in BigQuery'
-time: 2022-12-20T19:37:31.982821+01:00
-custom:
-  Author: victoriapm, emmyoop
-  Issue: 444 568
diff --git a/.changes/1.5.0/Features-20230223-145508.yaml b/.changes/1.5.0/Features-20230223-145508.yaml
deleted file mode 100644
index cab391802..000000000
--- a/.changes/1.5.0/Features-20230223-145508.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Enforce contracts on models materialized as tables and views
-time: 2023-02-23T14:55:08.186645-05:00
-custom:
-  Author: michelleark emmyoop
-  Issue: 529 555
diff --git a/.changes/1.5.0/Features-20230228-094234.yaml b/.changes/1.5.0/Features-20230228-094234.yaml
deleted file mode 100644
index 4929c0834..000000000
--- a/.changes/1.5.0/Features-20230228-094234.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: 'Adding `bytes_billed` to `BigQueryAdapterResponse` '
-time: 2023-02-28T09:42:34.557696-03:00
-custom:
-  Author: bruno-szdl
-  Issue: "560"
diff --git a/.changes/1.5.0/Features-20230303-132509.yaml b/.changes/1.5.0/Features-20230303-132509.yaml
deleted file mode 100644
index 3a0ba8403..000000000
--- a/.changes/1.5.0/Features-20230303-132509.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: add dataproc serverless config to profile
-time: 2023-03-03T13:25:09.02695-08:00
-custom:
-  Author: colin-rogers-dbt torkjel
-  Issue: "530"
diff --git a/.changes/1.5.0/Features-20230314-171221.yaml b/.changes/1.5.0/Features-20230314-171221.yaml
deleted file mode 100644
index 067812f09..000000000
--- a/.changes/1.5.0/Features-20230314-171221.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Modify addapter to support unified constraint fields
-time: 2023-03-14T17:12:21.287702-04:00
-custom:
-  Author: peterallenwebb
-  Issue: "567"
diff --git a/.changes/1.5.0/Features-20230315-120554.yaml b/.changes/1.5.0/Features-20230315-120554.yaml
deleted file mode 100644
index 65718d06b..000000000
--- a/.changes/1.5.0/Features-20230315-120554.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Modify create_table_as to use contract column order
-time: 2023-03-15T12:05:54.52431-04:00
-custom:
-  Author: gshank
-  Issue: "579"
diff --git a/.changes/1.5.0/Fixes-20221213-102005.yaml b/.changes/1.5.0/Fixes-20221213-102005.yaml
deleted file mode 100644
index 5e4c486d8..000000000
--- a/.changes/1.5.0/Fixes-20221213-102005.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: Make BQ access_grant updates thread safe
-time: 2022-12-13T10:20:05.714134-08:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "266"
-  PR: "404"
diff --git a/.changes/1.5.0/Fixes-20230125-174159.yaml b/.changes/1.5.0/Fixes-20230125-174159.yaml
deleted file mode 100644
index 4100080d4..000000000
--- a/.changes/1.5.0/Fixes-20230125-174159.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: Use IEC standard abbreviations (GiB, TiB, etc)
-time: 2023-01-25T17:41:59.921173-07:00
-custom:
-  Author: dbeatty10
-  Issue: "477"
-  PR: "477"
diff --git a/.changes/1.5.0/Fixes-20230202-010332.yaml b/.changes/1.5.0/Fixes-20230202-010332.yaml
deleted file mode 100644
index f6062e7ea..000000000
--- a/.changes/1.5.0/Fixes-20230202-010332.yaml
+++ /dev/null
@@ -1,8 +0,0 @@
-kind: Fixes
-body: Fix time ingestion partitioning option regression when combined using `require_partition_filter`
-  option on incremental run
-time: 2023-02-02T01:03:32.577336+01:00
-custom:
-  Author: Kayrnt
-  Issue: "483"
-  PR: "485"
diff --git a/.changes/1.5.0/Fixes-20230213-203317.yaml b/.changes/1.5.0/Fixes-20230213-203317.yaml
deleted file mode 100644
index ae64c4e2d..000000000
--- a/.changes/1.5.0/Fixes-20230213-203317.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Pin dataproc serverless spark runtime to `1.1`
-time: 2023-02-13T20:33:17.839861-08:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "531"
diff --git a/.changes/1.5.0/Fixes-20230216-140756.yaml b/.changes/1.5.0/Fixes-20230216-140756.yaml
deleted file mode 100644
index b273c563e..000000000
--- a/.changes/1.5.0/Fixes-20230216-140756.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Allow policy_tags to be removed
-time: 2023-02-16T14:07:56.313767Z
-custom:
-  Author: garsir
-  Issue: "349"
diff --git a/.changes/1.5.0/Fixes-20230227-110426.yaml b/.changes/1.5.0/Fixes-20230227-110426.yaml
deleted file mode 100644
index c23c53ee6..000000000
--- a/.changes/1.5.0/Fixes-20230227-110426.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Update connections.py with retry logic for 502 BadGateway
-time: 2023-02-27T11:04:26.978726+11:00
-custom:
-  Author: patkearns10
-  Issue: "562"
diff --git a/.changes/1.5.0/Fixes-20230309-181313.yaml b/.changes/1.5.0/Fixes-20230309-181313.yaml
deleted file mode 100644
index 8681f5eaf..000000000
--- a/.changes/1.5.0/Fixes-20230309-181313.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Fix for Python incremental model regression
-time: 2023-03-09T18:13:13.512904-08:00
-custom:
-  Author: nssalian
-  Issue: "581"
diff --git a/.changes/1.5.0/Fixes-20230315-130504.yaml b/.changes/1.5.0/Fixes-20230315-130504.yaml
deleted file mode 100644
index 5229dca17..000000000
--- a/.changes/1.5.0/Fixes-20230315-130504.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Fix failing test by removing no erroneous asserts.
-time: 2023-03-15T13:05:04.747323-07:00
-custom:
-  Author: versusfacit
-  Issue: "605"
diff --git a/.changes/1.5.0/Fixes-20230322-162200.yaml b/.changes/1.5.0/Fixes-20230322-162200.yaml
deleted file mode 100644
index a80a3f963..000000000
--- a/.changes/1.5.0/Fixes-20230322-162200.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Use _make_ref_key_dict instead of _make_ref_key_msg
-time: 2023-03-22T16:22:00.091222-04:00
-custom:
-  Author: gshank
-  Issue: "621"
diff --git a/.changes/1.5.0/Under the Hood-20221209-161550.yaml b/.changes/1.5.0/Under the Hood-20221209-161550.yaml
deleted file mode 100644
index 846cda6f6..000000000
--- a/.changes/1.5.0/Under the Hood-20221209-161550.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: Remove manual retry to get python job status on cluster
-time: 2022-12-09T16:15:50.834922-08:00
-custom:
-  Author: ChenyuLInx
-  Issue: "422"
-  PR: "423"
diff --git a/.changes/1.5.0/Under the Hood-20230130-170118.yaml b/.changes/1.5.0/Under the Hood-20230130-170118.yaml
deleted file mode 100644
index 39e0ee8f8..000000000
--- a/.changes/1.5.0/Under the Hood-20230130-170118.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: remove tox call to integration tests
-time: 2023-01-30T17:01:18.971636-08:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "459"
-  PR: "498"
diff --git a/.changes/1.5.0/Under the Hood-20230223-105149.yaml b/.changes/1.5.0/Under the Hood-20230223-105149.yaml
deleted file mode 100644
index ca315cc1d..000000000
--- a/.changes/1.5.0/Under the Hood-20230223-105149.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Treat contract config as a python object
-time: 2023-02-23T10:51:49.737457-05:00
-custom:
-  Author: gshank emmyoop
-  Issue: 548 633
diff --git a/.changes/unreleased/Features-20230406-104433.yaml b/.changes/unreleased/Features-20230406-104433.yaml
deleted file mode 100644
index 4ec1d8120..000000000
--- a/.changes/unreleased/Features-20230406-104433.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Add support for model-level constraints
-time: 2023-04-06T10:44:33.045896-04:00
-custom:
-  Author: peterallenwebb
-  Issue: "569"
diff --git a/.changes/unreleased/Fixes-20230222-133301.yaml b/.changes/unreleased/Fixes-20230222-133301.yaml
deleted file mode 100644
index 7c613a109..000000000
--- a/.changes/unreleased/Fixes-20230222-133301.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Align partitions to be updated in incremental insert_overwrite to _dbt_max_partition wrt to nulls
-time: 2023-02-22T13:33:01.607225-08:00
-custom:
-  Author: patkearns10
-  Issue: "544"
diff --git a/.changes/unreleased/Fixes-20230408-035117.yaml b/.changes/unreleased/Fixes-20230408-035117.yaml
deleted file mode 100644
index 56799823b..000000000
--- a/.changes/unreleased/Fixes-20230408-035117.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: ' add full refresh capabilities to tabular bigquery python models to accommodate
-  schema changes'
-time: 2023-04-08T03:51:17.167349-07:00
-custom:
-  Author: versusfacit
-  Issue: "653"
diff --git a/.changes/unreleased/Fixes-20230411-143515.yaml b/.changes/unreleased/Fixes-20230411-143515.yaml
deleted file mode 100644
index c88b53094..000000000
--- a/.changes/unreleased/Fixes-20230411-143515.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Repair accidental change to dev-requirements
-time: 2023-04-11T14:35:15.369296-04:00
-custom:
-  Author: peterallenwebb
-  Issue: "657"
diff --git a/.changes/unreleased/Under the Hood-20230411-143129.yaml b/.changes/unreleased/Under the Hood-20230411-143129.yaml
deleted file mode 100644
index 2d41e7cff..000000000
--- a/.changes/unreleased/Under the Hood-20230411-143129.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Update bigquery__format_column macro to support prettier ContractError message"
-time: 2023-04-11T14:31:29.378726+02:00
-custom:
-  Author: jtcohen6
-  Issue: "656"
diff --git a/CHANGELOG.md b/CHANGELOG.md
index a1e6f96bf..0a8d15f37 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,75 +5,6 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
-## dbt-bigquery 1.5.0-b4 - March 30, 2023
-
-### Features
-
-- Adding `bytes_billed` to `BigQueryAdapterResponse`  ([#560](https://github.com/dbt-labs/dbt-bigquery/issues/560))
-- Modify addapter to support unified constraint fields ([#567](https://github.com/dbt-labs/dbt-bigquery/issues/567))
-- Modify create_table_as to use contract column order ([#579](https://github.com/dbt-labs/dbt-bigquery/issues/579))
-
-### Fixes
-
-- Use _make_ref_key_dict instead of _make_ref_key_msg ([#621](https://github.com/dbt-labs/dbt-bigquery/issues/621))
-
-### Under the Hood
-
-- Treat contract config as a python object ([#548](https://github.com/dbt-labs/dbt-bigquery/issues/548), [#633](https://github.com/dbt-labs/dbt-bigquery/issues/633))
-
-### Contributors
-- [@bruno-szdl](https://github.com/bruno-szdl) ([#560](https://github.com/dbt-labs/dbt-bigquery/issues/560))
-
-
-## dbt-bigquery 1.5.0-b3 - March 16, 2023
-
-### Features
-
-- Enforce contracts on models materialized as tables and views ([#529](https://github.com/dbt-labs/dbt-bigquery/issues/529), [#555](https://github.com/dbt-labs/dbt-bigquery/issues/555))
-- add dataproc serverless config to profile ([#530](https://github.com/dbt-labs/dbt-bigquery/issues/530))
-
-### Fixes
-
-- Fix time ingestion partitioning option regression when combined using `require_partition_filter` option on incremental run ([#483](https://github.com/dbt-labs/dbt-bigquery/issues/483))
-- Allow policy_tags to be removed ([#349](https://github.com/dbt-labs/dbt-bigquery/issues/349))
-- Update connections.py with retry logic for 502 BadGateway ([#562](https://github.com/dbt-labs/dbt-bigquery/issues/562))
-- Fix for Python incremental model regression ([#581](https://github.com/dbt-labs/dbt-bigquery/issues/581))
-- Fix failing test by removing no erroneous asserts. ([#605](https://github.com/dbt-labs/dbt-bigquery/issues/605))
-
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#483](https://github.com/dbt-labs/dbt-bigquery/issues/483))
-- [@garsir](https://github.com/garsir) ([#349](https://github.com/dbt-labs/dbt-bigquery/issues/349))
-- [@patkearns10](https://github.com/patkearns10) ([#562](https://github.com/dbt-labs/dbt-bigquery/issues/562))
-- [@torkjel](https://github.com/torkjel) ([#530](https://github.com/dbt-labs/dbt-bigquery/issues/530))
-
-## dbt-bigquery 1.5.0-b2 - March 02, 2023
-
-### Under the Hood
-
-- Rename constraints_enabled to contract ([#548](https://github.com/dbt-labs/dbt-bigquery/issues/548))
-
-## dbt-bigquery 1.5.0-b1 - February 22, 2023
-
-### Features
-
-- add option to specify excluded columns on merge ([#5260](https://github.com/dbt-labs/dbt-bigquery/issues/5260))
-- dbt-constraints support for BigQuery as per dbt-core issue #1358 ([#444](https://github.com/dbt-labs/dbt-bigquery/issues/444))
-
-### Fixes
-
-- Make BQ access_grant updates thread safe ([#266](https://github.com/dbt-labs/dbt-bigquery/issues/266))
-- Use IEC standard abbreviations (GiB, TiB, etc) ([#477](https://github.com/dbt-labs/dbt-bigquery/issues/477))
-- Pin dataproc serverless spark runtime to `1.1` ([#531](https://github.com/dbt-labs/dbt-bigquery/issues/531))
-
-### Under the Hood
-
-- Remove manual retry to get python job status on cluster ([#422](https://github.com/dbt-labs/dbt-bigquery/issues/422))
-- remove tox call to integration tests ([#459](https://github.com/dbt-labs/dbt-bigquery/issues/459))
-
-### Contributors
-- [@dave-connors-3](https://github.com/dave-connors-3) ([#5260](https://github.com/dbt-labs/dbt-bigquery/issues/5260))
-- [@victoriapm](https://github.com/victoriapm) ([#444](https://github.com/dbt-labs/dbt-bigquery/issues/444))
-
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 * [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 4a7a8147e..fa6c5a1ac 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.5.0b4"
+version = "1.5.0rc1"
diff --git a/setup.py b/setup.py
index b401f311e..910877c82 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.5.0b4"
+package_version = "1.5.0rc1"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From bc843c6183112bc853414b8a7a6b5d1fafc3ddf5 Mon Sep 17 00:00:00 2001
From: "github-actions[bot]"
 <41898282+github-actions[bot]@users.noreply.github.com>
Date: Tue, 18 Apr 2023 11:41:41 -0400
Subject: [PATCH 608/860] [create-pull-request] automated change (#670)

Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
---
 .bumpversion.cfg                     | 2 +-
 .changes/1.6.0-a1.md                 | 1 +
 CHANGELOG.md                         | 4 ++++
 dbt/adapters/bigquery/__version__.py | 2 +-
 setup.py                             | 2 +-
 5 files changed, 8 insertions(+), 3 deletions(-)
 create mode 100644 .changes/1.6.0-a1.md

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 5d390e604..4fef75693 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.5.0rc1
+current_version = 1.6.0a1
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.6.0-a1.md b/.changes/1.6.0-a1.md
new file mode 100644
index 000000000..b724b7b13
--- /dev/null
+++ b/.changes/1.6.0-a1.md
@@ -0,0 +1 @@
+## dbt-bigquery 1.6.0-a1 - April 17, 2023
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 0a8d15f37..065c8d636 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,6 +5,10 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.6.0-a1 - April 17, 2023
+
+
+
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 * [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index fa6c5a1ac..07fc02eef 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.5.0rc1"
+version = "1.6.0a1"
diff --git a/setup.py b/setup.py
index 910877c82..c70e08f7e 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.5.0rc1"
+package_version = "1.6.0a1"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From 9b6a6d379218cc636e35c04042c246a8ec0cb1cf Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Thu, 20 Apr 2023 20:12:52 -0700
Subject: [PATCH 609/860] Adap 444/reformat the repo to precommit hooks (#663)

* Testing with the changes for BQ.WIP

* Fix for tests

* Nit and remove old file

* Fix nits and use string

* remove branch dependency

* Delete the integration dir after porting over the last test

* Restore to varchar

* Naming for update records expression

* Clean up

* Clean up code and get mypy passing

* Fix false flake error

---------

Co-authored-by: Neelesh Srinivas Salian <neeleshssalian@gmail.com>
Co-authored-by: Neelesh Salian <nssalian@users.noreply.github.com>
Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .pre-commit-config.yaml                       |   3 -
 dbt/adapters/bigquery/impl.py                 |  10 +-
 tests/conftest.py                             |  38 +--
 .../column_types/test_alter_column_types.py   |  25 +-
 .../adapter/column_types/test_column_types.py |   9 +-
 .../adapter/hooks/test_model_hooks.py         |   4 +
 .../adapter/hooks/test_run_hooks.py           |   2 +-
 .../incremental_strategy_fixtures.py          |  62 ++---
 tests/functional/adapter/incremental/seeds.py |   2 +-
 .../test_incremental_merge_exclude_columns.py |   7 +-
 .../test_incremental_on_schema_change.py      |  37 ++-
 .../test_incremental_predicates.py            |  19 +-
 .../test_incremental_strategies.py            |  43 ++-
 .../incremental/test_incremental_unique_id.py |   3 +-
 .../query_comment_test/test_query_comment.py  |   8 +-
 .../adapter/simple_bigquery_view/fixtures.py  |  94 +++----
 .../adapter/simple_bigquery_view/seeds.py     |   2 +-
 .../test_simple_bigquery_view.py              |  48 +++-
 .../adapter/simple_copy/test_simple_copy.py   |  17 +-
 .../test_store_test_failures.py               |   6 +-
 tests/functional/adapter/test_aliases.py      |  20 +-
 .../adapter/test_changing_relation_type.py    |   9 +-
 tests/functional/adapter/test_constraints.py  |  73 +++--
 tests/functional/adapter/test_dbt_debug.py    |   1 -
 .../adapter/test_grant_access_to.py           |   2 -
 tests/functional/adapter/test_grants.py       |   3 +-
 tests/functional/adapter/test_persist_docs.py |  99 +++----
 tests/functional/adapter/test_python_model.py |  16 +-
 tests/functional/adapter/test_simple_seed.py  |  59 ++--
 .../functional/adapter/test_simple_snaphot.py |  12 +-
 .../data/ndjson/source.ndjson                 |   2 +-
 .../upload_file_tests/test_upload_file.py     |  99 ++++---
 .../adapter/utils/test_data_types.py          |  13 +-
 .../adapter/utils/test_timestamps.py          |   2 +-
 tests/functional/adapter/utils/test_utils.py  |   1 -
 tests/functional/test_changing_partitions.py  | 263 ++++++++++--------
 tests/functional/test_delete_column_policy.py |  29 +-
 .../test_get_columns_incomplete_database.py   |   9 +-
 tests/functional/test_hours_to_expiration.py  |  16 +-
 .../test_incremental_materialization.py       |  15 +-
 tests/functional/test_location_change.py      |   8 +-
 .../test_override_database/fixtures.py        |  32 ++-
 .../test_override_database.py                 | 109 ++++----
 tests/functional/test_update_column_policy.py |  25 +-
 .../test_update_field_description.py          |  29 +-
 tests/unit/mock_adapter.py                    |   4 +-
 tests/unit/test_bigquery_adapter.py           | 196 ++++++-------
 tests/unit/test_configure_dataproc_batch.py   |  63 +++--
 tests/unit/test_dataset.py                    |  12 +-
 tests/unit/utils.py                           | 121 ++++----
 50 files changed, 915 insertions(+), 866 deletions(-)

diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index b721d3ab7..a98ba0417 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -1,8 +1,5 @@
 # For more on configuring pre-commit hooks (see https://pre-commit.com/)
 
-# TODO: remove global exclusion of tests when testing overhaul is complete
-exclude: '^tests/.*'
-
 default_language_version:
   python: python3
 
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 9016dfb83..fd99927a2 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -2,7 +2,7 @@
 import threading
 from typing import Dict, List, Optional, Any, Set, Union, Type
 
-from dbt.contracts.graph.nodes import ColumnLevelConstraint, ModelLevelConstraint, ConstraintType
+from dbt.contracts.graph.nodes import ColumnLevelConstraint, ModelLevelConstraint, ConstraintType  # type: ignore
 from dbt.dataclass_schema import dbtClassMixin, ValidationError
 
 import dbt.deprecations
@@ -10,7 +10,7 @@
 import dbt.clients.agate_helper
 
 from dbt import ui  # type: ignore
-from dbt.adapters.base import (
+from dbt.adapters.base import (  # type: ignore
     BaseAdapter,
     ConstraintSupport,
     available,
@@ -21,7 +21,7 @@
     PythonJobHelper,
 )
 
-from dbt.adapters.cache import _make_ref_key_dict
+from dbt.adapters.cache import _make_ref_key_dict  # type: ignore
 
 from dbt.adapters.bigquery.relation import BigQueryRelation
 from dbt.adapters.bigquery.dataset import add_access_entry_to_dataset
@@ -923,7 +923,7 @@ def python_submission_helpers(self) -> Dict[str, Type[PythonJobHelper]]:
 
     @classmethod
     def render_column_constraint(cls, constraint: ColumnLevelConstraint) -> Optional[str]:
-        c = super().render_column_constraint(constraint)
+        c = super().render_column_constraint(constraint)  # type: ignore
         if (
             constraint.type == ConstraintType.primary_key
             or constraint.type == ConstraintType.foreign_key
@@ -933,7 +933,7 @@ def render_column_constraint(cls, constraint: ColumnLevelConstraint) -> Optional
 
     @classmethod
     def render_model_constraint(cls, constraint: ModelLevelConstraint) -> Optional[str]:
-        c = super().render_model_constraint(constraint)
+        c = super().render_model_constraint(constraint)  # type: ignore
         if (
             constraint.type == ConstraintType.primary_key
             or constraint.type == ConstraintType.foreign_key
diff --git a/tests/conftest.py b/tests/conftest.py
index 5c30f49fd..0ba0091fb 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -26,29 +26,31 @@ def dbt_profile_target(request):
 
 def oauth_target():
     return {
-        'type': 'bigquery',
-        'method': 'oauth',
-        'threads': 1,
-        'job_retries': 2,
-        'dataproc_region': os.getenv("DATAPROC_REGION"),
-        'dataproc_cluster_name': os.getenv("DATAPROC_CLUSTER_NAME"),
-        'gcs_bucket': os.getenv("GCS_BUCKET")
+        "type": "bigquery",
+        "method": "oauth",
+        "threads": 1,
+        "job_retries": 2,
+        "dataproc_region": os.getenv("DATAPROC_REGION"),
+        "dataproc_cluster_name": os.getenv("DATAPROC_CLUSTER_NAME"),
+        "gcs_bucket": os.getenv("GCS_BUCKET"),
     }
 
 
 def service_account_target():
-    credentials_json_str = os.getenv('BIGQUERY_TEST_SERVICE_ACCOUNT_JSON').replace("'", '')
+    credentials_json_str = os.getenv("BIGQUERY_TEST_SERVICE_ACCOUNT_JSON").replace("'", "")
     credentials = json.loads(credentials_json_str)
-    project_id = credentials.get('project_id')
+    project_id = credentials.get("project_id")
     return {
-        'type': 'bigquery',
-        'method': 'service-account-json',
-        'threads': 1,
-        'job_retries': 2,
-        'project': project_id,
-        'keyfile_json': credentials,
+        "type": "bigquery",
+        "method": "service-account-json",
+        "threads": 1,
+        "job_retries": 2,
+        "project": project_id,
+        "keyfile_json": credentials,
         # following 3 for python model
-        'dataproc_region': os.getenv("DATAPROC_REGION"),
-        'dataproc_cluster_name': os.getenv("DATAPROC_CLUSTER_NAME"), # only needed for cluster submission method
-        'gcs_bucket': os.getenv("GCS_BUCKET")
+        "dataproc_region": os.getenv("DATAPROC_REGION"),
+        "dataproc_cluster_name": os.getenv(
+            "DATAPROC_CLUSTER_NAME"
+        ),  # only needed for cluster submission method
+        "gcs_bucket": os.getenv("GCS_BUCKET"),
     }
diff --git a/tests/functional/adapter/column_types/test_alter_column_types.py b/tests/functional/adapter/column_types/test_alter_column_types.py
index 9bfb88dc1..2d32e6a3c 100644
--- a/tests/functional/adapter/column_types/test_alter_column_types.py
+++ b/tests/functional/adapter/column_types/test_alter_column_types.py
@@ -6,43 +6,34 @@
 from tests.functional.adapter.column_types.fixtures import (
     _MACRO_TEST_ALTER_COLUMN_TYPE,
     _MODEL_ALT_SQL,
-    _ALT_SCHEMA_YML
+    _ALT_SCHEMA_YML,
 )
 
 
-
 class BaseAlterColumnTypes(BaseColumnTypes):
-
     @pytest.fixture(scope="class")
     def macros(self):
         return {
             "test_is_type.sql": macro_test_is_type_sql,
-            "test_alter_column_type.sql": _MACRO_TEST_ALTER_COLUMN_TYPE
+            "test_alter_column_type.sql": _MACRO_TEST_ALTER_COLUMN_TYPE,
         }
 
     def run_and_alter_and_test(self, alter_column_type_args):
         results = run_dbt(["run"])
         assert len(results) == 1
-        run_dbt(['run-operation', 'test_alter_column_type', '--args', alter_column_type_args])
+        run_dbt(["run-operation", "test_alter_column_type", "--args", alter_column_type_args])
         results = run_dbt(["test"])
         assert len(results) == 1
 
 
-
 class TestBigQueryAlterColumnTypes(BaseAlterColumnTypes):
-
     @pytest.fixture(scope="class")
     def models(self):
-        return {
-            "model.sql": _MODEL_ALT_SQL,
-            "schema.yml": _ALT_SCHEMA_YML
-        }
+        return {"model.sql": _MODEL_ALT_SQL, "schema.yml": _ALT_SCHEMA_YML}
 
     def test_bigquery_alter_column_types(self, project):
-        alter_column_type_args = yaml.safe_dump({
-            'model_name': 'model',
-            'column_name': 'int64_col',
-            'new_column_type': 'string'
-        })
+        alter_column_type_args = yaml.safe_dump(
+            {"model_name": "model", "column_name": "int64_col", "new_column_type": "string"}
+        )
 
-        self.run_and_alter_and_test(alter_column_type_args)
\ No newline at end of file
+        self.run_and_alter_and_test(alter_column_type_args)
diff --git a/tests/functional/adapter/column_types/test_column_types.py b/tests/functional/adapter/column_types/test_column_types.py
index 8f7ee4351..54675aa01 100644
--- a/tests/functional/adapter/column_types/test_column_types.py
+++ b/tests/functional/adapter/column_types/test_column_types.py
@@ -2,14 +2,11 @@
 from dbt.tests.adapter.column_types.test_column_types import BaseColumnTypes
 from tests.functional.adapter.column_types.fixtures import _MODEL_SQL, _SCHEMA_YML
 
-class TestBigQueryColumnTypes(BaseColumnTypes):
 
+class TestBigQueryColumnTypes(BaseColumnTypes):
     @pytest.fixture(scope="class")
     def models(self):
-        return {
-            "model.sql": _MODEL_SQL,
-            "schema.yml": _SCHEMA_YML
-        }
+        return {"model.sql": _MODEL_SQL, "schema.yml": _SCHEMA_YML}
 
     def test_run_and_test(self, project):
-        self.run_and_test()
\ No newline at end of file
+        self.run_and_test()
diff --git a/tests/functional/adapter/hooks/test_model_hooks.py b/tests/functional/adapter/hooks/test_model_hooks.py
index ea722dbea..0d0d46c6b 100644
--- a/tests/functional/adapter/hooks/test_model_hooks.py
+++ b/tests/functional/adapter/hooks/test_model_hooks.py
@@ -1,6 +1,7 @@
 from dbt.tests.adapter.hooks import test_model_hooks as core_base
 import pytest
 
+
 class TestBigQueryPrePostModelHooks(core_base.TestPrePostModelHooks):
     def check_hooks(self, state, project, host, count=1):
         self.get_ctx_vars(state, count=count, project=project)
@@ -10,13 +11,16 @@ class TestBigQueryPrePostModelHooksUnderscores(core_base.TestPrePostModelHooksUn
     def check_hooks(self, state, project, host, count=1):
         self.get_ctx_vars(state, count=count, project=project)
 
+
 class TestBigQueryHookRefs(core_base.TestHookRefs):
     def check_hooks(self, state, project, host, count=1):
         self.get_ctx_vars(state, count=count, project=project)
 
+
 class TestBigQueryPrePostModelHooksOnSeeds(core_base.TestPrePostModelHooksOnSeeds):
     def check_hooks(self, state, project, host, count=1):
         self.get_ctx_vars(state, count=count, project=project)
+
     @pytest.fixture(scope="class")
     def project_config_update(self):
         return {
diff --git a/tests/functional/adapter/hooks/test_run_hooks.py b/tests/functional/adapter/hooks/test_run_hooks.py
index 18a1f2bca..23cb4ebde 100644
--- a/tests/functional/adapter/hooks/test_run_hooks.py
+++ b/tests/functional/adapter/hooks/test_run_hooks.py
@@ -35,4 +35,4 @@ def project_config_update(self):
 
 class TestAfterRunHooksBigQuery(core_base.TestAfterRunHooks):
     def check_hooks(self, state, project, host):
-        self.get_ctx_vars(state, project)
\ No newline at end of file
+        self.get_ctx_vars(state, project)
diff --git a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
index 084f4da6b..b3d45ae80 100644
--- a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
+++ b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
@@ -18,23 +18,23 @@
 
 
 with data as (
-    
+
     {% if not is_incremental() %}
-    
+
         select 1 as id, cast('2020-01-01' as datetime) as date_time union all
         select 2 as id, cast('2020-01-01' as datetime) as date_time union all
         select 3 as id, cast('2020-01-01' as datetime) as date_time union all
         select 4 as id, cast('2020-01-01' as datetime) as date_time
-    
+
     {% else %}
-    
+
         select 1 as id, cast('2020-01-01' as datetime) as date_time union all
         select 2 as id, cast('2020-01-01' as datetime) as date_time union all
         select 3 as id, cast('2020-01-01' as datetime) as date_time union all
         select 4 as id, cast('2020-01-02' as datetime) as date_time union all
         select 5 as id, cast('2020-01-02' as datetime) as date_time union all
         select 6 as id, cast('2020-01-02' as datetime) as date_time
-    
+
     {% endif %}
 
 )
@@ -62,23 +62,23 @@
 
 
 with data as (
-    
+
     {% if not is_incremental() %}
-    
+
         select 1 as id, cast('2020-01-01' as datetime) as date_time union all
         select 2 as id, cast('2020-01-01' as datetime) as date_time union all
         select 3 as id, cast('2020-01-01' as datetime) as date_time union all
         select 4 as id, cast('2020-01-01' as datetime) as date_time
-    
+
     {% else %}
-    
+
         select 1 as id, cast('2020-01-01' as datetime) as date_time union all
         select 2 as id, cast('2020-01-01' as datetime) as date_time union all
         select 3 as id, cast('2020-01-01' as datetime) as date_time union all
         select 4 as id, cast('2020-01-02' as datetime) as date_time union all
         select 5 as id, cast('2020-01-02' as datetime) as date_time union all
         select 6 as id, cast('2020-01-02' as datetime) as date_time
-    
+
     {% endif %}
 
 )
@@ -105,23 +105,23 @@
 
 
 with data as (
-    
+
     {% if not is_incremental() %}
-    
+
         select 1 as id, cast('2020-01-01' as date) as date_day union all
         select 2 as id, cast('2020-01-01' as date) as date_day union all
         select 3 as id, cast('2020-01-01' as date) as date_day union all
         select 4 as id, cast('2020-01-01' as date) as date_day
 
     {% else %}
-    
+
         -- we want to overwrite the 4 records in the 2020-01-01 partition
         -- with the 2 records below, but add two more in the 2020-01-02 partition
         select 10 as id, cast('2020-01-01' as date) as date_day union all
         select 20 as id, cast('2020-01-01' as date) as date_day union all
         select 30 as id, cast('2020-01-02' as date) as date_day union all
         select 40 as id, cast('2020-01-02' as date) as date_day
-    
+
     {% endif %}
 
 )
@@ -148,23 +148,23 @@
 
 
 with data as (
-    
+
     {% if not is_incremental() %}
-    
+
         select 1 as id, cast('2020-01-01' as datetime) as date_time union all
         select 2 as id, cast('2020-01-01' as datetime) as date_time union all
         select 3 as id, cast('2020-01-01' as datetime) as date_time union all
         select 4 as id, cast('2020-01-01' as datetime) as date_time
 
     {% else %}
-        
+
         -- we want to overwrite the 4 records in the 2020-01-01 partition
         -- with the 2 records below, but add two more in the 2020-01-02 partition
         select 10 as id, cast('2020-01-01' as datetime) as date_time union all
         select 20 as id, cast('2020-01-01' as datetime) as date_time union all
         select 30 as id, cast('2020-01-02' as datetime) as date_time union all
         select 40 as id, cast('2020-01-02' as datetime) as date_time
-    
+
     {% endif %}
 
 )
@@ -236,23 +236,23 @@
 
 
 with data as (
-    
+
     {% if not is_incremental() %}
-    
+
         select 1 as id, cast('2020-01-01' as date) as date_day union all
         select 2 as id, cast('2020-01-01' as date) as date_day union all
         select 3 as id, cast('2020-01-01' as date) as date_day union all
         select 4 as id, cast('2020-01-01' as date) as date_day
 
     {% else %}
-        
+
         -- we want to overwrite the 4 records in the 2020-01-01 partition
         -- with the 2 records below, but add two more in the 2020-01-02 partition
         select 10 as id, cast('2020-01-01' as date) as date_day union all
         select 20 as id, cast('2020-01-01' as date) as date_day union all
         select 30 as id, cast('2020-01-02' as date) as date_day union all
         select 40 as id, cast('2020-01-02' as date) as date_day
-    
+
     {% endif %}
 
 )
@@ -284,23 +284,23 @@
 
 
 with data as (
-    
+
     {% if not is_incremental() %}
-    
+
         select 1 as id, 20200101 as date_int union all
         select 2 as id, 20200101 as date_int union all
         select 3 as id, 20200101 as date_int union all
         select 4 as id, 20200101 as date_int
 
     {% else %}
-        
+
         -- we want to overwrite the 4 records in the 20200101 partition
         -- with the 2 records below, but add two more in the 20200102 partition
         select 10 as id, 20200101 as date_int union all
         select 20 as id, 20200101 as date_int union all
         select 30 as id, 20200102 as date_int union all
         select 40 as id, 20200102 as date_int
-    
+
     {% endif %}
 
 )
@@ -328,23 +328,23 @@
 
 
 with data as (
-    
+
     {% if not is_incremental() %}
-    
+
         select 1 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
         select 2 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
         select 3 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
         select 4 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour
 
     {% else %}
-    
+
         -- we want to overwrite the 4 records in the 2020-01-01 01:00:00 partition
         -- with the 2 records below, but add two more in the 2020-01-00 02:00:00 partition
         select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
         select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
         select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour union all
         select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
-    
+
     {% endif %}
 
 )
@@ -399,4 +399,4 @@
 {% if is_incremental() %}
 where date_time > '2020-01-01'
 {% endif %}
-""".lstrip()
\ No newline at end of file
+""".lstrip()
diff --git a/tests/functional/adapter/incremental/seeds.py b/tests/functional/adapter/incremental/seeds.py
index ccc265e5f..9198bd244 100644
--- a/tests/functional/adapter/incremental/seeds.py
+++ b/tests/functional/adapter/incremental/seeds.py
@@ -54,4 +54,4 @@
 20
 30
 40
-""".lstrip()
\ No newline at end of file
+""".lstrip()
diff --git a/tests/functional/adapter/incremental/test_incremental_merge_exclude_columns.py b/tests/functional/adapter/incremental/test_incremental_merge_exclude_columns.py
index c8c326004..022ebca07 100644
--- a/tests/functional/adapter/incremental/test_incremental_merge_exclude_columns.py
+++ b/tests/functional/adapter/incremental/test_incremental_merge_exclude_columns.py
@@ -1,8 +1,7 @@
-import pytest
+from dbt.tests.adapter.incremental.test_incremental_merge_exclude_columns import (
+    BaseMergeExcludeColumns,
+)
 
-from dbt.tests.util import run_dbt
-from dbt.tests.adapter.incremental.test_incremental_merge_exclude_columns import BaseMergeExcludeColumns
 
 class TestMergeExcludeColumns(BaseMergeExcludeColumns):
     pass
-
diff --git a/tests/functional/adapter/incremental/test_incremental_on_schema_change.py b/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
index 882b378a2..08ee44bfa 100644
--- a/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
+++ b/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
@@ -40,8 +40,8 @@ class TestIncrementalOnSchemaChange(BaseIncrementalOnSchemaChange):
 
 {% if is_incremental() %}
 
-SELECT id, 
-       cast(field1 as {{string_type}}) as field1, 
+SELECT id,
+       cast(field1 as {{string_type}}) as field1,
        cast(field3 as {{string_type}}) as field3, -- to validate new fields
        cast(field4 as {{string_type}}) AS field4 -- to validate new fields
 
@@ -49,8 +49,8 @@ class TestIncrementalOnSchemaChange(BaseIncrementalOnSchemaChange):
 
 {% else %}
 
-select id, 
-       cast(field1 as {{string_type}}) as field1, 
+select id,
+       cast(field1 as {{string_type}}) as field1,
        cast(field2 as {{string_type}}) as field2
 
 from source_data where id <= 3
@@ -125,30 +125,27 @@ class TestIncrementalOnSchemaChange(BaseIncrementalOnSchemaChange):
 {% endif %}
 """
 
+
 class TestIncrementalOnSchemaChangeBigQuerySpecific(BaseIncrementalOnSchemaChangeSetup):
     @pytest.fixture(scope="class")
     def models(self):
         return {
             "model_a.sql": _MODELS__A,
-            "incremental_sync_all_columns_dynamic_insert_overwrite.sql":
-                _MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_DYNAMIC_INSERT_OVERWRITE,
-            "incremental_sync_all_columns_target.sql":
-                _MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_TARGET,
-            "incremental_time_ingestion_partitioning.sql":
-                _MODELS__INCREMENTAL_TIME_INGESTION_PARTITIONING,
-            "incremental_time_ingestion_partitioning_target.sql":
-                _MODELS__INCREMENTAL_TIME_INGESTION_PARTITIONING_TARGET,
+            "incremental_sync_all_columns_dynamic_insert_overwrite.sql": _MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_DYNAMIC_INSERT_OVERWRITE,
+            "incremental_sync_all_columns_target.sql": _MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_TARGET,
+            "incremental_time_ingestion_partitioning.sql": _MODELS__INCREMENTAL_TIME_INGESTION_PARTITIONING,
+            "incremental_time_ingestion_partitioning_target.sql": _MODELS__INCREMENTAL_TIME_INGESTION_PARTITIONING_TARGET,
         }
-    
+
     def test_run_incremental_sync_all_columns_dynamic_insert_overwrite(self, project):
-        select = 'model_a incremental_sync_all_columns_dynamic_insert_overwrite incremental_sync_all_columns_target'
-        compare_source = 'incremental_sync_all_columns_dynamic_insert_overwrite'
-        compare_target = 'incremental_sync_all_columns_target'
+        select = "model_a incremental_sync_all_columns_dynamic_insert_overwrite incremental_sync_all_columns_target"
+        compare_source = "incremental_sync_all_columns_dynamic_insert_overwrite"
+        compare_target = "incremental_sync_all_columns_target"
         self.run_twice_and_assert(select, compare_source, compare_target, project)
-    
+
     # TODO: this test was added here, but it doesn't actually use 'on_schema_change'
     def test_run_incremental_time_ingestion_partitioning(self, project):
-        select = 'model_a incremental_time_ingestion_partitioning incremental_time_ingestion_partitioning_target'
-        compare_source = 'incremental_time_ingestion_partitioning'
-        compare_target = 'incremental_time_ingestion_partitioning_target'
+        select = "model_a incremental_time_ingestion_partitioning incremental_time_ingestion_partitioning_target"
+        compare_source = "incremental_time_ingestion_partitioning"
+        compare_target = "incremental_time_ingestion_partitioning_target"
         self.run_twice_and_assert(select, compare_source, compare_target, project)
diff --git a/tests/functional/adapter/incremental/test_incremental_predicates.py b/tests/functional/adapter/incremental/test_incremental_predicates.py
index cebd17bab..b3cbed8a8 100644
--- a/tests/functional/adapter/incremental/test_incremental_predicates.py
+++ b/tests/functional/adapter/incremental/test_incremental_predicates.py
@@ -6,22 +6,19 @@ class TestIncrementalPredicatesMergeBigQuery(BaseIncrementalPredicates):
     @pytest.fixture(scope="class")
     def project_config_update(self):
         return {
-            "models": { 
-                "+incremental_predicates": [
-                    "dbt_internal_dest.id != 2"
-                    ],
-                "+incremental_strategy": "merge"
+            "models": {
+                "+incremental_predicates": ["dbt_internal_dest.id != 2"],
+                "+incremental_strategy": "merge",
             }
         }
 
+
 class TestPredicatesMergeBigQuery(BaseIncrementalPredicates):
     @pytest.fixture(scope="class")
     def project_config_update(self):
         return {
-            "models": { 
-                "+predicates": [
-                    "dbt_internal_dest.id != 2"
-                    ],
-                "+incremental_strategy": "merge"
+            "models": {
+                "+predicates": ["dbt_internal_dest.id != 2"],
+                "+incremental_strategy": "merge",
             }
-        }
\ No newline at end of file
+        }
diff --git a/tests/functional/adapter/incremental/test_incremental_strategies.py b/tests/functional/adapter/incremental/test_incremental_strategies.py
index aae438214..a49872310 100644
--- a/tests/functional/adapter/incremental/test_incremental_strategies.py
+++ b/tests/functional/adapter/incremental/test_incremental_strategies.py
@@ -5,8 +5,26 @@
     run_dbt,
 )
 from dbt.tests.adapter.simple_seed.test_seed import SeedConfigBase
-from tests.functional.adapter.incremental.seeds import *
-from tests.functional.adapter.incremental.incremental_strategy_fixtures import *
+from tests.functional.adapter.incremental.seeds import (
+    seed_data_csv,
+    seed_incremental_overwrite_date_expected_csv,
+    seed_incremental_overwrite_day_expected_csv,
+    seed_incremental_overwrite_range_expected_csv,
+    seed_incremental_overwrite_time_expected_csv,
+    seed_merge_expected_csv,
+    seed_incremental_overwrite_day_with_time_partition_expected_csv,
+)
+from tests.functional.adapter.incremental.incremental_strategy_fixtures import (
+    merge_range_sql,
+    merge_time_sql,
+    overwrite_date_sql,
+    overwrite_day_sql,
+    overwrite_day_with_copy_partitions_sql,
+    overwrite_partitions_sql,
+    overwrite_range_sql,
+    overwrite_time_sql,
+    overwrite_day_with_time_ingestion_sql,
+)
 
 
 class TestBigQueryScripting(SeedConfigBase):
@@ -25,7 +43,7 @@ def models(self):
             "incremental_overwrite_partitions.sql": overwrite_partitions_sql,
             "incremental_overwrite_range.sql": overwrite_range_sql,
             "incremental_overwrite_time.sql": overwrite_time_sql,
-            "incremental_overwrite_day_with_time_ingestion.sql": overwrite_day_with_time_ingestion_sql
+            "incremental_overwrite_day_with_time_ingestion.sql": overwrite_day_with_time_ingestion_sql,
         }
 
     @pytest.fixture(scope="class")
@@ -37,25 +55,22 @@ def seeds(self):
             "incremental_overwrite_date_expected.csv": seed_incremental_overwrite_date_expected_csv,
             "incremental_overwrite_day_expected.csv": seed_incremental_overwrite_day_expected_csv,
             "incremental_overwrite_range_expected.csv": seed_incremental_overwrite_range_expected_csv,
-            "incremental_overwrite_day_with_time_partition_expected.csv": seed_incremental_overwrite_day_with_time_partition_expected_csv
+            "incremental_overwrite_day_with_time_partition_expected.csv": seed_incremental_overwrite_day_with_time_partition_expected_csv,
         }
 
     def test__bigquery_assert_incremental_configurations_apply_the_right_strategy(self, project):
-        run_dbt(['seed'])
+        run_dbt(["seed"])
         results = run_dbt()
         assert len(results) == 9
 
         results = run_dbt()
         assert len(results) == 9
         incremental_strategies = [
-            ('incremental_merge_range', 'merge_expected'),
+            ("incremental_merge_range", "merge_expected"),
             ("incremental_merge_time", "merge_expected"),
-            ("incremental_overwrite_time",
-             "incremental_overwrite_time_expected"),
-            ("incremental_overwrite_date",
-             "incremental_overwrite_date_expected"),
-            ("incremental_overwrite_partitions",
-             "incremental_overwrite_date_expected"),
+            ("incremental_overwrite_time", "incremental_overwrite_time_expected"),
+            ("incremental_overwrite_date", "incremental_overwrite_date_expected"),
+            ("incremental_overwrite_partitions", "incremental_overwrite_date_expected"),
             ("incremental_overwrite_day", "incremental_overwrite_day_expected"),
             ("incremental_overwrite_range", "incremental_overwrite_range_expected"),
         ]
@@ -67,6 +82,8 @@ def test__bigquery_assert_incremental_configurations_apply_the_right_strategy(se
 
         # since this table requires a partition filter which check_relations_equal doesn't support extra where clauses
         # we just check column types
-        created = get_relation_columns(project.adapter, "incremental_overwrite_day_with_copy_partitions")
+        created = get_relation_columns(
+            project.adapter, "incremental_overwrite_day_with_copy_partitions"
+        )
         expected = get_relation_columns(project.adapter, "incremental_overwrite_day_expected")
         assert created == expected
diff --git a/tests/functional/adapter/incremental/test_incremental_unique_id.py b/tests/functional/adapter/incremental/test_incremental_unique_id.py
index a485ba9a7..a061adfb5 100644
--- a/tests/functional/adapter/incremental/test_incremental_unique_id.py
+++ b/tests/functional/adapter/incremental/test_incremental_unique_id.py
@@ -1,6 +1,5 @@
-import pytest
 from dbt.tests.adapter.incremental.test_incremental_unique_id import BaseIncrementalUniqueKey
 
 
 class TestUniqueKeyBigQuery(BaseIncrementalUniqueKey):
-    pass
\ No newline at end of file
+    pass
diff --git a/tests/functional/adapter/query_comment_test/test_query_comment.py b/tests/functional/adapter/query_comment_test/test_query_comment.py
index aa7061150..efa138065 100644
--- a/tests/functional/adapter/query_comment_test/test_query_comment.py
+++ b/tests/functional/adapter/query_comment_test/test_query_comment.py
@@ -1,4 +1,3 @@
-import pytest
 from dbt.tests.adapter.query_comment.test_query_comment import (
     BaseQueryComments,
     BaseMacroQueryComments,
@@ -12,17 +11,22 @@
 class TestQueryCommentsBigQuery(BaseQueryComments):
     pass
 
+
 class TestMacroQueryCommentsBigQuery(BaseMacroQueryComments):
     pass
 
+
 class TestMacroArgsQueryCommentsBigQuery(BaseMacroArgsQueryComments):
     pass
 
+
 class TestMacroInvalidQueryCommentsBigQuery(BaseMacroInvalidQueryComments):
     pass
 
+
 class TestNullQueryCommentsBigQuery(BaseNullQueryComments):
     pass
 
+
 class TestEmptyQueryCommentsBigQuery(BaseEmptyQueryComments):
-    pass
\ No newline at end of file
+    pass
diff --git a/tests/functional/adapter/simple_bigquery_view/fixtures.py b/tests/functional/adapter/simple_bigquery_view/fixtures.py
index af43b5f0f..9c13750f4 100644
--- a/tests/functional/adapter/simple_bigquery_view/fixtures.py
+++ b/tests/functional/adapter/simple_bigquery_view/fixtures.py
@@ -4,11 +4,11 @@
 
 clustered_model_sql = """
 {{
-	config(
-		materialized = "table",
-		partition_by = {"field": "updated_at", "data_type": "date"},
-		cluster_by = "dupe",
-	)
+    config(
+        materialized = "table",
+        partition_by = {"field": "updated_at", "data_type": "date"},
+        cluster_by = "dupe",
+    )
 }}
 
 select * from {{ ref('view_model') }}
@@ -20,10 +20,10 @@
 
 labeled_model_sql = """
 {{
-	config(
-		materialized = "table",
-		labels = {'town': 'fish', 'analytics': 'yes'}
-	)
+    config(
+        materialized = "table",
+        labels = {'town': 'fish', 'analytics': 'yes'}
+    )
 }}
 
 select * from {{ ref('view_model') }}
@@ -31,11 +31,11 @@
 
 multi_clustered_model_sql = """
 {{
-	config(
-		materialized = "table",
-		partition_by = {"field": "updated_at", "data_type": "date"},
-		cluster_by = ["dupe","id"],
-	)
+    config(
+        materialized = "table",
+        partition_by = {"field": "updated_at", "data_type": "date"},
+        cluster_by = ["dupe","id"],
+    )
 }}
 
 select * from {{ ref('view_model') }}
@@ -44,10 +44,10 @@
 
 partitioned_model_sql = """
 {{
-	config(
-		materialized = "table",
-		partition_by = {'field': 'updated_at', 'data_type': 'date'},
-	)
+    config(
+        materialized = "table",
+        partition_by = {'field': 'updated_at', 'data_type': 'date'},
+    )
 }}
 
 select * from {{ ref('view_model') }}
@@ -121,13 +121,13 @@
 
 {# This will fail if it is not extracted correctly #}
 {% call set_sql_header(config) %}
-  	CREATE TEMPORARY FUNCTION a_to_b(str STRING)
-	RETURNS STRING AS (
-	  CASE
-	  WHEN LOWER(str) = 'a' THEN 'b'
-	  ELSE str
-	  END
-	);
+    CREATE TEMPORARY FUNCTION a_to_b(str STRING)
+    RETURNS STRING AS (
+      CASE
+      WHEN LOWER(str) = 'a' THEN 'b'
+      ELSE str
+      END
+    );
 {% endcall %}
 
 select a_to_b(dupe) as dupe from {{ ref('view_model') }}
@@ -140,13 +140,13 @@
 {% call set_sql_header(config) %}
     DECLARE int_var INT64 DEFAULT 42;
 
-  	CREATE TEMPORARY FUNCTION a_to_b(str STRING)
-	RETURNS STRING AS (
-	  CASE
-	  WHEN LOWER(str) = 'a' THEN 'b'
-	  ELSE str
-	  END
-	);
+    CREATE TEMPORARY FUNCTION a_to_b(str STRING)
+    RETURNS STRING AS (
+      CASE
+      WHEN LOWER(str) = 'a' THEN 'b'
+      ELSE str
+      END
+    );
 {% endcall %}
 
 select a_to_b(dupe) as dupe from {{ ref('view_model') }}
@@ -171,13 +171,13 @@
 {% call set_sql_header(config) %}
     DECLARE int_var INT64 DEFAULT 42;
 
-  	CREATE TEMPORARY FUNCTION a_to_b(str STRING)
-	RETURNS STRING AS (
-	  CASE
-	  WHEN LOWER(str) = 'a' THEN 'b'
-	  ELSE str
-	  END
-	);
+    CREATE TEMPORARY FUNCTION a_to_b(str STRING)
+    RETURNS STRING AS (
+      CASE
+      WHEN LOWER(str) = 'a' THEN 'b'
+      ELSE str
+      END
+    );
 {% endcall %}
 
 select
@@ -205,13 +205,13 @@
 
 {# This will fail if it is not extracted correctly #}
 {% call set_sql_header(config) %}
-  	CREATE TEMPORARY FUNCTION a_to_b(str STRING)
-	RETURNS STRING AS (
-	  CASE
-	  WHEN LOWER(str) = 'a' THEN 'b'
-	  ELSE str
-	  END
-	);
+    CREATE TEMPORARY FUNCTION a_to_b(str STRING)
+    RETURNS STRING AS (
+      CASE
+      WHEN LOWER(str) = 'a' THEN 'b'
+      ELSE str
+      END
+    );
 {% endcall %}
 
 select
@@ -365,4 +365,4 @@
         {% do exceptions.raise_compiler_error('Incorrect number of columns (expected 1): ' ~ clen) %}
     {% endif %}
 {% endmacro %}
-""".lstrip()
\ No newline at end of file
+""".lstrip()
diff --git a/tests/functional/adapter/simple_bigquery_view/seeds.py b/tests/functional/adapter/simple_bigquery_view/seeds.py
index ccc265e5f..9198bd244 100644
--- a/tests/functional/adapter/simple_bigquery_view/seeds.py
+++ b/tests/functional/adapter/simple_bigquery_view/seeds.py
@@ -54,4 +54,4 @@
 20
 30
 40
-""".lstrip()
\ No newline at end of file
+""".lstrip()
diff --git a/tests/functional/adapter/simple_bigquery_view/test_simple_bigquery_view.py b/tests/functional/adapter/simple_bigquery_view/test_simple_bigquery_view.py
index 75d6eb3d8..6c7bc03e0 100644
--- a/tests/functional/adapter/simple_bigquery_view/test_simple_bigquery_view.py
+++ b/tests/functional/adapter/simple_bigquery_view/test_simple_bigquery_view.py
@@ -3,8 +3,32 @@
 import time
 from dbt.tests.util import run_dbt
 from dbt.tests.adapter.simple_seed.test_seed import SeedConfigBase
-from tests.functional.adapter.simple_bigquery_view.seeds import *
-from tests.functional.adapter.simple_bigquery_view.fixtures import *
+from tests.functional.adapter.simple_bigquery_view.seeds import (
+    seed_data_csv,
+    seed_merge_expected_csv,
+    seed_incremental_overwrite_time_expected_csv,
+    seed_incremental_overwrite_date_expected_csv,
+    seed_incremental_overwrite_day_expected_csv,
+    seed_incremental_overwrite_range_expected_csv,
+)
+from tests.functional.adapter.simple_bigquery_view.fixtures import (
+    clustered_model_sql,
+    funky_case_sql,
+    labeled_model_sql,
+    multi_clustered_model_sql,
+    partitioned_model_sql,
+    sql_header_model_sql,
+    sql_header_model_incr_sql,
+    sql_header_model_incr_insert_overwrite_sql,
+    sql_header_model_incr_insert_overwrite_static_sql,
+    tabel_model_sql,
+    view_model_sql,
+    schema_yml,
+    test_creation_sql,
+    test_int_inference_sql,
+    test_project_for_job_id_sql,
+    wrapped_macros_sql,
+)
 
 
 class BaseBigQueryRun(SeedConfigBase):
@@ -18,7 +42,7 @@ def macros(self):
             "test_creation.sql": test_creation_sql,
             "test_int_inference.sql": test_int_inference_sql,
             "test_project_for_job_id.sql": test_project_for_job_id_sql,
-            "wrapped_macros.sql": wrapped_macros_sql
+            "wrapped_macros.sql": wrapped_macros_sql,
         }
 
     @pytest.fixture(scope="class")
@@ -35,7 +59,7 @@ def models(self):
             "sql_header_model_incr_insert_overwrite_static.sql": sql_header_model_incr_insert_overwrite_static_sql,
             "table_model.sql": tabel_model_sql,
             "view_model.sql": view_model_sql,
-            "schema.yml": schema_yml
+            "schema.yml": schema_yml,
         }
 
     @pytest.fixture(scope="class")
@@ -46,22 +70,22 @@ def seeds(self):
             "incremental_overwrite_time_expected.csv": seed_incremental_overwrite_time_expected_csv,
             "incremental_overwrite_date_expected.csv": seed_incremental_overwrite_date_expected_csv,
             "incremental_overwrite_day_expected.csv": seed_incremental_overwrite_day_expected_csv,
-            "incremental_overwrite_range_expected.csv": seed_incremental_overwrite_range_expected_csv
+            "incremental_overwrite_range_expected.csv": seed_incremental_overwrite_range_expected_csv,
         }
 
     def assert_nondupes_pass(self, project):
         # The 'dupe' model should fail, but all others should pass
-        test_results = run_dbt(['test'], expect_pass=False)
+        test_results = run_dbt(["test"], expect_pass=False)
 
         for test_result in test_results:
-            if 'dupe' in test_result.node.name:
-                assert test_result.status == 'fail'
+            if "dupe" in test_result.node.name:
+                assert test_result.status == "fail"
                 assert not test_result.skipped
                 assert test_result.failures > 0
 
             # assert that actual tests pass
             else:
-                assert test_result.status == 'pass'
+                assert test_result.status == "pass"
                 assert not test_result.skipped
                 assert test_result.failures == 0
 
@@ -69,8 +93,8 @@ def assert_nondupes_pass(self, project):
 class TestSimpleBigQueryRun(BaseBigQueryRun):
     def test__bigquery_simple_run(self, project):
         # make sure seed works twice. Full-refresh is a no-op
-        run_dbt(['seed'])
-        run_dbt(['seed', '--full-refresh'])
+        run_dbt(["seed"])
+        run_dbt(["seed", "--full-refresh"])
 
         results = run_dbt()
         # Bump expected number of results when adding new model
@@ -82,7 +106,7 @@ class TestUnderscoreBigQueryRun(BaseBigQueryRun):
     prefix = "_test{}{:04}".format(int(time.time()), random.randint(0, 9999))
 
     def test_bigquery_run_twice(self, project):
-        run_dbt(['seed'])
+        run_dbt(["seed"])
         results = run_dbt()
         assert len(results) == 11
 
diff --git a/tests/functional/adapter/simple_copy/test_simple_copy.py b/tests/functional/adapter/simple_copy/test_simple_copy.py
index c590ab82f..a0ad14684 100644
--- a/tests/functional/adapter/simple_copy/test_simple_copy.py
+++ b/tests/functional/adapter/simple_copy/test_simple_copy.py
@@ -1,16 +1,10 @@
-import json
-import os
 import pytest
 
 from pathlib import Path
-from pytest import mark
 
 from dbt.tests.util import run_dbt, rm_file, write_file, check_relations_equal
 
-from dbt.tests.adapter.simple_copy.test_simple_copy import (
-   SimpleCopySetup,
-   SimpleCopyBase
-)
+from dbt.tests.adapter.simple_copy.test_simple_copy import SimpleCopyBase
 
 from tests.functional.adapter.simple_copy.fixtures import (
     _MODELS_INCREMENTAL_UPDATE_COLS,
@@ -19,6 +13,7 @@
     _SEEDS__SEED_MERGE_COLS_EXPECTED_RESULT,
 )
 
+
 class TestSimpleCopyBase(SimpleCopyBase):
     pass
 
@@ -26,9 +21,7 @@ class TestSimpleCopyBase(SimpleCopyBase):
 class TestIncrementalMergeColumns:
     @pytest.fixture(scope="class")
     def models(self):
-        return {
-            "incremental_update_cols.sql": _MODELS_INCREMENTAL_UPDATE_COLS
-        }
+        return {"incremental_update_cols.sql": _MODELS_INCREMENTAL_UPDATE_COLS}
 
     @pytest.fixture(scope="class")
     def seeds(self):
@@ -46,6 +39,4 @@ def test_incremental_merge_columns(self, project):
 
         run_dbt(["seed"])
         run_dbt(["run"])
-        check_relations_equal(
-            project.adapter, ["incremental_update_cols", "expected_result"]
-        )
+        check_relations_equal(project.adapter, ["incremental_update_cols", "expected_result"])
diff --git a/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py b/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
index b04579f4b..fbc224529 100644
--- a/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
+++ b/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
@@ -1,6 +1,6 @@
-import pytest
-
-from dbt.tests.adapter.store_test_failures_tests.test_store_test_failures import StoreTestFailuresBase
+from dbt.tests.adapter.store_test_failures_tests.test_store_test_failures import (
+    StoreTestFailuresBase,
+)
 
 
 class TestBigQueryStoreTestFailures(StoreTestFailuresBase):
diff --git a/tests/functional/adapter/test_aliases.py b/tests/functional/adapter/test_aliases.py
index d64a944fa..fa28ce5d9 100644
--- a/tests/functional/adapter/test_aliases.py
+++ b/tests/functional/adapter/test_aliases.py
@@ -1,10 +1,6 @@
 import pytest
 import os
-from dbt.tests.util import run_dbt
-from dbt.tests.adapter.aliases.test_aliases import (
-    BaseAliases,
-    BaseSameAliasDifferentDatabases
-)
+from dbt.tests.adapter.aliases.test_aliases import BaseAliases, BaseSameAliasDifferentDatabases
 
 MACROS__BIGQUERY_CAST_SQL = """
 {% macro bigquery__string_literal(s) %}
@@ -47,12 +43,13 @@
       value: duped_alias
 """
 
+
 class TestAliasesBigQuery(BaseAliases):
     @pytest.fixture(scope="class")
     def macros(self):
         return {
             "bigquery_cast.sql": MACROS__BIGQUERY_CAST_SQL,
-            "expect_value.sql": MACROS__EXPECT_VALUE_SQL
+            "expect_value.sql": MACROS__EXPECT_VALUE_SQL,
         }
 
 
@@ -74,7 +71,7 @@ def project_config_update(self):
     def macros(self):
         return {
             "bigquery_cast.sql": MACROS__BIGQUERY_CAST_SQL,
-            "expect_value.sql": MACROS__EXPECT_VALUE_SQL
+            "expect_value.sql": MACROS__EXPECT_VALUE_SQL,
         }
 
     @pytest.fixture(scope="class")
@@ -82,17 +79,14 @@ def models(self):
         return {
             "schema.yml": MODELS_SCHEMA_YML,
             "model_a.sql": MODELS_DUPE_CUSTOM_DATABASE_A,
-            "model_b.sql": MODELS_DUPE_CUSTOM_DATABASE_B
+            "model_b.sql": MODELS_DUPE_CUSTOM_DATABASE_B,
         }
 
     @pytest.fixture(autouse=True)
     def clean_up(self, project):
         yield
-        with project.adapter.connection_named('__test'):
+        with project.adapter.connection_named("__test"):
             relation = project.adapter.Relation.create(
-                database=os.getenv(
-                "BIGQUERY_TEST_ALT_DATABASE"
-                ),
-                 schema=project.test_schema
+                database=os.getenv("BIGQUERY_TEST_ALT_DATABASE"), schema=project.test_schema
             )
             project.adapter.drop_schema(relation)
diff --git a/tests/functional/adapter/test_changing_relation_type.py b/tests/functional/adapter/test_changing_relation_type.py
index 431e6c6ac..b2918cffc 100644
--- a/tests/functional/adapter/test_changing_relation_type.py
+++ b/tests/functional/adapter/test_changing_relation_type.py
@@ -1,8 +1,9 @@
 from dbt.tests.adapter.relations.test_changing_relation_type import BaseChangeRelationTypeValidator
 
+
 class TestBigQueryChangeRelationTypes(BaseChangeRelationTypeValidator):
     def test_changing_materialization_changes_relation_type(self, project):
-        self._run_and_check_materialization('view')
-        self._run_and_check_materialization('table', extra_args=['--full-refresh'])
-        self._run_and_check_materialization('view', extra_args=['--full-refresh'])
-        self._run_and_check_materialization('incremental', extra_args=['--full-refresh'])
\ No newline at end of file
+        self._run_and_check_materialization("view")
+        self._run_and_check_materialization("table", extra_args=["--full-refresh"])
+        self._run_and_check_materialization("view", extra_args=["--full-refresh"])
+        self._run_and_check_materialization("incremental", extra_args=["--full-refresh"])
diff --git a/tests/functional/adapter/test_constraints.py b/tests/functional/adapter/test_constraints.py
index e0d30cf6f..6b6189092 100644
--- a/tests/functional/adapter/test_constraints.py
+++ b/tests/functional/adapter/test_constraints.py
@@ -1,5 +1,4 @@
 import pytest
-from dbt.tests.util import relation_from_name
 from dbt.tests.adapter.constraints.test_constraints import (
     BaseTableConstraintsColumnsEqual,
     BaseViewConstraintsColumnsEqual,
@@ -7,7 +6,8 @@
     BaseConstraintsRuntimeDdlEnforcement,
     BaseConstraintsRollback,
     BaseIncrementalConstraintsRuntimeDdlEnforcement,
-    BaseIncrementalConstraintsRollback, BaseModelConstraintsRuntimeEnforcement,
+    BaseIncrementalConstraintsRollback,
+    BaseModelConstraintsRuntimeEnforcement,
 )
 from dbt.tests.adapter.constraints.fixtures import (
     my_model_sql,
@@ -18,7 +18,8 @@
     my_model_wrong_name_sql,
     my_model_view_wrong_name_sql,
     my_model_incremental_wrong_name_sql,
-    model_schema_yml, constrained_model_schema_yml,
+    model_schema_yml,
+    constrained_model_schema_yml,
 )
 
 _expected_sql_bigquery = """
@@ -30,11 +31,11 @@
 OPTIONS()
 as (
     select id,
-    color, 
-    date_day from 
-  ( 
-    select 'blue' as color, 
-    1 as id, 
+    color,
+    date_day from
+  (
+    select 'blue' as color,
+    1 as id,
     '2019-01-01' as date_day
   ) as model_subq
 );
@@ -45,6 +46,7 @@
 constraints_yml = model_schema_yml.replace("text", "string")
 model_constraints_yml = constrained_model_schema_yml.replace("text", "string")
 
+
 class BigQueryColumnEqualSetup:
     @pytest.fixture
     def string_type(self):
@@ -58,22 +60,25 @@ def int_type(self):
     def data_types(self, int_type, string_type):
         # sql_column_value, schema_data_type, error_data_type
         return [
-            ['1', int_type, int_type],
+            ["1", int_type, int_type],
             ["'1'", string_type, string_type],
-            ["cast('2019-01-01' as date)", 'date', 'DATE'],
-            ["true", 'bool', 'BOOL'],
-            ["cast('2013-11-03 00:00:00-07' as TIMESTAMP)", 'timestamp', 'TIMESTAMP'],
-            ["['a','b','c']", f'ARRAY<{string_type}>', f'ARRAY<{string_type}>'],
-            ["[1,2,3]", f'ARRAY<{int_type}>', f'ARRAY<{int_type}>'],
-            ["cast(1 as NUMERIC)", 'numeric', 'NUMERIC'],
-            ["""JSON '{"name": "Cooper", "forname": "Alice"}'""", 'json', 'JSON'],
-            ['STRUCT("Rudisha" AS name, [23.4, 26.3, 26.4, 26.1] AS laps)', 'STRUCT<name STRING, laps ARRAY<FLOAT64>>', 'STRUCT<name STRING, laps ARRAY<FLOAT64>>']
+            ["cast('2019-01-01' as date)", "date", "DATE"],
+            ["true", "bool", "BOOL"],
+            ["cast('2013-11-03 00:00:00-07' as TIMESTAMP)", "timestamp", "TIMESTAMP"],
+            ["['a','b','c']", f"ARRAY<{string_type}>", f"ARRAY<{string_type}>"],
+            ["[1,2,3]", f"ARRAY<{int_type}>", f"ARRAY<{int_type}>"],
+            ["cast(1 as NUMERIC)", "numeric", "NUMERIC"],
+            ["""JSON '{"name": "Cooper", "forname": "Alice"}'""", "json", "JSON"],
+            [
+                'STRUCT("Rudisha" AS name, [23.4, 26.3, 26.4, 26.1] AS laps)',
+                "STRUCT<name STRING, laps ARRAY<FLOAT64>>",
+                "STRUCT<name STRING, laps ARRAY<FLOAT64>>",
+            ],
         ]
 
 
 class TestBigQueryTableConstraintsColumnsEqual(
-    BigQueryColumnEqualSetup,
-    BaseTableConstraintsColumnsEqual
+    BigQueryColumnEqualSetup, BaseTableConstraintsColumnsEqual
 ):
     @pytest.fixture(scope="class")
     def models(self):
@@ -85,8 +90,7 @@ def models(self):
 
 
 class TestBigQueryViewConstraintsColumnsEqual(
-    BigQueryColumnEqualSetup,
-    BaseViewConstraintsColumnsEqual
+    BigQueryColumnEqualSetup, BaseViewConstraintsColumnsEqual
 ):
     @pytest.fixture(scope="class")
     def models(self):
@@ -98,8 +102,7 @@ def models(self):
 
 
 class TestBigQueryIncrementalConstraintsColumnsEqual(
-    BigQueryColumnEqualSetup,
-    BaseIncrementalConstraintsColumnsEqual
+    BigQueryColumnEqualSetup, BaseIncrementalConstraintsColumnsEqual
 ):
     @pytest.fixture(scope="class")
     def models(self):
@@ -110,9 +113,7 @@ def models(self):
         }
 
 
-class TestBigQueryTableConstraintsRuntimeDdlEnforcement(
-    BaseConstraintsRuntimeDdlEnforcement
-):
+class TestBigQueryTableConstraintsRuntimeDdlEnforcement(BaseConstraintsRuntimeDdlEnforcement):
     @pytest.fixture(scope="class")
     def models(self):
         return {
@@ -125,9 +126,7 @@ def expected_sql(self, project):
         return _expected_sql_bigquery
 
 
-class TestBigQueryTableConstraintsRollback(
-    BaseConstraintsRollback
-):
+class TestBigQueryTableConstraintsRollback(BaseConstraintsRollback):
     @pytest.fixture(scope="class")
     def models(self):
         return {
@@ -139,6 +138,7 @@ def models(self):
     def expected_error_messages(self):
         return ["Required field id cannot be null"]
 
+
 class TestBigQueryIncrementalConstraintsRuntimeDdlEnforcement(
     BaseIncrementalConstraintsRuntimeDdlEnforcement
 ):
@@ -154,9 +154,7 @@ def expected_sql(self, project):
         return _expected_sql_bigquery
 
 
-class TestBigQueryIncrementalConstraintsRollback(
-    BaseIncrementalConstraintsRollback
-):
+class TestBigQueryIncrementalConstraintsRollback(BaseIncrementalConstraintsRollback):
     @pytest.fixture(scope="class")
     def models(self):
         return {
@@ -170,7 +168,6 @@ def expected_error_messages(self):
 
 
 class TestBigQueryModelConstraintsRuntimeEnforcement(BaseModelConstraintsRuntimeEnforcement):
-
     @pytest.fixture(scope="class")
     def models(self):
         return {
@@ -190,11 +187,11 @@ def expected_sql(self):
 OPTIONS()
 as (
     select id,
-    color, 
-    date_day from 
-  ( 
-    select 1 as id, 
-    'blue' as color, 
+    color,
+    date_day from
+  (
+    select 1 as id,
+    'blue' as color,
     '2019-01-01' as date_day
   ) as model_subq
 );
diff --git a/tests/functional/adapter/test_dbt_debug.py b/tests/functional/adapter/test_dbt_debug.py
index d77977589..0741d7228 100644
--- a/tests/functional/adapter/test_dbt_debug.py
+++ b/tests/functional/adapter/test_dbt_debug.py
@@ -1,4 +1,3 @@
-import pytest
 from dbt.tests.util import run_dbt
 from dbt.tests.adapter.dbt_debug.test_dbt_debug import BaseDebug, BaseDebugProfileVariable
 
diff --git a/tests/functional/adapter/test_grant_access_to.py b/tests/functional/adapter/test_grant_access_to.py
index 76996479d..89ec5a6c8 100644
--- a/tests/functional/adapter/test_grant_access_to.py
+++ b/tests/functional/adapter/test_grant_access_to.py
@@ -1,6 +1,4 @@
-from abc import abstractmethod
 import pytest
-import os
 from dbt.tests.util import run_dbt
 
 SELECT_1 = """
diff --git a/tests/functional/adapter/test_grants.py b/tests/functional/adapter/test_grants.py
index b35e4787e..38f7e0ccc 100644
--- a/tests/functional/adapter/test_grants.py
+++ b/tests/functional/adapter/test_grants.py
@@ -1,5 +1,3 @@
-import pytest
-
 from dbt.tests.adapter.grants.base_grants import BaseGrants
 from dbt.tests.adapter.grants.test_model_grants import BaseModelGrants
 from dbt.tests.adapter.grants.test_incremental_grants import BaseIncrementalGrants
@@ -17,6 +15,7 @@ def privilege_grantee_name_overrides(self):
             "invalid_user": "user:fake@dbtlabs.com",
         }
 
+
 class TestModelGrantsBigQuery(BaseGrantsBigQuery, BaseModelGrants):
     pass
 
diff --git a/tests/functional/adapter/test_persist_docs.py b/tests/functional/adapter/test_persist_docs.py
index 7717b190b..2e32b200d 100644
--- a/tests/functional/adapter/test_persist_docs.py
+++ b/tests/functional/adapter/test_persist_docs.py
@@ -1,16 +1,11 @@
 import json
-import os
 import pytest
 
-from dbt.tests.util import (
-    run_dbt
-)
+from dbt.tests.util import run_dbt
 
 from dbt.tests.adapter.persist_docs.test_persist_docs import (
     BasePersistDocsBase,
     BasePersistDocs,
-    BasePersistDocsColumnMissing,
-    BasePersistDocsCommentOnQuotedColumn,
 )
 
 _MODELS__TABLE_MODEL_NESTED = """
@@ -76,58 +71,56 @@ class TestPersistDocsSimple(BasePersistDocsBase):
     @pytest.fixture(scope="class")
     def project_config_update(self):
         return {
-            'models': {
-                'test': {
-                    '+persist_docs': {
+            "models": {
+                "test": {
+                    "+persist_docs": {
                         "relation": True,
                         "columns": True,
                     },
                 }
             },
-            'seeds': {
-                'test': {
-                    '+persist_docs': {
+            "seeds": {
+                "test": {
+                    "+persist_docs": {
                         "relation": True,
                         "columns": True,
                     },
                 }
             },
         }
+
     def test_persist_docs(self, project):
-        run_dbt(['seed'])
+        run_dbt(["seed"])
         run_dbt()
         desc_map = {
-            'seed': 'Seed model description',
-            'table_model': 'Table model description',
-            'view_model': 'View model description',
+            "seed": "Seed model description",
+            "table_model": "Table model description",
+            "view_model": "View model description",
         }
-        for node_id in ['seed', 'table_model', 'view_model']:
-            with project.adapter.connection_named('_test'):
-                client = project.adapter.connections \
-                    .get_thread_connection().handle
-
-                table_id = "{}.{}.{}".format(
-                    project.database,
-                    project.test_schema,
-                    node_id
-                )
+        for node_id in ["seed", "table_model", "view_model"]:
+            with project.adapter.connection_named("_test"):
+                client = project.adapter.connections.get_thread_connection().handle
+
+                table_id = "{}.{}.{}".format(project.database, project.test_schema, node_id)
                 bq_table = client.get_table(table_id)
 
                 bq_schema = bq_table.schema
 
                 assert bq_table.description.startswith(desc_map[node_id])
-                assert bq_schema[0].description.startswith('id Column description ')
-                if not node_id.startswith('view'):
-                    assert bq_schema[1].description.startswith('Some stuff here and then a call to')
+                assert bq_schema[0].description.startswith("id Column description ")
+                if not node_id.startswith("view"):
+                    assert bq_schema[1].description.startswith(
+                        "Some stuff here and then a call to"
+                    )
 
 
 class TestPersistDocsColumnMissing(BasePersistDocsBase):
     @pytest.fixture(scope="class")
     def project_config_update(self):
         return {
-            'models': {
-                'test': {
-                    '+persist_docs': {
+            "models": {
+                "test": {
+                    "+persist_docs": {
                         "columns": True,
                     },
                 }
@@ -147,15 +140,15 @@ def properties(self):
     def models(self):
         return {
             "table_model_nested.sql": _MODELS__TABLE_MODEL_NESTED,
-            "view_model_nested.sql": _MODELS__VIEW_MODEL_NESTED
+            "view_model_nested.sql": _MODELS__VIEW_MODEL_NESTED,
         }
 
     @pytest.fixture(scope="class")
     def project_config_update(self):
         return {
-            'models': {
-                'test': {
-                    '+persist_docs': {
+            "models": {
+                "test": {
+                    "+persist_docs": {
                         "relation": True,
                         "columns": True,
                     },
@@ -172,36 +165,30 @@ def test_persist_docs(self, project):
 
         Note: dbt-bigquery does not allow comments on models with children nodes
         """
-        run_dbt(['seed'])
+        run_dbt(["seed"])
         run_dbt()
 
-        run_dbt(['docs', 'generate'])
-        with open('target/catalog.json') as fp:
+        run_dbt(["docs", "generate"])
+        with open("target/catalog.json") as fp:
             catalog_data = json.load(fp)
-        assert 'nodes' in catalog_data
-        assert len(catalog_data['nodes']) == 3  # seed, table, and view model
+        assert "nodes" in catalog_data
+        assert len(catalog_data["nodes"]) == 3  # seed, table, and view model
 
-        for node_id in ['table_model_nested', 'view_model_nested']:
+        for node_id in ["table_model_nested", "view_model_nested"]:
             # check the descriptions using the api
-            with project.adapter.connection_named('_test'):
-                client = project.adapter.connections \
-                    .get_thread_connection().handle
-
-                table_id = "{}.{}.{}".format(
-                    project.database,
-                    project.test_schema,
-                    node_id
-                )
+            with project.adapter.connection_named("_test"):
+                client = project.adapter.connections.get_thread_connection().handle
+
+                table_id = "{}.{}.{}".format(project.database, project.test_schema, node_id)
                 bq_schema = client.get_table(table_id).schema
 
                 level_1_field = bq_schema[0]
                 level_2_field = level_1_field.fields[0]
                 level_3_field = level_2_field.fields[0]
-                assert level_3_field.description == \
-                       "level_3 column description"
+                assert level_3_field.description == "level_3 column description"
 
             # check the descriptions in the catalog
-            node = catalog_data['nodes']['model.test.{}'.format(node_id)]
+            node = catalog_data["nodes"]["model.test.{}".format(node_id)]
 
-            level_3_column = node['columns']['level_1.level_2.level_3_a']
-            assert level_3_column['comment'] == "level_3 column description"
+            level_3_column = node["columns"]["level_1.level_2.level_3_a"]
+            assert level_3_column["comment"] == "level_3 column description"
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
index 734fe67d5..955bc6be0 100644
--- a/tests/functional/adapter/test_python_model.py
+++ b/tests/functional/adapter/test_python_model.py
@@ -3,8 +3,9 @@
 from dbt.tests.util import run_dbt, write_file
 import dbt.tests.adapter.python_model.test_python_model as dbt_tests
 
-TEST_SKIP_MESSAGE = "Skipping the Tests since Dataproc serverless is not stable. " \
-                    "TODO: Fix later"
+TEST_SKIP_MESSAGE = (
+    "Skipping the Tests since Dataproc serverless is not stable. " "TODO: Fix later"
+)
 
 
 @pytest.mark.skip(reason=TEST_SKIP_MESSAGE)
@@ -42,16 +43,17 @@ def model(dbt, spark):
 
 @pytest.mark.skip(reason=TEST_SKIP_MESSAGE)
 class TestChangingSchemaDataproc:
-
     @pytest.fixture(scope="class")
     def models(self):
-        return {
-            "simple_python_model.py": models__simple_python_model
-        }
+        return {"simple_python_model.py": models__simple_python_model}
 
     def test_changing_schema(self, project, logs_dir):
         run_dbt(["run"])
-        write_file(models__simple_python_model_v2, project.project_root + '/models', "simple_python_model.py")
+        write_file(
+            models__simple_python_model_v2,
+            project.project_root + "/models",
+            "simple_python_model.py",
+        )
         run_dbt(["run"])
         log_file = os.path.join(logs_dir, "dbt.log")
         with open(log_file, "r") as f:
diff --git a/tests/functional/adapter/test_simple_seed.py b/tests/functional/adapter/test_simple_seed.py
index b803e66d4..c8b178467 100644
--- a/tests/functional/adapter/test_simple_seed.py
+++ b/tests/functional/adapter/test_simple_seed.py
@@ -1,9 +1,6 @@
 import pytest
 from dbt.tests.adapter.simple_seed.fixtures import macros__schema_test
-from dbt.tests.adapter.simple_seed.seeds import (
-    seeds__enabled_in_config_csv,
-    seeds__tricky_csv
-)
+from dbt.tests.adapter.simple_seed.seeds import seeds__enabled_in_config_csv, seeds__tricky_csv
 from dbt.tests.adapter.simple_seed.test_seed import SeedConfigBase
 from dbt.tests.adapter.utils.base_utils import run_dbt
 
@@ -82,7 +79,7 @@ def seeds(self):
         return {
             "seed_enabled.csv": seeds__enabled_in_config_csv,
             "seed_tricky.csv": seeds__tricky_csv,
-            "seed_configs.csv": _SEED_CONFIGS_CSV
+            "seed_configs.csv": _SEED_CONFIGS_CSV,
         }
 
     @pytest.fixture(scope="class")
@@ -93,28 +90,26 @@ def macros(self):
 
     @pytest.fixture(scope="class")
     def models(self):
-        return {
-            "models-bq.yml": _SCHEMA_YML
-        }
+        return {"models-bq.yml": _SCHEMA_YML}
 
     @pytest.fixture(scope="class")
     def project_config_update(self):
         return {
-            'config-version': 2,
-            'seeds': {
-                'test': {
-                    'enabled': False,
-                    'quote_columns': True,
-                    'seed_enabled': {
-                        'enabled': True,
-                        '+column_types': self.seed_enabled_types(),
+            "config-version": 2,
+            "seeds": {
+                "test": {
+                    "enabled": False,
+                    "quote_columns": True,
+                    "seed_enabled": {
+                        "enabled": True,
+                        "+column_types": self.seed_enabled_types(),
                     },
-                    'seed_tricky': {
-                        'enabled': True,
-                        '+column_types': self.seed_tricky_types(),
+                    "seed_tricky": {
+                        "enabled": True,
+                        "+column_types": self.seed_tricky_types(),
                     },
-                    'seed_configs': {
-                        'enabled': True,
+                    "seed_configs": {
+                        "enabled": True,
                     },
                 },
             },
@@ -130,17 +125,14 @@ def seed_enabled_types():
     @staticmethod
     def seed_tricky_types():
         return {
-            'seed_id_str': 'STRING',
-            'looks_like_a_bool': 'STRING',
-            'looks_like_a_date': 'STRING',
+            "seed_id_str": "STRING",
+            "looks_like_a_bool": "STRING",
+            "looks_like_a_date": "STRING",
         }
 
     @staticmethod
     def table_labels():
-        return {
-            "contains_pii": "yes",
-            "contains_pie": "no"
-        }
+        return {"contains_pii": "yes", "contains_pie": "no"}
 
     def test__bigquery_simple_seed_with_column_override_bigquery(self, project):
         seed_results = run_dbt(["seed"])
@@ -151,14 +143,9 @@ def test__bigquery_simple_seed_with_column_override_bigquery(self, project):
     def test__bigquery_seed_table_with_labels_config_bigquery(self, project):
         seed_results = run_dbt(["seed"])
         assert len(seed_results) == 3
-        with project.adapter.connection_named('_test'):
-            client = project.adapter.connections\
-                .get_thread_connection().handle
-            table_id = "{}.{}.{}".format(
-                project.database,
-                project.test_schema,
-                'seed_configs'
-            )
+        with project.adapter.connection_named("_test"):
+            client = project.adapter.connections.get_thread_connection().handle
+            table_id = "{}.{}.{}".format(project.database, project.test_schema, "seed_configs")
             bq_table = client.get_table(table_id)
 
             assert bq_table.labels
diff --git a/tests/functional/adapter/test_simple_snaphot.py b/tests/functional/adapter/test_simple_snaphot.py
index 045da5d44..50da51a76 100644
--- a/tests/functional/adapter/test_simple_snaphot.py
+++ b/tests/functional/adapter/test_simple_snaphot.py
@@ -2,7 +2,10 @@
 
 from dbt.tests.util import run_dbt
 
-from dbt.tests.adapter.simple_snapshot.test_snapshot import BaseSimpleSnapshotBase, BaseSnapshotCheck
+from dbt.tests.adapter.simple_snapshot.test_snapshot import (
+    BaseSimpleSnapshotBase,
+    BaseSnapshotCheck,
+)
 
 SNAPSHOT_TIMESTAMP_SQL = """
 {% snapshot snapshot %}
@@ -32,12 +35,7 @@ def test_updates_are_captured_by_snapshot(self, project):
         Update the last 5 records. Show that all ids are current, but the last 5 reflect updates.
         """
         date_add_expression = "date_add(updated_at, interval 1 day)"
-        self.update_fact_records(
-            {
-                "updated_at": date_add_expression
-            },
-            "id between 16 and 20"
-        )
+        self.update_fact_records({"updated_at": date_add_expression}, "id between 16 and 20")
         run_dbt(["snapshot"])
         self._assert_results(
             ids_with_current_snapshot_records=range(1, 21),
diff --git a/tests/functional/adapter/upload_file_tests/data/ndjson/source.ndjson b/tests/functional/adapter/upload_file_tests/data/ndjson/source.ndjson
index 365d8330b..709898645 100644
--- a/tests/functional/adapter/upload_file_tests/data/ndjson/source.ndjson
+++ b/tests/functional/adapter/upload_file_tests/data/ndjson/source.ndjson
@@ -97,4 +97,4 @@
 {"favorite_color":"green","id":97,"first_name":"Phillip","email":"phoward2o@usa.gov","ip_address":"'255.247.0.175'","updated_at":"2002-12-26 08:44:45"}
 {"favorite_color":"green","id":98,"first_name":"Gloria","email":"gwalker2p@usa.gov","ip_address":"'156.140.7.128'","updated_at":"1997-10-04 07:58:58"}
 {"favorite_color":"green","id":99,"first_name":"Paul","email":"pjohnson2q@umn.edu","ip_address":"'183.59.198.197'","updated_at":"1991-11-14 12:33:55"}
-{"favorite_color":"green","id":100,"first_name":"Frank","email":"fgreene2r@blogspot.com","ip_address":"'150.143.68.121'","updated_at":"2010-06-12 23:55:39"}
\ No newline at end of file
+{"favorite_color":"green","id":100,"first_name":"Frank","email":"fgreene2r@blogspot.com","ip_address":"'150.143.68.121'","updated_at":"2010-06-12 23:55:39"}
diff --git a/tests/functional/adapter/upload_file_tests/test_upload_file.py b/tests/functional/adapter/upload_file_tests/test_upload_file.py
index 2393d4225..474db11ed 100644
--- a/tests/functional/adapter/upload_file_tests/test_upload_file.py
+++ b/tests/functional/adapter/upload_file_tests/test_upload_file.py
@@ -1,9 +1,5 @@
 import pytest
-from dbt.tests.util import (
-    get_relation_columns,
-    run_dbt,
-    run_sql_with_adapter
-)
+from dbt.tests.util import get_relation_columns, run_dbt, run_sql_with_adapter
 import datetime
 import yaml
 
@@ -29,9 +25,22 @@ def macros(self):
     def perform_uploaded_table_checks(table_schema, table_name, project):
         # Test the column names, and data types of the created table
         col_result = get_relation_columns(project.adapter, f"{table_schema}.{table_name}")
-        assert [col_obj[0] for col_obj in col_result] == ['email', 'favorite_color', 'first_name', 'id', 'ip_address',
-                                                          'updated_at']
-        assert [col_obj[1] for col_obj in col_result] == ['STRING', 'STRING', 'STRING', 'INT64', 'STRING', 'TIMESTAMP']
+        assert [col_obj[0] for col_obj in col_result] == [
+            "email",
+            "favorite_color",
+            "first_name",
+            "id",
+            "ip_address",
+            "updated_at",
+        ]
+        assert [col_obj[1] for col_obj in col_result] == [
+            "STRING",
+            "STRING",
+            "STRING",
+            "INT64",
+            "STRING",
+            "TIMESTAMP",
+        ]
 
         # Test the values of the created table
         value_query = f"""
@@ -48,54 +57,62 @@ def perform_uploaded_table_checks(table_schema, table_name, project):
         # There should be 100 distinct id values in this table
         assert value_results[0][1] == 100
         # Maximum updated_at value should be 2016-09-19 14:45:51
-        assert value_results[0][2] == datetime.datetime(2016, 9, 19, 14, 45, 51, tzinfo=datetime.timezone.utc)
+        assert value_results[0][2] == datetime.datetime(
+            2016, 9, 19, 14, 45, 51, tzinfo=datetime.timezone.utc
+        )
 
     def test_bigquery_upload_file_csv(self, project):
         # Create a table from an uploaded CSV file
-        upload_args = yaml.safe_dump({
-            'local_file_path': f"{project.test_data_dir}/csv/source.csv",
-            'database': project.database,
-            'table_schema': project.test_schema,
-            'table_name': 'TestUploadFileCSV',
-            'skip_leading_rows': 1,
-            'autodetect': True,
-            'write_disposition': 'WRITE_TRUNCATE'
-        })
-        upload_result = run_dbt(['run-operation', 'upload_file', '--args', upload_args])
+        upload_args = yaml.safe_dump(
+            {
+                "local_file_path": f"{project.test_data_dir}/csv/source.csv",
+                "database": project.database,
+                "table_schema": project.test_schema,
+                "table_name": "TestUploadFileCSV",
+                "skip_leading_rows": 1,
+                "autodetect": True,
+                "write_disposition": "WRITE_TRUNCATE",
+            }
+        )
+        upload_result = run_dbt(["run-operation", "upload_file", "--args", upload_args])
         assert upload_result.success
 
         # Check if the uploaded table contains expected values and schema
-        self.perform_uploaded_table_checks(project.test_schema, 'TestUploadFileCSV', project)
+        self.perform_uploaded_table_checks(project.test_schema, "TestUploadFileCSV", project)
 
     def test_bigquery_upload_file_ndjson(self, project):
         # Create a table from an uploaded NDJSON file
-        upload_args = yaml.safe_dump({
-            'local_file_path': f"{project.test_data_dir}/ndjson/source.ndjson",
-            'database': project.database,
-            'table_schema': project.test_schema,
-            'table_name': 'TestUploadFileNDJSON',
-            'autodetect': True,
-            'source_format': 'NEWLINE_DELIMITED_JSON',
-            'write_disposition': 'WRITE_TRUNCATE'
-        })
-        upload_result = run_dbt(['run-operation', 'upload_file', '--args', upload_args])
+        upload_args = yaml.safe_dump(
+            {
+                "local_file_path": f"{project.test_data_dir}/ndjson/source.ndjson",
+                "database": project.database,
+                "table_schema": project.test_schema,
+                "table_name": "TestUploadFileNDJSON",
+                "autodetect": True,
+                "source_format": "NEWLINE_DELIMITED_JSON",
+                "write_disposition": "WRITE_TRUNCATE",
+            }
+        )
+        upload_result = run_dbt(["run-operation", "upload_file", "--args", upload_args])
         assert upload_result.success
 
         # Check if the uploaded table contains expected values and schema
-        self.perform_uploaded_table_checks(project.test_schema, 'TestUploadFileNDJSON', project)
+        self.perform_uploaded_table_checks(project.test_schema, "TestUploadFileNDJSON", project)
 
     def test_bigquery_upload_file_parquet(self, project):
         # Create a table from an uploaded parquet file
-        upload_args = yaml.safe_dump({
-            'local_file_path': f"{project.test_data_dir}/parquet/source.parquet",
-            'database': project.database,
-            'table_schema': project.test_schema,
-            'table_name': 'TestUploadFileParquet',
-            'source_format': 'PARQUET',
-            'write_disposition': 'WRITE_TRUNCATE'
-        })
-        upload_result = run_dbt(['run-operation', 'upload_file', '--args', upload_args])
+        upload_args = yaml.safe_dump(
+            {
+                "local_file_path": f"{project.test_data_dir}/parquet/source.parquet",
+                "database": project.database,
+                "table_schema": project.test_schema,
+                "table_name": "TestUploadFileParquet",
+                "source_format": "PARQUET",
+                "write_disposition": "WRITE_TRUNCATE",
+            }
+        )
+        upload_result = run_dbt(["run-operation", "upload_file", "--args", upload_args])
         assert upload_result.success
 
         # Check if the uploaded table contains expected values and schema
-        self.perform_uploaded_table_checks(project.test_schema, 'TestUploadFileParquet', project)
+        self.perform_uploaded_table_checks(project.test_schema, "TestUploadFileParquet", project)
diff --git a/tests/functional/adapter/utils/test_data_types.py b/tests/functional/adapter/utils/test_data_types.py
index e1669910c..722313dcc 100644
--- a/tests/functional/adapter/utils/test_data_types.py
+++ b/tests/functional/adapter/utils/test_data_types.py
@@ -1,4 +1,3 @@
-import pytest
 from dbt.tests.adapter.utils.data_types.test_type_bigint import BaseTypeBigInt
 from dbt.tests.adapter.utils.data_types.test_type_float import BaseTypeFloat
 from dbt.tests.adapter.utils.data_types.test_type_int import BaseTypeInt
@@ -11,29 +10,27 @@
 class TestTypeBigInt(BaseTypeBigInt):
     pass
 
-    
+
 class TestTypeFloat(BaseTypeFloat):
     pass
 
-    
+
 class TestTypeInt(BaseTypeInt):
     pass
 
-    
+
 class TestTypeNumeric(BaseTypeNumeric):
     def numeric_fixture_type(self):
         return "numeric"
 
-    
+
 class TestTypeString(BaseTypeString):
     pass
 
-    
+
 class TestTypeTimestamp(BaseTypeTimestamp):
     pass
 
 
 class TestTypeBoolean(BaseTypeBoolean):
     pass
-
-    
\ No newline at end of file
diff --git a/tests/functional/adapter/utils/test_timestamps.py b/tests/functional/adapter/utils/test_timestamps.py
index 2f35e40ee..40b5e0dce 100644
--- a/tests/functional/adapter/utils/test_timestamps.py
+++ b/tests/functional/adapter/utils/test_timestamps.py
@@ -15,4 +15,4 @@ def expected_schema(self):
     def expected_sql(self):
         return """select current_timestamp() as current_timestamp,
                 current_timestamp as current_timestamp_in_utc_backcompat,
-                current_timestamp as current_timestamp_backcompat"""
\ No newline at end of file
+                current_timestamp as current_timestamp_backcompat"""
diff --git a/tests/functional/adapter/utils/test_utils.py b/tests/functional/adapter/utils/test_utils.py
index 5f5c5d83b..6fb2d05d2 100644
--- a/tests/functional/adapter/utils/test_utils.py
+++ b/tests/functional/adapter/utils/test_utils.py
@@ -11,7 +11,6 @@
 from dbt.tests.adapter.utils.test_dateadd import BaseDateAdd
 from dbt.tests.adapter.utils.test_datediff import BaseDateDiff
 from dbt.tests.adapter.utils.test_date_trunc import BaseDateTrunc
-from dbt.tests.adapter.utils.test_escape_single_quotes import BaseEscapeSingleQuotesQuote
 from dbt.tests.adapter.utils.test_escape_single_quotes import BaseEscapeSingleQuotesBackslash
 from dbt.tests.adapter.utils.test_except import BaseExcept
 from dbt.tests.adapter.utils.test_hash import BaseHash
diff --git a/tests/functional/test_changing_partitions.py b/tests/functional/test_changing_partitions.py
index ad6977556..908f1ddfe 100644
--- a/tests/functional/test_changing_partitions.py
+++ b/tests/functional/test_changing_partitions.py
@@ -50,177 +50,210 @@
 
 
 class BaseBigQueryChangingPartition:
-
     @pytest.fixture(scope="class")
     def macros(self):
-        return {
-            "partition_metadata.sql": _MACRO_SQL
-        }
+        return {"partition_metadata.sql": _MACRO_SQL}
 
-    @pytest.fixture(scope='class')
+    @pytest.fixture(scope="class")
     def models(self):
-        return {
-            "my_model.sql": _MODEL_SQL,
-            "schema.yml": _SCHEMA_YML
-        }
-
+        return {"my_model.sql": _MODEL_SQL, "schema.yml": _SCHEMA_YML}
 
     def run_changes(self, before, after):
-        results = run_dbt(['run', '--vars', json.dumps(before)])
+        results = run_dbt(["run", "--vars", json.dumps(before)])
         assert len(results) == 1
 
-        results = run_dbt(['run', '--vars', json.dumps(after)])
+        results = run_dbt(["run", "--vars", json.dumps(after)])
         assert len(results) == 1
 
     def partitions_test(self, expected):
-        test_results = run_dbt(['test', '--vars', json.dumps(expected)])
+        test_results = run_dbt(["test", "--vars", json.dumps(expected)])
 
         for result in test_results:
             assert result.status == "pass"
-            assert result.skipped == False
+            assert not result.skipped
             assert result.failures == 0
 
-class TestBigQueryChangingPartition(BaseBigQueryChangingPartition):
 
+class TestBigQueryChangingPartition(BaseBigQueryChangingPartition):
     def test_bigquery_add_partition(self, project):
-        before = {"partition_by": None,
-                  "cluster_by": None,
-                  'partition_expiration_days': None,
-                  'require_partition_filter': None}
-        after = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'},
-                 "cluster_by": None,
-                 'partition_expiration_days': 7,
-                 'require_partition_filter': True}
+        before = {
+            "partition_by": None,
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
+        after = {
+            "partition_by": {"field": "cur_time", "data_type": "timestamp"},
+            "cluster_by": None,
+            "partition_expiration_days": 7,
+            "require_partition_filter": True,
+        }
         self.run_changes(before, after)
         self.partitions_test({"expected": 1})
 
     def test_bigquery_add_partition_year(self, project):
-        before = {"partition_by": None,
-                  "cluster_by": None,
-                  'partition_expiration_days': None,
-                  'require_partition_filter': None}
-        after = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp', 'granularity': 'year'},
-                 "cluster_by": None,
-                 'partition_expiration_days': None,
-                 'require_partition_filter': None}
+        before = {
+            "partition_by": None,
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
+        after = {
+            "partition_by": {"field": "cur_time", "data_type": "timestamp", "granularity": "year"},
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
         self.run_changes(before, after)
         self.partitions_test({"expected": 1})
 
     def test_bigquery_add_partition_month(self, project):
-        before = {"partition_by": None,
-                  "cluster_by": None,
-                  'partition_expiration_days': None,
-                  'require_partition_filter': None}
-        after = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp', 'granularity': 'month'},
-                 "cluster_by": None,
-                 'partition_expiration_days': None,
-                 'require_partition_filter': None}
-        self.run_changes(before, after)
-        self.partitions_test({"expected": 1})
-
-    def test_bigquery_add_partition_hour(self, project):
-        before = {"partition_by": None,
-                  "cluster_by": None,
-                  'partition_expiration_days': None,
-                  'require_partition_filter': None}
-        after = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp', 'granularity': 'hour'},
-                 "cluster_by": None,
-                 'partition_expiration_days': None,
-                 'require_partition_filter': None}
+        before = {
+            "partition_by": None,
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
+        after = {
+            "partition_by": {
+                "field": "cur_time",
+                "data_type": "timestamp",
+                "granularity": "month",
+            },
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
         self.run_changes(before, after)
         self.partitions_test({"expected": 1})
 
     def test_bigquery_add_partition_hour(self, project):
-        before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp', 'granularity': 'day'},
-                  "cluster_by": None,
-                  'partition_expiration_days': None,
-                  'require_partition_filter': None}
-        after = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp', 'granularity': 'hour'},
-                 "cluster_by": None,
-                 'partition_expiration_days': None,
-                 'require_partition_filter': None}
+        before = {
+            "partition_by": {"field": "cur_time", "data_type": "timestamp", "granularity": "day"},
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
+        after = {
+            "partition_by": {"field": "cur_time", "data_type": "timestamp", "granularity": "hour"},
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
         self.run_changes(before, after)
         self.partitions_test({"expected": 1})
 
     def test_bigquery_remove_partition(self, project):
-        before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'},
-                  "cluster_by": None,
-                  'partition_expiration_days': None,
-                  'require_partition_filter': None}
-        after = {"partition_by": None,
-                 "cluster_by": None,
-                 'partition_expiration_days': None,
-                 'require_partition_filter': None}
+        before = {
+            "partition_by": {"field": "cur_time", "data_type": "timestamp"},
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
+        after = {
+            "partition_by": None,
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
         self.run_changes(before, after)
 
     def test_bigquery_change_partitions(self, project):
-        before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'},
-                  "cluster_by": None,
-                  'partition_expiration_days': None,
-                  'require_partition_filter': None}
-        after = {"partition_by": {'field': "cur_date"},
-                 "cluster_by": None,
-                 'partition_expiration_days': 7,
-                 'require_partition_filter': True}
+        before = {
+            "partition_by": {"field": "cur_time", "data_type": "timestamp"},
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
+        after = {
+            "partition_by": {"field": "cur_date"},
+            "cluster_by": None,
+            "partition_expiration_days": 7,
+            "require_partition_filter": True,
+        }
         self.run_changes(before, after)
         self.partitions_test({"expected": 1})
         self.run_changes(after, before)
         self.partitions_test({"expected": 1})
 
     def test_bigquery_change_partitions_from_int(self, project):
-        before = {"partition_by": {"field": "id", "data_type": "int64", "range": {"start": 0, "end": 10, "interval": 1}},
-                  "cluster_by": None,
-                  'partition_expiration_days': None,
-                  'require_partition_filter': None}
-        after = {"partition_by": {"field": "cur_date", "data_type": "date"},
-                 "cluster_by": None,
-                 'partition_expiration_days': None,
-                 'require_partition_filter': None}
+        before = {
+            "partition_by": {
+                "field": "id",
+                "data_type": "int64",
+                "range": {"start": 0, "end": 10, "interval": 1},
+            },
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
+        after = {
+            "partition_by": {"field": "cur_date", "data_type": "date"},
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
         self.run_changes(before, after)
         self.partitions_test({"expected": 1})
         self.run_changes(after, before)
         self.partitions_test({"expected": 2})
 
     def test_bigquery_add_clustering(self, project):
-        before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'},
-                  "cluster_by": None,
-                  'partition_expiration_days': None,
-                  'require_partition_filter': None}
-        after = {"partition_by": {'field': "cur_date"},
-                 "cluster_by": "id",
-                 'partition_expiration_days': None,
-                 'require_partition_filter': None}
+        before = {
+            "partition_by": {"field": "cur_time", "data_type": "timestamp"},
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
+        after = {
+            "partition_by": {"field": "cur_date"},
+            "cluster_by": "id",
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
         self.run_changes(before, after)
 
     def test_bigquery_remove_clustering(self, project):
-        before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'},
-                  "cluster_by": "id",
-                  'partition_expiration_days': None,
-                  'require_partition_filter': None}
-        after = {"partition_by": {'field': "cur_date"},
-                 "cluster_by": None,
-                 'partition_expiration_days': None,
-                 'require_partition_filter': None}
+        before = {
+            "partition_by": {"field": "cur_time", "data_type": "timestamp"},
+            "cluster_by": "id",
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
+        after = {
+            "partition_by": {"field": "cur_date"},
+            "cluster_by": None,
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
         self.run_changes(before, after)
 
     def test_bigquery_change_clustering(self, project):
-        before = {"partition_by": {'field': 'cur_time', 'data_type': 'timestamp'},
-                  "cluster_by": "id",
-                  'partition_expiration_days': None,
-                  'require_partition_filter': None}
-        after = {"partition_by": {'field': "cur_date"},
-                 "cluster_by": "name",
-                 'partition_expiration_days': None,
-                 'require_partition_filter': None}
+        before = {
+            "partition_by": {"field": "cur_time", "data_type": "timestamp"},
+            "cluster_by": "id",
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
+        after = {
+            "partition_by": {"field": "cur_date"},
+            "cluster_by": "name",
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
         self.run_changes(before, after)
 
     def test_bigquery_change_clustering_strict(self, project):
-        before = {'partition_by': {'field': 'cur_time', 'data_type': 'timestamp'},
-                  'cluster_by': 'id',
-                  'partition_expiration_days': None,
-                  'require_partition_filter': None}
-        after = {'partition_by': {'field': 'cur_date', 'data_type': 'date'},
-                 'cluster_by': 'name',
-                 'partition_expiration_days': None,
-                 'require_partition_filter': None}
+        before = {
+            "partition_by": {"field": "cur_time", "data_type": "timestamp"},
+            "cluster_by": "id",
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
+        after = {
+            "partition_by": {"field": "cur_date", "data_type": "date"},
+            "cluster_by": "name",
+            "partition_expiration_days": None,
+            "require_partition_filter": None,
+        }
         self.run_changes(before, after)
diff --git a/tests/functional/test_delete_column_policy.py b/tests/functional/test_delete_column_policy.py
index 9ba2353a3..ae2c6bba7 100644
--- a/tests/functional/test_delete_column_policy.py
+++ b/tests/functional/test_delete_column_policy.py
@@ -1,7 +1,5 @@
 import pytest
-from dbt.tests.util import (
-    run_dbt, get_connection, relation_from_name, write_config_file
-)
+from dbt.tests.util import run_dbt, get_connection, relation_from_name, write_config_file
 
 from dbt.adapters.bigquery import BigQueryRelation
 
@@ -45,30 +43,27 @@ class TestBigqueryDeleteColumnPolicy:
 
     @pytest.fixture(scope="class")
     def project_config_update(self):
-        return {
-            'config-version': 2,
-            'vars': {
-                'policy_tag': _POLICY_TAG
-            }
-        }
+        return {"config-version": 2, "vars": {"policy_tag": _POLICY_TAG}}
 
     @pytest.fixture(scope="class")
     def models(self):
-        return {
-            f"{_POLICY_TAG_MODEL_NAME}.sql": _POLICY_TAG_MODEL,
-            "schema.yml": _POLICY_TAG_YML
-        }
+        return {f"{_POLICY_TAG_MODEL_NAME}.sql": _POLICY_TAG_MODEL, "schema.yml": _POLICY_TAG_YML}
 
     def test_bigquery_delete_column_policy_tag(self, project):
-        results = run_dbt(['run', '-f', '--models', 'policy_tag_table'])
+        results = run_dbt(["run", "-f", "--models", "policy_tag_table"])
         assert len(results) == 1
-        write_config_file(_POLICY_TAG_YML_NO_POLICY_TAGS, project.project_root + '/models', "schema.yml") #update the model to remove the policy tag
-        new_results = run_dbt(['run', '-f', '--models', 'policy_tag_table'])
+        write_config_file(
+            _POLICY_TAG_YML_NO_POLICY_TAGS, project.project_root + "/models", "schema.yml"
+        )  # update the model to remove the policy tag
+        new_results = run_dbt(["run", "-f", "--models", "policy_tag_table"])
         assert len(new_results) == 1
         relation: BigQueryRelation = relation_from_name(project.adapter, _POLICY_TAG_MODEL_NAME)
         adapter = project.adapter
         with get_connection(project.adapter) as conn:
             table = conn.handle.get_table(
-                adapter.connections.get_bq_table(relation.database, relation.schema, relation.table))
+                adapter.connections.get_bq_table(
+                    relation.database, relation.schema, relation.table
+                )
+            )
             for schema_field in table.schema:
                 assert schema_field.policy_tags is None
diff --git a/tests/functional/test_get_columns_incomplete_database.py b/tests/functional/test_get_columns_incomplete_database.py
index 35e37779d..4fd92cdb2 100644
--- a/tests/functional/test_get_columns_incomplete_database.py
+++ b/tests/functional/test_get_columns_incomplete_database.py
@@ -33,6 +33,7 @@
       - get_cols_in
 """
 
+
 class TestIncompleteRelationSetup:
     @pytest.fixture(scope="class")
     def properties(self):
@@ -44,11 +45,9 @@ def macros(self):
 
     @pytest.fixture(scope="class")
     def models(self):
-        return { "my_model.sql": models__my_model }
+        return {"my_model.sql": models__my_model}
 
 
 class TestIncompleteRelation(TestIncompleteRelationSetup):
-
-  def test_incomplete_relation(self, project):
-     run_dbt(["build"])
-
+    def test_incomplete_relation(self, project):
+        run_dbt(["build"])
diff --git a/tests/functional/test_hours_to_expiration.py b/tests/functional/test_hours_to_expiration.py
index e38917756..8dbc71149 100644
--- a/tests/functional/test_hours_to_expiration.py
+++ b/tests/functional/test_hours_to_expiration.py
@@ -5,8 +5,8 @@
 select 1 as id
 """
 
-class BaseBigQueryHoursToExpiration:
 
+class BaseBigQueryHoursToExpiration:
     @pytest.fixture(scope="class")
     def models(self):
         return {
@@ -16,19 +16,11 @@ def models(self):
     @pytest.fixture(scope="class")
     def project_config_update(self):
         return {
-            "models": {
-                "test": {
-                    "materialized": "table",
-                    "model": {
-                        "hours_to_expiration": "4"
-                    }
-                }
-            }
+            "models": {"test": {"materialized": "table", "model": {"hours_to_expiration": "4"}}}
         }
 
-class TestBigQueryHoursToExpiration(BaseBigQueryHoursToExpiration):
 
+class TestBigQueryHoursToExpiration(BaseBigQueryHoursToExpiration):
     def test_bigquery_hours_to_expiration(self, project):
         _, stdout = run_dbt_and_capture(["--debug", "run"])
-        assert 'expiration_timestamp=TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 4 hour)' in stdout
-
+        assert "expiration_timestamp=TIMESTAMP_ADD(CURRENT_TIMESTAMP(), INTERVAL 4 hour)" in stdout
diff --git a/tests/functional/test_incremental_materialization.py b/tests/functional/test_incremental_materialization.py
index 6932363b3..dce74db7c 100644
--- a/tests/functional/test_incremental_materialization.py
+++ b/tests/functional/test_incremental_materialization.py
@@ -1,10 +1,7 @@
 import pytest
-import os
-from dbt.tests.util import (
-    run_dbt
-)
+from dbt.tests.util import run_dbt
 
-# This is a short term hack, we need to go back 
+# This is a short term hack, we need to go back
 # and make adapter implementations of:
 # https://github.com/dbt-labs/dbt-core/pull/6330
 
@@ -29,16 +26,16 @@
 -- Test Comment To Prevent Reccurence of https://github.com/dbt-labs/dbt-core/issues/6485
 """
 
+
 class BaseIncrementalModelConfig:
     @pytest.fixture(scope="class")
     def models(self):
-        return {
-            "test_incremental.sql": _INCREMENTAL_MODEL
-        }
+        return {"test_incremental.sql": _INCREMENTAL_MODEL}
+
 
 class TestIncrementalModel(BaseIncrementalModelConfig):
     def test_incremental_model_succeeds(self, project):
         results = run_dbt(["run"])
         assert len(results) == 1
         results = run_dbt(["run"])
-        assert len(results) == 1
\ No newline at end of file
+        assert len(results) == 1
diff --git a/tests/functional/test_location_change.py b/tests/functional/test_location_change.py
index 805d06daf..930c6b04d 100644
--- a/tests/functional/test_location_change.py
+++ b/tests/functional/test_location_change.py
@@ -6,11 +6,11 @@
 select 1 as id
 """
 
-_INVALID_LOCATION = os.getenv('DBT_TEST_BIGQUERY_BAD_LOCATION', 'northamerica-northeast1')
-_VALID_LOCATION = os.getenv('DBT_TEST_BIGQUERY_INITIAL_LOCATION', 'US')
+_INVALID_LOCATION = os.getenv("DBT_TEST_BIGQUERY_BAD_LOCATION", "northamerica-northeast1")
+_VALID_LOCATION = os.getenv("DBT_TEST_BIGQUERY_INITIAL_LOCATION", "US")
 
-class BaseBigQueryLocation:
 
+class BaseBigQueryLocation:
     @pytest.fixture(scope="class")
     def models(self):
         return {
@@ -19,7 +19,6 @@ def models(self):
 
 
 class TestBigqueryValidLocation(BaseBigQueryLocation):
-
     def test_bigquery_valid_location(self, project):
         results = run_dbt()
         for result in results:
@@ -27,7 +26,6 @@ def test_bigquery_valid_location(self, project):
 
 
 class TestBigqueryInvalidLocation(BaseBigQueryLocation):
-
     @pytest.fixture(scope="class")
     def profiles_config_update(self, dbt_profile_target):
         outputs = {"default": dbt_profile_target}
diff --git a/tests/functional/test_override_database/fixtures.py b/tests/functional/test_override_database/fixtures.py
index 315a74e8f..470f42552 100644
--- a/tests/functional/test_override_database/fixtures.py
+++ b/tests/functional/test_override_database/fixtures.py
@@ -14,8 +14,8 @@
 
 models__view_1_sql = """
 {#
-	We are running against a database that must be quoted.
-	These calls ensure that we trigger an error if we're failing to quote at parse-time
+    We are running against a database that must be quoted.
+    These calls ensure that we trigger an error if we're failing to quote at parse-time
 #}
 {% do adapter.already_exists(this.schema, this.table) %}
 {% do adapter.get_relation(this.database, this.schema, this.table) %}
@@ -45,27 +45,29 @@
 5,e
 """
 
+
 @pytest.fixture(scope="class")
 def models():
     return {
-      "view_2.sql": models__view_2_sql,
-      "view_1.sql": models__view_1_sql,
-      "subfolder":
-        {
-          "view_4.sql": models__subfolder__view_4_sql,
-          "view_3.sql": models__subfolder__view_3_sql,
-          },
+        "view_2.sql": models__view_2_sql,
+        "view_1.sql": models__view_1_sql,
+        "subfolder": {
+            "view_4.sql": models__subfolder__view_4_sql,
+            "view_3.sql": models__subfolder__view_3_sql,
+        },
     }
 
+
 @pytest.fixture(scope="class")
 def seeds():
-    return {
-      "seed.csv": seeds__seed_csv
-      }
+    return {"seed.csv": seeds__seed_csv}
+
 
 @pytest.fixture(scope="class")
-def project_files(project_root, models, seeds,):
+def project_files(
+    project_root,
+    models,
+    seeds,
+):
     write_project_files(project_root, "models", models)
     write_project_files(project_root, "seeds", seeds)
-
-
diff --git a/tests/functional/test_override_database/test_override_database.py b/tests/functional/test_override_database/test_override_database.py
index a328809bc..32af1fd19 100644
--- a/tests/functional/test_override_database/test_override_database.py
+++ b/tests/functional/test_override_database/test_override_database.py
@@ -1,13 +1,11 @@
 import pytest
 import os
-from dbt.tests.util import (
-    run_dbt,
-    check_relations_equal_with_relations
-)
-from tests.functional.test_override_database.fixtures import (
+from dbt.tests.util import run_dbt, check_relations_equal_with_relations
+
+from tests.functional.test_override_database.fixtures import (  # noqa: F401
     models,
     seeds,
-    project_files
+    project_files,
 )
 
 ALT_DATABASE = os.getenv("BIGQUERY_TEST_ALT_DATABASE")
@@ -31,13 +29,15 @@ def project_config_update(self):
             },
             "seeds": {
                 "quote_columns": False,
-            }
+            },
         }
 
     @pytest.fixture(scope="function")
     def clean_up(self, project):
         yield
-        relation = project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema)
+        relation = project.adapter.Relation.create(
+            database=ALT_DATABASE, schema=project.test_schema
+        )
         project.adapter.drop_schema(relation)
 
 
@@ -45,13 +45,20 @@ class TestModelOverrideBigQuery(BaseOverrideDatabase):
     def run_database_override(self, project):
         run_dbt(["seed"])
         assert len(run_dbt(["run"])) == 4
-        check_relations_equal_with_relations(project.adapter, [
-            project.adapter.Relation.create(schema=project.test_schema, identifier="seed"),
-            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_2"),
-            project.adapter.Relation.create(schema=project.test_schema, identifier="view_1"),
-            project.adapter.Relation.create(schema=project.test_schema, identifier="view_3"),
-            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_4")
-        ])
+        check_relations_equal_with_relations(
+            project.adapter,
+            [
+                project.adapter.Relation.create(schema=project.test_schema, identifier="seed"),
+                project.adapter.Relation.create(
+                    database=ALT_DATABASE, schema=project.test_schema, identifier="view_2"
+                ),
+                project.adapter.Relation.create(schema=project.test_schema, identifier="view_1"),
+                project.adapter.Relation.create(schema=project.test_schema, identifier="view_3"),
+                project.adapter.Relation.create(
+                    database=ALT_DATABASE, schema=project.test_schema, identifier="view_4"
+                ),
+            ],
+        )
 
     def test_bigquery_database_override(self, project, clean_up):
         self.run_database_override(project)
@@ -64,13 +71,22 @@ def run_database_override(self, project):
         self.assertExpectedRelations(project)
 
     def assertExpectedRelations(self, project):
-        check_relations_equal_with_relations(project.adapter, [
-            project.adapter.Relation.create(schema=project.test_schema, identifier="seed"),
-            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_2"),
-            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_1"),
-            project.adapter.Relation.create(schema=project.test_schema, identifier="view_3"),
-            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_4")
-        ])
+        check_relations_equal_with_relations(
+            project.adapter,
+            [
+                project.adapter.Relation.create(schema=project.test_schema, identifier="seed"),
+                project.adapter.Relation.create(
+                    database=ALT_DATABASE, schema=project.test_schema, identifier="view_2"
+                ),
+                project.adapter.Relation.create(
+                    database=ALT_DATABASE, schema=project.test_schema, identifier="view_1"
+                ),
+                project.adapter.Relation.create(schema=project.test_schema, identifier="view_3"),
+                project.adapter.Relation.create(
+                    database=ALT_DATABASE, schema=project.test_schema, identifier="view_4"
+                ),
+            ],
+        )
 
 
 class TestProjectModelOverrideBigQuery(BaseTestProjectModelOverrideBigQuery):
@@ -78,16 +94,9 @@ class TestProjectModelOverrideBigQuery(BaseTestProjectModelOverrideBigQuery):
     def project_config_update(self):
         return {
             "config-version": 2,
-            "vars": {
-                "alternate_db": ALT_DATABASE,
-            },
             "models": {
                 "database": ALT_DATABASE,
-                "test": {
-                    "subfolder": {
-                        "database": "{{ target.database }}"
-                    }
-                }
+                "test": {"subfolder": {"database": "{{ target.database }}"}},
             },
             "seed-paths": ["seeds"],
             "vars": {
@@ -98,7 +107,7 @@ def project_config_update(self):
             },
             "seeds": {
                 "quote_columns": False,
-            }
+            },
         }
 
     def test_bigquery_database_override(self, project, clean_up):
@@ -110,16 +119,9 @@ class TestProjectModelAliasOverrideBigQuery(BaseTestProjectModelOverrideBigQuery
     def project_config_update(self):
         return {
             "config-version": 2,
-            "vars": {
-                "alternate_db": ALT_DATABASE,
-            },
             "models": {
                 "project": ALT_DATABASE,
-                "test": {
-                    "subfolder": {
-                        "project": "{{ target.database }}"
-                    }
-                }
+                "test": {"subfolder": {"project": "{{ target.database }}"}},
             },
             "seed-paths": ["seeds"],
             "vars": {
@@ -130,7 +132,7 @@ def project_config_update(self):
             },
             "seeds": {
                 "quote_columns": False,
-            }
+            },
         }
 
     def test_bigquery_project_override(self, project, clean_up):
@@ -146,21 +148,28 @@ def project_config_update(self):
             "vars": {
                 "alternate_db": ALT_DATABASE,
             },
-            "seeds": {
-                "database": ALT_DATABASE
-            }
+            "seeds": {"database": ALT_DATABASE},
         }
 
     def run_database_override(self, project):
         run_dbt(["seed"])
         assert len(run_dbt(["run"])) == 4
-        check_relations_equal_with_relations(project.adapter, [
-            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="seed"),
-            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_2"),
-            project.adapter.Relation.create(schema=project.test_schema, identifier="view_1"),
-            project.adapter.Relation.create(schema=project.test_schema, identifier="view_3"),
-            project.adapter.Relation.create(database=ALT_DATABASE, schema=project.test_schema, identifier="view_4")
-        ])
+        check_relations_equal_with_relations(
+            project.adapter,
+            [
+                project.adapter.Relation.create(
+                    database=ALT_DATABASE, schema=project.test_schema, identifier="seed"
+                ),
+                project.adapter.Relation.create(
+                    database=ALT_DATABASE, schema=project.test_schema, identifier="view_2"
+                ),
+                project.adapter.Relation.create(schema=project.test_schema, identifier="view_1"),
+                project.adapter.Relation.create(schema=project.test_schema, identifier="view_3"),
+                project.adapter.Relation.create(
+                    database=ALT_DATABASE, schema=project.test_schema, identifier="view_4"
+                ),
+            ],
+        )
 
     def test_bigquery_database_override(self, project, clean_up):
         self.run_database_override(project)
diff --git a/tests/functional/test_update_column_policy.py b/tests/functional/test_update_column_policy.py
index 0599c0099..bd7fb8f58 100644
--- a/tests/functional/test_update_column_policy.py
+++ b/tests/functional/test_update_column_policy.py
@@ -1,7 +1,5 @@
 import pytest
-from dbt.tests.util import (
-    run_dbt, get_connection, relation_from_name
-)
+from dbt.tests.util import run_dbt, get_connection, relation_from_name
 
 from dbt.adapters.bigquery import BigQueryRelation
 
@@ -30,32 +28,29 @@
 _POLICY_TAG = "projects/dbt-test-env/locations/us/taxonomies/5785568062805976401/policyTags/135489647357012267"
 _POLICY_TAG_MODEL_NAME = "policy_tag_table"
 
+
 class TestBigqueryUpdateColumnPolicy:
     """See BQ docs for more info on policy tags:
     https://cloud.google.com/bigquery/docs/column-level-security#work_with_policy_tags"""
+
     @pytest.fixture(scope="class")
     def project_config_update(self):
-        return {
-            'config-version': 2,
-            'vars': {
-                'policy_tag': _POLICY_TAG
-            }
-        }
+        return {"config-version": 2, "vars": {"policy_tag": _POLICY_TAG}}
 
     @pytest.fixture(scope="class")
     def models(self):
-        return {
-            f"{_POLICY_TAG_MODEL_NAME}.sql": _POLICY_TAG_MODEL,
-            "schema.yml": _POLICY_TAG_YML
-        }
+        return {f"{_POLICY_TAG_MODEL_NAME}.sql": _POLICY_TAG_MODEL, "schema.yml": _POLICY_TAG_YML}
 
     def test_bigquery_update_column_policy_tag(self, project):
-        results = run_dbt(['run', '--models', 'policy_tag_table'])
+        results = run_dbt(["run", "--models", "policy_tag_table"])
         assert len(results) == 1
         relation: BigQueryRelation = relation_from_name(project.adapter, _POLICY_TAG_MODEL_NAME)
         adapter = project.adapter
         with get_connection(project.adapter) as conn:
             table = conn.handle.get_table(
-                adapter.connections.get_bq_table(relation.database, relation.schema, relation.table))
+                adapter.connections.get_bq_table(
+                    relation.database, relation.schema, relation.table
+                )
+            )
             for schema_field in table.schema:
                 assert schema_field.policy_tags.names == (_POLICY_TAG,)
diff --git a/tests/functional/test_update_field_description.py b/tests/functional/test_update_field_description.py
index 68be9c42f..56d7a1754 100644
--- a/tests/functional/test_update_field_description.py
+++ b/tests/functional/test_update_field_description.py
@@ -1,9 +1,5 @@
 import pytest
-from dbt.tests.util import (
-    relation_from_name,
-    get_connection,
-    run_dbt
-)
+from dbt.tests.util import relation_from_name, get_connection, run_dbt
 
 from dbt.adapters.bigquery import BigQueryRelation
 
@@ -18,7 +14,7 @@
   1 field
 """
 _FIELD_DESCRIPTION_MODEL_NAME = "field_description_model"
-_FIELD_DESCRIPTION = 'this is not a field'
+_FIELD_DESCRIPTION = "this is not a field"
 _FIELD_DESCRIPTION_MODEL_YML = """
 version: 2
 
@@ -29,30 +25,31 @@
     description: '{{ var("field_description") }}'
 """
 
+
 class TestBigqueryUpdateColumnDescription:
     @pytest.fixture(scope="class")
     def project_config_update(self):
-        return {
-            'config-version': 2,
-            'vars': {
-                'field_description': _FIELD_DESCRIPTION
-            }
-        }
+        return {"config-version": 2, "vars": {"field_description": _FIELD_DESCRIPTION}}
 
     @pytest.fixture(scope="class")
     def models(self):
         return {
             f"{_FIELD_DESCRIPTION_MODEL_NAME}.sql": _FIELD_DESCRIPTION_MODEL,
-            "schema.yml": _FIELD_DESCRIPTION_MODEL_YML
+            "schema.yml": _FIELD_DESCRIPTION_MODEL_YML,
         }
 
     def test_bigquery_update_column_description(self, project):
-        results = run_dbt(['run'])
+        results = run_dbt(["run"])
         assert len(results) == 1
-        relation: BigQueryRelation = relation_from_name(project.adapter, _FIELD_DESCRIPTION_MODEL_NAME)
+        relation: BigQueryRelation = relation_from_name(
+            project.adapter, _FIELD_DESCRIPTION_MODEL_NAME
+        )
         adapter = project.adapter
         with get_connection(project.adapter) as conn:
             table = conn.handle.get_table(
-                adapter.connections.get_bq_table(relation.database, relation.schema, relation.table))
+                adapter.connections.get_bq_table(
+                    relation.database, relation.schema, relation.table
+                )
+            )
             for schema_field in table.schema:
                 assert schema_field.description == _FIELD_DESCRIPTION
diff --git a/tests/unit/mock_adapter.py b/tests/unit/mock_adapter.py
index 4cd9a9717..d3bdf87b2 100644
--- a/tests/unit/mock_adapter.py
+++ b/tests/unit/mock_adapter.py
@@ -6,11 +6,11 @@
 
 def adapter_factory():
     class MockAdapter(BaseAdapter):
-        ConnectionManager = mock.MagicMock(TYPE='mock')
+        ConnectionManager = mock.MagicMock(TYPE="mock")
         responder = mock.MagicMock()
         # some convenient defaults
         responder.quote.side_effect = lambda identifier: '"{}"'.format(identifier)
-        responder.date_function.side_effect = lambda: 'unitdate()'
+        responder.date_function.side_effect = lambda: "unitdate()"
         responder.is_cancelable.side_effect = lambda: False
 
         @contextmanager
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 28aca8db8..4dece85a6 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -31,7 +31,7 @@
 
 def _bq_conn():
     conn = MagicMock()
-    conn.get.side_effect = lambda x: 'bigquery' if x == 'type' else None
+    conn.get.side_effect = lambda x: "bigquery" if x == "type" else None
     return conn
 
 
@@ -104,43 +104,40 @@ def setUp(self):
                     "threads": 1,
                     "location": "Solar Station",
                 },
-                'dataproc-serverless-configured' : {
-                    'type': 'bigquery',
-                    'method': 'oauth',
-                    'schema': 'dummy_schema',
-                    'threads': 1,
-                    'gcs_bucket': 'dummy-bucket',
-                    'dataproc_region': 'europe-west1',
-                    'submission_method': 'serverless',
-                    'dataproc_batch': {
-                        'environment_config' : {
-                            'execution_config' : {
-                                'service_account': 'dbt@dummy-project.iam.gserviceaccount.com',
-                                'subnetwork_uri': 'dataproc',
-                                'network_tags': ["foo", "bar"]
+                "dataproc-serverless-configured": {
+                    "type": "bigquery",
+                    "method": "oauth",
+                    "schema": "dummy_schema",
+                    "threads": 1,
+                    "gcs_bucket": "dummy-bucket",
+                    "dataproc_region": "europe-west1",
+                    "submission_method": "serverless",
+                    "dataproc_batch": {
+                        "environment_config": {
+                            "execution_config": {
+                                "service_account": "dbt@dummy-project.iam.gserviceaccount.com",
+                                "subnetwork_uri": "dataproc",
+                                "network_tags": ["foo", "bar"],
                             }
                         },
-                        'labels': {
-                            'dbt': 'rocks',
-                            'number': '1'
-                        },
-                        'runtime_config': {
-                            'properties': {
-                                'spark.executor.instances': '4',
-                                'spark.driver.memory': '1g'
+                        "labels": {"dbt": "rocks", "number": "1"},
+                        "runtime_config": {
+                            "properties": {
+                                "spark.executor.instances": "4",
+                                "spark.driver.memory": "1g",
                             }
-                        }
-                    }
+                        },
+                    },
+                },
+                "dataproc-serverless-default": {
+                    "type": "bigquery",
+                    "method": "oauth",
+                    "schema": "dummy_schema",
+                    "threads": 1,
+                    "gcs_bucket": "dummy-bucket",
+                    "dataproc_region": "europe-west1",
+                    "submission_method": "serverless",
                 },
-                'dataproc-serverless-default' : {
-                    'type': 'bigquery',
-                    'method': 'oauth',
-                    'schema': 'dummy_schema',
-                    'threads': 1,
-                    'gcs_bucket': 'dummy-bucket',
-                    'dataproc_region': 'europe-west1',
-                    'submission_method': 'serverless'
-                }
             },
             "target": "oauth",
         }
@@ -162,7 +159,7 @@ def tearDown(self):
     def get_adapter(self, target) -> BigQueryAdapter:
         project = self.project_cfg.copy()
         profile = self.raw_profile.copy()
-        profile['target'] = target
+        profile["target"] = target
 
         config = config_from_parts_or_dicts(
             project=project,
@@ -172,26 +169,31 @@ def get_adapter(self, target) -> BigQueryAdapter:
 
         adapter.connections.query_header = MacroQueryStringSetter(config, MagicMock(macros={}))
 
-        self.qh_patch = patch.object(adapter.connections.query_header, 'add')
+        self.qh_patch = patch.object(adapter.connections.query_header, "add")
         self.mock_query_header_add = self.qh_patch.start()
-        self.mock_query_header_add.side_effect = lambda q: '/* dbt */\n{}'.format(q)
+        self.mock_query_header_add.side_effect = lambda q: "/* dbt */\n{}".format(q)
 
         inject_adapter(adapter, BigQueryPlugin)
         return adapter
 
 
 class TestBigQueryAdapterAcquire(BaseTestBigQueryAdapter):
-    @patch('dbt.adapters.bigquery.connections.get_bigquery_defaults', return_value=('credentials', 'project_id'))
-    @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn())
-    def test_acquire_connection_oauth_no_project_validations(self, mock_open_connection, mock_get_bigquery_defaults):
-        adapter = self.get_adapter('oauth-no-project')
+    @patch(
+        "dbt.adapters.bigquery.connections.get_bigquery_defaults",
+        return_value=("credentials", "project_id"),
+    )
+    @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
+    def test_acquire_connection_oauth_no_project_validations(
+        self, mock_open_connection, mock_get_bigquery_defaults
+    ):
+        adapter = self.get_adapter("oauth-no-project")
         mock_get_bigquery_defaults.assert_called_once()
         try:
-            connection = adapter.acquire_connection('dummy')
-            self.assertEqual(connection.type, 'bigquery')
+            connection = adapter.acquire_connection("dummy")
+            self.assertEqual(connection.type, "bigquery")
 
         except dbt.exceptions.DbtValidationError as e:
-            self.fail('got DbtValidationError: {}'.format(str(e)))
+            self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
             raise
@@ -200,15 +202,15 @@ def test_acquire_connection_oauth_no_project_validations(self, mock_open_connect
         connection.handle
         mock_open_connection.assert_called_once()
 
-    @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn())
+    @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
     def test_acquire_connection_oauth_validations(self, mock_open_connection):
-        adapter = self.get_adapter('oauth')
+        adapter = self.get_adapter("oauth")
         try:
-            connection = adapter.acquire_connection('dummy')
-            self.assertEqual(connection.type, 'bigquery')
+            connection = adapter.acquire_connection("dummy")
+            self.assertEqual(connection.type, "bigquery")
 
         except dbt.exceptions.DbtValidationError as e:
-            self.fail('got DbtValidationError: {}'.format(str(e)))
+            self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
             raise
@@ -217,17 +219,22 @@ def test_acquire_connection_oauth_validations(self, mock_open_connection):
         connection.handle
         mock_open_connection.assert_called_once()
 
-    @patch('dbt.adapters.bigquery.connections.get_bigquery_defaults', return_value=('credentials', 'project_id'))
-    @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn())
-    def test_acquire_connection_dataproc_serverless(self, mock_open_connection, mock_get_bigquery_defaults):
-        adapter = self.get_adapter('dataproc-serverless-configured')
+    @patch(
+        "dbt.adapters.bigquery.connections.get_bigquery_defaults",
+        return_value=("credentials", "project_id"),
+    )
+    @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
+    def test_acquire_connection_dataproc_serverless(
+        self, mock_open_connection, mock_get_bigquery_defaults
+    ):
+        adapter = self.get_adapter("dataproc-serverless-configured")
         mock_get_bigquery_defaults.assert_called_once()
         try:
-            connection = adapter.acquire_connection('dummy')
-            self.assertEqual(connection.type, 'bigquery')
+            connection = adapter.acquire_connection("dummy")
+            self.assertEqual(connection.type, "bigquery")
 
         except dbt.exceptions.ValidationException as e:
-            self.fail('got ValidationException: {}'.format(str(e)))
+            self.fail("got ValidationException: {}".format(str(e)))
 
         except BaseException:
             raise
@@ -236,15 +243,15 @@ def test_acquire_connection_dataproc_serverless(self, mock_open_connection, mock
         connection.handle
         mock_open_connection.assert_called_once()
 
-    @patch('dbt.adapters.bigquery.BigQueryConnectionManager.open', return_value=_bq_conn())
+    @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
     def test_acquire_connection_service_account_validations(self, mock_open_connection):
-        adapter = self.get_adapter('service_account')
+        adapter = self.get_adapter("service_account")
         try:
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
         except dbt.exceptions.DbtValidationError as e:
-            self.fail('got DbtValidationError: {}'.format(str(e)))
+            self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
             raise
@@ -261,7 +268,7 @@ def test_acquire_connection_oauth_token_validations(self, mock_open_connection):
             self.assertEqual(connection.type, "bigquery")
 
         except dbt.exceptions.DbtValidationError as e:
-            self.fail('got DbtValidationError: {}'.format(str(e)))
+            self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
             raise
@@ -278,7 +285,7 @@ def test_acquire_connection_oauth_credentials_validations(self, mock_open_connec
             self.assertEqual(connection.type, "bigquery")
 
         except dbt.exceptions.DbtValidationError as e:
-            self.fail('got DbtValidationError: {}'.format(str(e)))
+            self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
             raise
@@ -297,7 +304,7 @@ def test_acquire_connection_impersonated_service_account_validations(
             self.assertEqual(connection.type, "bigquery")
 
         except dbt.exceptions.DbtValidationError as e:
-            self.fail('got DbtValidationError: {}'.format(str(e)))
+            self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
             raise
@@ -315,7 +322,7 @@ def test_acquire_connection_priority(self, mock_open_connection):
             self.assertEqual(connection.credentials.priority, "batch")
 
         except dbt.exceptions.DbtValidationError as e:
-            self.fail('got DbtValidationError: {}'.format(str(e)))
+            self.fail("got DbtValidationError: {}".format(str(e)))
 
         mock_open_connection.assert_not_called()
         connection.handle
@@ -330,7 +337,7 @@ def test_acquire_connection_maximum_bytes_billed(self, mock_open_connection):
             self.assertEqual(connection.credentials.maximum_bytes_billed, 0)
 
         except dbt.exceptions.DbtValidationError as e:
-            self.fail('got DbtValidationError: {}'.format(str(e)))
+            self.fail("got DbtValidationError: {}".format(str(e)))
 
         mock_open_connection.assert_not_called()
         connection.handle
@@ -486,7 +493,6 @@ def setUp(self):
         pass
 
     def test_replace(self):
-
         kwargs = {
             "type": None,
             "path": {"database": "test-project", "schema": "test_schema", "identifier": "my_view"},
@@ -611,7 +617,6 @@ def test_is_retryable(self):
         self.assertTrue(_is_retryable(rate_limit_error))
 
     def test_drop_dataset(self):
-
         mock_table = Mock()
         mock_table.reference = "table1"
         self.mock_client.list_tables.return_value = [mock_table]
@@ -647,21 +652,21 @@ def test_copy_bq_table_appends(self):
         )
         args, kwargs = self.mock_client.copy_table.call_args
         self.assertEqual(
-            kwargs['job_config'].write_disposition,
-            dbt.adapters.bigquery.impl.WRITE_APPEND)
+            kwargs["job_config"].write_disposition, dbt.adapters.bigquery.impl.WRITE_APPEND
+        )
 
     def test_copy_bq_table_truncates(self):
-        self._copy_table(
-            write_disposition=dbt.adapters.bigquery.impl.WRITE_TRUNCATE)
+        self._copy_table(write_disposition=dbt.adapters.bigquery.impl.WRITE_TRUNCATE)
         args, kwargs = self.mock_client.copy_table.call_args
         self.mock_client.copy_table.assert_called_once_with(
-            [self._table_ref('project', 'dataset', 'table1')],
-            self._table_ref('project', 'dataset', 'table2'),
-            job_config=ANY)
+            [self._table_ref("project", "dataset", "table1")],
+            self._table_ref("project", "dataset", "table2"),
+            job_config=ANY,
+        )
         args, kwargs = self.mock_client.copy_table.call_args
         self.assertEqual(
-            kwargs['job_config'].write_disposition,
-            dbt.adapters.bigquery.impl.WRITE_TRUNCATE)
+            kwargs["job_config"].write_disposition, dbt.adapters.bigquery.impl.WRITE_TRUNCATE
+        )
 
     def test_job_labels_valid_json(self):
         expected = {"key": "value"}
@@ -676,10 +681,10 @@ def _table_ref(self, proj, ds, table):
         return self.connections.table_ref(proj, ds, table)
 
     def _copy_table(self, write_disposition):
-        source = BigQueryRelation.create(
-            database='project', schema='dataset', identifier='table1')
+        source = BigQueryRelation.create(database="project", schema="dataset", identifier="table1")
         destination = BigQueryRelation.create(
-            database='project', schema='dataset', identifier='table2')
+            database="project", schema="dataset", identifier="table2"
+        )
         self.connections.copy_bq_table(source, destination, write_disposition)
 
 
@@ -782,7 +787,8 @@ def test_parse_partition_by(self):
         self.assertEqual(
             adapter.parse_partition_by(
                 {"field": "ts", "data_type": "timestamp", "granularity": "MONTH"}
-            ).to_dict(omit_none=True), {
+            ).to_dict(omit_none=True),
+            {
                 "field": "ts",
                 "data_type": "timestamp",
                 "granularity": "MONTH",
@@ -892,10 +898,9 @@ def test_hours_to_expiration(self):
         self.assertEqual(expected, actual)
 
     def test_hours_to_expiration_temporary(self):
-        adapter = self.get_adapter('oauth')
-        mock_config = create_autospec(
-            RuntimeConfigObject)
-        config = {'hours_to_expiration': 4}
+        adapter = self.get_adapter("oauth")
+        mock_config = create_autospec(RuntimeConfigObject)
+        config = {"hours_to_expiration": 4}
         mock_config.get.side_effect = lambda name: config.get(name)
 
         expected = {
@@ -905,10 +910,9 @@ def test_hours_to_expiration_temporary(self):
         self.assertEqual(expected, actual)
 
     def test_table_kms_key_name(self):
-        adapter = self.get_adapter('oauth')
-        mock_config = create_autospec(
-            RuntimeConfigObject)
-        config = {'kms_key_name': 'some_key'}
+        adapter = self.get_adapter("oauth")
+        mock_config = create_autospec(RuntimeConfigObject)
+        config = {"kms_key_name": "some_key"}
         mock_config.get.side_effect = lambda name: config.get(name)
 
         expected = {"kms_key_name": "'some_key'"}
@@ -916,10 +920,9 @@ def test_table_kms_key_name(self):
         self.assertEqual(expected, actual)
 
     def test_view_kms_key_name(self):
-        adapter = self.get_adapter('oauth')
-        mock_config = create_autospec(
-            RuntimeConfigObject)
-        config = {'kms_key_name': 'some_key'}
+        adapter = self.get_adapter("oauth")
+        mock_config = create_autospec(RuntimeConfigObject)
+        config = {"kms_key_name": "some_key"}
         mock_config.get.side_effect = lambda name: config.get(name)
 
         expected = {}
@@ -1052,14 +1055,18 @@ def test_grant_access_to_calls_update_with_valid_access_entry(self):
                 "type": None,
                 "path": {
                     "database": "another-test-project",
-                    "schema": "test_schema_2", "identifier": "my_view"
+                    "schema": "test_schema_2",
+                    "identifier": "my_view",
                 },
                 "quote_policy": {"identifier": True},
             }
         )
         grant_target_dict = {"dataset": "someOtherDataset", "project": "someProject"}
         self.adapter.grant_access_to(
-            entity=a_different_entity, entity_type="view", role=None, grant_target_dict=grant_target_dict
+            entity=a_different_entity,
+            entity_type="view",
+            role=None,
+            grant_target_dict=grant_target_dict,
         )
         self.mock_client.update_dataset.assert_called_once()
 
@@ -1082,7 +1089,6 @@ def test_sanitize_label(input, output):
 )
 def test_sanitize_label_length(label_length):
     random_string = "".join(
-        random.choice(string.ascii_uppercase + string.digits)
-        for i in range(label_length)
+        random.choice(string.ascii_uppercase + string.digits) for i in range(label_length)
     )
     assert len(_sanitize_label(random_string)) <= _VALIDATE_LABEL_LENGTH_LIMIT
diff --git a/tests/unit/test_configure_dataproc_batch.py b/tests/unit/test_configure_dataproc_batch.py
index 8222645f4..58ff52bab 100644
--- a/tests/unit/test_configure_dataproc_batch.py
+++ b/tests/unit/test_configure_dataproc_batch.py
@@ -5,52 +5,73 @@
 
 from .test_bigquery_adapter import BaseTestBigQueryAdapter
 
+
 # Test application of dataproc_batch configuration to a
 # google.cloud.dataproc_v1.Batch object.
 # This reuses the machinery from BaseTestBigQueryAdapter to get hold of the
 # parsed credentials
 class TestConfigureDataprocBatch(BaseTestBigQueryAdapter):
-
-    @patch('dbt.adapters.bigquery.connections.get_bigquery_defaults', return_value=('credentials', 'project_id'))
+    @patch(
+        "dbt.adapters.bigquery.connections.get_bigquery_defaults",
+        return_value=("credentials", "project_id"),
+    )
     def test_update_dataproc_serverless_batch(self, mock_get_bigquery_defaults):
-        adapter = self.get_adapter('dataproc-serverless-configured')
+        adapter = self.get_adapter("dataproc-serverless-configured")
         mock_get_bigquery_defaults.assert_called_once()
 
-        credentials = adapter.acquire_connection('dummy').credentials
+        credentials = adapter.acquire_connection("dummy").credentials
         self.assertIsNotNone(credentials)
 
         batchConfig = credentials.dataproc_batch
         self.assertIsNotNone(batchConfig)
 
-        raw_batch_config = self.raw_profile['outputs']['dataproc-serverless-configured']['dataproc_batch']
-        raw_environment_config = raw_batch_config['environment_config']
-        raw_execution_config = raw_environment_config['execution_config']
-        raw_labels: dict[str, any] = raw_batch_config['labels']
-        raw_rt_config = raw_batch_config['runtime_config']
+        raw_batch_config = self.raw_profile["outputs"]["dataproc-serverless-configured"][
+            "dataproc_batch"
+        ]
+        raw_environment_config = raw_batch_config["environment_config"]
+        raw_execution_config = raw_environment_config["execution_config"]
+        raw_labels: dict[str, any] = raw_batch_config["labels"]
+        raw_rt_config = raw_batch_config["runtime_config"]
 
-        raw_batch_config = self.raw_profile['outputs']['dataproc-serverless-configured']['dataproc_batch']
+        raw_batch_config = self.raw_profile["outputs"]["dataproc-serverless-configured"][
+            "dataproc_batch"
+        ]
 
         batch = dataproc_v1.Batch()
 
         ServerlessDataProcHelper._update_batch_from_config(raw_batch_config, batch)
 
-        # google's protobuf types expose maps as dict[str, str]
-        to_str_values = lambda d: dict([(k, str(v)) for (k, v) in d.items()])
+        def to_str_values(d):
+            """google's protobuf types expose maps as dict[str, str]"""
+            return dict([(k, str(v)) for (k, v) in d.items()])
 
-        self.assertEqual(batch.environment_config.execution_config.service_account, raw_execution_config['service_account'])
+        self.assertEqual(
+            batch.environment_config.execution_config.service_account,
+            raw_execution_config["service_account"],
+        )
         self.assertFalse(batch.environment_config.execution_config.network_uri)
-        self.assertEqual(batch.environment_config.execution_config.subnetwork_uri, raw_execution_config['subnetwork_uri'])
-        self.assertEqual(batch.environment_config.execution_config.network_tags, raw_execution_config['network_tags'])
+        self.assertEqual(
+            batch.environment_config.execution_config.subnetwork_uri,
+            raw_execution_config["subnetwork_uri"],
+        )
+        self.assertEqual(
+            batch.environment_config.execution_config.network_tags,
+            raw_execution_config["network_tags"],
+        )
         self.assertEqual(batch.labels, to_str_values(raw_labels))
-        self.assertEqual(batch.runtime_config.properties, to_str_values(raw_rt_config['properties']))
-
-
-    @patch('dbt.adapters.bigquery.connections.get_bigquery_defaults', return_value=('credentials', 'project_id'))
+        self.assertEqual(
+            batch.runtime_config.properties, to_str_values(raw_rt_config["properties"])
+        )
+
+    @patch(
+        "dbt.adapters.bigquery.connections.get_bigquery_defaults",
+        return_value=("credentials", "project_id"),
+    )
     def test_default_dataproc_serverless_batch(self, mock_get_bigquery_defaults):
-        adapter = self.get_adapter('dataproc-serverless-default')
+        adapter = self.get_adapter("dataproc-serverless-default")
         mock_get_bigquery_defaults.assert_called_once()
 
-        credentials = adapter.acquire_connection('dummy').credentials
+        credentials = adapter.acquire_connection("dummy").credentials
         self.assertIsNotNone(credentials)
 
         batchConfig = credentials.dataproc_batch
diff --git a/tests/unit/test_dataset.py b/tests/unit/test_dataset.py
index 53109e5cf..681593185 100644
--- a/tests/unit/test_dataset.py
+++ b/tests/unit/test_dataset.py
@@ -10,7 +10,11 @@ def test_add_access_entry_to_dataset_idempotently_adds_entries():
     entity = BigQueryRelation.from_dict(
         {
             "type": None,
-            "path": {"database": "test-project", "schema": "test_schema", "identifier": "my_table"},
+            "path": {
+                "database": "test-project",
+                "schema": "test_schema",
+                "identifier": "my_table",
+            },
             "quote_policy": {"identifier": False},
         }
     ).to_dict()
@@ -29,7 +33,11 @@ def test_add_access_entry_to_dataset_does_not_add_with_pre_existing_entries():
     entity_2 = BigQueryRelation.from_dict(
         {
             "type": None,
-            "path": {"database": "test-project", "schema": "test_schema", "identifier": "some_other_view"},
+            "path": {
+                "database": "test-project",
+                "schema": "test_schema",
+                "identifier": "some_other_view",
+            },
             "quote_policy": {"identifier": False},
         }
     ).to_dict()
diff --git a/tests/unit/utils.py b/tests/unit/utils.py
index 982680449..7cd2bb9ab 100644
--- a/tests/unit/utils.py
+++ b/tests/unit/utils.py
@@ -26,22 +26,22 @@ def normalize(path):
 
 
 class Obj:
-    which = 'blah'
+    which = "blah"
     single_threaded = False
 
 
-def mock_connection(name, state='open'):
+def mock_connection(name, state="open"):
     conn = mock.MagicMock()
     conn.name = name
     conn.state = state
     return conn
 
 
-def profile_from_dict(profile, profile_name, cli_vars='{}'):
+def profile_from_dict(profile, profile_name, cli_vars="{}"):
     from dbt.config import Profile
     from dbt.config.renderer import ProfileRenderer
-    from dbt.context.base import generate_base_context
     from dbt.config.utils import parse_cli_vars
+
     if not isinstance(cli_vars, dict):
         cli_vars = parse_cli_vars(cli_vars)
 
@@ -51,6 +51,7 @@ def profile_from_dict(profile, profile_name, cli_vars='{}'):
     # flags global. This is a bit of a hack, but it's the best way to do it.
     from dbt.flags import set_from_args
     from argparse import Namespace
+
     set_from_args(Namespace(), None)
     return Profile.from_raw_profile_info(
         profile,
@@ -59,16 +60,16 @@ def profile_from_dict(profile, profile_name, cli_vars='{}'):
     )
 
 
-def project_from_dict(project, profile, packages=None, selectors=None, cli_vars='{}'):
-    from dbt.config import Project
+def project_from_dict(project, profile, packages=None, selectors=None, cli_vars="{}"):
     from dbt.config.renderer import DbtProjectYamlRenderer
     from dbt.config.utils import parse_cli_vars
+
     if not isinstance(cli_vars, dict):
         cli_vars = parse_cli_vars(cli_vars)
 
     renderer = DbtProjectYamlRenderer(profile, cli_vars)
 
-    project_root = project.pop('project-root', os.getcwd())
+    project_root = project.pop("project-root", os.getcwd())
 
     partial = PartialProject.from_dicts(
         project_root=project_root,
@@ -79,18 +80,18 @@ def project_from_dict(project, profile, packages=None, selectors=None, cli_vars=
     return partial.render(renderer)
 
 
-
-def config_from_parts_or_dicts(project, profile, packages=None, selectors=None, cli_vars='{}'):
+def config_from_parts_or_dicts(project, profile, packages=None, selectors=None, cli_vars="{}"):
     from dbt.config import Project, Profile, RuntimeConfig
     from copy import deepcopy
     from dbt.config.utils import parse_cli_vars
+
     if not isinstance(cli_vars, dict):
         cli_vars = parse_cli_vars(cli_vars)
 
     if isinstance(project, Project):
         profile_name = project.profile_name
     else:
-        profile_name = project.get('profile')
+        profile_name = project.get("profile")
 
     if not isinstance(profile, Profile):
         profile = profile_from_dict(
@@ -110,16 +111,13 @@ def config_from_parts_or_dicts(project, profile, packages=None, selectors=None,
 
     args = Obj()
     args.vars = cli_vars
-    args.profile_dir = '/dev/null'
-    return RuntimeConfig.from_parts(
-        project=project,
-        profile=profile,
-        args=args
-    )
+    args.profile_dir = "/dev/null"
+    return RuntimeConfig.from_parts(project=project, profile=profile, args=args)
 
 
 def inject_plugin(plugin):
     from dbt.adapters.factory import FACTORY
+
     key = plugin.adapter.type()
     FACTORY.plugins[key] = plugin
 
@@ -127,6 +125,7 @@ def inject_plugin(plugin):
 def inject_plugin_for(config):
     # from dbt.adapters.postgres import Plugin, PostgresAdapter
     from dbt.adapters.factory import FACTORY
+
     FACTORY.load_plugin(config.credentials.type)
     adapter = FACTORY.get_adapter(config)
     return adapter
@@ -138,12 +137,14 @@ def inject_adapter(value, plugin):
     """
     inject_plugin(plugin)
     from dbt.adapters.factory import FACTORY
+
     key = value.type()
     FACTORY.adapters[key] = value
 
 
 def clear_plugin(plugin):
     from dbt.adapters.factory import FACTORY
+
     key = plugin.adapter.type()
     FACTORY.plugins.pop(key, None)
     FACTORY.adapters.pop(key, None)
@@ -163,7 +164,7 @@ def assert_from_dict(self, obj, dct, cls=None):
         if cls is None:
             cls = self.ContractType
         cls.validate(dct)
-        self.assertEqual(cls.from_dict(dct),  obj)
+        self.assertEqual(cls.from_dict(dct), obj)
 
     def assert_symmetric(self, obj, dct, cls=None):
         self.assert_to_dict(obj, dct)
@@ -186,7 +187,7 @@ def compare_dicts(dict1, dict2):
     common_keys = set(first_set).intersection(set(second_set))
     found_differences = False
     for key in common_keys:
-        if dict1[key] != dict2[key] :
+        if dict1[key] != dict2[key]:
             print(f"--- --- first dict: {key}: {str(dict1[key])}")
             print(f"--- --- second dict: {key}: {str(dict2[key])}")
             found_differences = True
@@ -201,7 +202,7 @@ def assert_from_dict(obj, dct, cls=None):
         cls = obj.__class__
     cls.validate(dct)
     obj_from_dict = cls.from_dict(dct)
-    if hasattr(obj, 'created_at'):
+    if hasattr(obj, "created_at"):
         obj_from_dict.created_at = 1
         obj.created_at = 1
     assert obj_from_dict == obj
@@ -209,10 +210,10 @@ def assert_from_dict(obj, dct, cls=None):
 
 def assert_to_dict(obj, dct):
     obj_to_dict = obj.to_dict(omit_none=True)
-    if 'created_at' in obj_to_dict:
-        obj_to_dict['created_at'] = 1
-    if 'created_at' in dct:
-        dct['created_at'] = 1
+    if "created_at" in obj_to_dict:
+        obj_to_dict["created_at"] = 1
+    if "created_at" in dct:
+        dct["created_at"] = 1
     assert obj_to_dict == dct
 
 
@@ -230,25 +231,26 @@ def assert_fails_validation(dct, cls):
 def generate_name_macros(package):
     from dbt.contracts.graph.parsed import ParsedMacro
     from dbt.node_types import NodeType
+
     name_sql = {}
-    for component in ('database', 'schema', 'alias'):
-        if component == 'alias':
-            source = 'node.name'
+    for component in ("database", "schema", "alias"):
+        if component == "alias":
+            source = "node.name"
         else:
-            source = f'target.{component}'
-        name = f'generate_{component}_name'
-        sql = f'{{% macro {name}(value, node) %}} {{% if value %}} {{{{ value }}}} {{% else %}} {{{{ {source} }}}} {{% endif %}} {{% endmacro %}}'
+            source = f"target.{component}"
+        name = f"generate_{component}_name"
+        sql = f"{{% macro {name}(value, node) %}} {{% if value %}} {{{{ value }}}} {{% else %}} {{{{ {source} }}}} {{% endif %}} {{% endmacro %}}"
         name_sql[name] = sql
 
     for name, sql in name_sql.items():
         pm = ParsedMacro(
             name=name,
             resource_type=NodeType.Macro,
-            unique_id=f'macro.{package}.{name}',
+            unique_id=f"macro.{package}.{name}",
             package_name=package,
-            original_file_path=normalize('macros/macro.sql'),
-            root_path='./dbt_packages/root',
-            path=normalize('macros/macro.sql'),
+            original_file_path=normalize("macros/macro.sql"),
+            root_path="./dbt_packages/root",
+            path=normalize("macros/macro.sql"),
             macro_sql=sql,
         )
         yield pm
@@ -257,17 +259,18 @@ def generate_name_macros(package):
 class TestAdapterConversions(TestCase):
     def _get_tester_for(self, column_type):
         from dbt.clients import agate_helper
+
         if column_type is agate.TimeDelta:  # dbt never makes this!
             return agate.TimeDelta()
 
         for instance in agate_helper.DEFAULT_TYPE_TESTER._possible_types:
-            if isinstance(instance, column_type): # include child types 
+            if isinstance(instance, column_type):  # include child types
                 return instance
 
-        raise ValueError(f'no tester for {column_type}')
+        raise ValueError(f"no tester for {column_type}")
 
     def _make_table_of(self, rows, column_types):
-        column_names = list(string.ascii_letters[:len(rows[0])])
+        column_names = list(string.ascii_letters[: len(rows[0])])
         if isinstance(column_types, type):
             column_types = [self._get_tester_for(column_types) for _ in column_names]
         else:
@@ -276,50 +279,48 @@ def _make_table_of(self, rows, column_types):
         return table
 
 
-def MockMacro(package, name='my_macro', **kwargs):
+def MockMacro(package, name="my_macro", **kwargs):
     from dbt.contracts.graph.parsed import ParsedMacro
     from dbt.node_types import NodeType
 
     mock_kwargs = dict(
         resource_type=NodeType.Macro,
         package_name=package,
-        unique_id=f'macro.{package}.{name}',
-        original_file_path='/dev/null',
+        unique_id=f"macro.{package}.{name}",
+        original_file_path="/dev/null",
     )
 
     mock_kwargs.update(kwargs)
 
-    macro = mock.MagicMock(
-        spec=ParsedMacro,
-        **mock_kwargs
-    )
+    macro = mock.MagicMock(spec=ParsedMacro, **mock_kwargs)
     macro.name = name
     return macro
 
 
-def MockMaterialization(package, name='my_materialization', adapter_type=None, **kwargs):
+def MockMaterialization(package, name="my_materialization", adapter_type=None, **kwargs):
     if adapter_type is None:
-        adapter_type = 'default'
-    kwargs['adapter_type'] = adapter_type
-    return MockMacro(package, f'materialization_{name}_{adapter_type}', **kwargs)
+        adapter_type = "default"
+    kwargs["adapter_type"] = adapter_type
+    return MockMacro(package, f"materialization_{name}_{adapter_type}", **kwargs)
 
 
-def MockGenerateMacro(package, component='some_component', **kwargs):
-    name = f'generate_{component}_name'
+def MockGenerateMacro(package, component="some_component", **kwargs):
+    name = f"generate_{component}_name"
     return MockMacro(package, name=name, **kwargs)
 
 
 def MockSource(package, source_name, name, **kwargs):
     from dbt.node_types import NodeType
     from dbt.contracts.graph.parsed import ParsedSourceDefinition
+
     src = mock.MagicMock(
         __class__=ParsedSourceDefinition,
         resource_type=NodeType.Source,
         source_name=source_name,
         package_name=package,
-        unique_id=f'source.{package}.{source_name}.{name}',
-        search_name=f'{source_name}.{name}',
-        **kwargs
+        unique_id=f"source.{package}.{source_name}.{name}",
+        search_name=f"{source_name}.{name}",
+        **kwargs,
     )
     src.name = name
     return src
@@ -328,6 +329,7 @@ def MockSource(package, source_name, name, **kwargs):
 def MockNode(package, name, resource_type=None, **kwargs):
     from dbt.node_types import NodeType
     from dbt.contracts.graph.parsed import ParsedModelNode, ParsedSeedNode
+
     if resource_type is None:
         resource_type = NodeType.Model
     if resource_type == NodeType.Model:
@@ -335,14 +337,14 @@ def MockNode(package, name, resource_type=None, **kwargs):
     elif resource_type == NodeType.Seed:
         cls = ParsedSeedNode
     else:
-        raise ValueError(f'I do not know how to handle {resource_type}')
+        raise ValueError(f"I do not know how to handle {resource_type}")
     node = mock.MagicMock(
         __class__=cls,
         resource_type=resource_type,
         package_name=package,
-        unique_id=f'{str(resource_type)}.{package}.{name}',
+        unique_id=f"{str(resource_type)}.{package}.{name}",
         search_name=name,
-        **kwargs
+        **kwargs,
     )
     node.name = name
     return node
@@ -351,22 +353,23 @@ def MockNode(package, name, resource_type=None, **kwargs):
 def MockDocumentation(package, name, **kwargs):
     from dbt.node_types import NodeType
     from dbt.contracts.graph.parsed import ParsedDocumentation
+
     doc = mock.MagicMock(
         __class__=ParsedDocumentation,
         resource_type=NodeType.Documentation,
         package_name=package,
         search_name=name,
-        unique_id=f'{package}.{name}',
-        **kwargs
+        unique_id=f"{package}.{name}",
+        **kwargs,
     )
     doc.name = name
     return doc
 
 
-def load_internal_manifest_macros(config, macro_hook = lambda m: None):
+def load_internal_manifest_macros(config, macro_hook=lambda m: None):
     from dbt.parser.manifest import ManifestLoader
-    return ManifestLoader.load_macros(config, macro_hook)
 
+    return ManifestLoader.load_macros(config, macro_hook)
 
 
 def dict_replace(dct, **kwargs):

From 2ffa7686d8e637ca2403a9086d151b8a94823285 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 21 Apr 2023 00:20:36 -0400
Subject: [PATCH 610/860] Update pytest requirement from ~=7.2 to ~=7.3 (#654)

Updates the requirements on [pytest](https://github.com/pytest-dev/pytest) to permit the latest version.
- [Release notes](https://github.com/pytest-dev/pytest/releases)
- [Changelog](https://github.com/pytest-dev/pytest/blob/main/CHANGELOG.rst)
- [Commits](https://github.com/pytest-dev/pytest/compare/7.2.0...7.3.0)

---
updated-dependencies:
- dependency-name: pytest
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index d0ee86a79..a51336219 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -18,7 +18,7 @@ pip-tools~=6.12
 pre-commit~=2.21;python_version=="3.7"
 pre-commit~=3.2;python_version>="3.8"
 pre-commit-hooks~=4.4
-pytest~=7.2
+pytest~=7.3
 pytest-csv~=3.0
 pytest-dotenv~=0.5.2
 pytest-logbook~=1.2

From 16c77504c09cc4d9e52ad9ab1e0dc832a250f2bf Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 21 Apr 2023 01:07:29 -0400
Subject: [PATCH 611/860] Bump mypy from 1.1.1 to 1.2.0 (#649)

Bumps [mypy](https://github.com/python/mypy) from 1.1.1 to 1.2.0.
- [Release notes](https://github.com/python/mypy/releases)
- [Commits](https://github.com/python/mypy/compare/v1.1.1...v1.2.0)

---
updated-dependencies:
- dependency-name: mypy
  dependency-type: direct:development
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index a51336219..ae01ce115 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -13,7 +13,7 @@ flake8~=6.0;python_version>="3.8"
 flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
-mypy==1.1.1  # patch updates have historically introduced breaking changes
+mypy==1.2.0  # patch updates have historically introduced breaking changes
 pip-tools~=6.12
 pre-commit~=2.21;python_version=="3.7"
 pre-commit~=3.2;python_version>="3.8"

From 8081a27ec38a515bc83c6afd1b57784190d3f44e Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 21 Apr 2023 10:51:12 -0400
Subject: [PATCH 612/860] Update black requirement from ~=23.1 to ~=23.3 (#637)

Updates the requirements on [black](https://github.com/psf/black) to permit the latest version.
- [Release notes](https://github.com/psf/black/releases)
- [Changelog](https://github.com/psf/black/blob/main/CHANGES.md)
- [Commits](https://github.com/psf/black/compare/23.1.0...23.3.0)

---
updated-dependencies:
- dependency-name: black
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index ae01ce115..2b59c6535 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -5,7 +5,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor
-black~=23.1
+black~=23.3
 bumpversion~=0.6.0
 click~=8.1
 flake8~=5.0;python_version=="3.7"

From 85c0a961919a4a5a4c56410c5d2db773665a1a59 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 21 Apr 2023 11:26:39 -0400
Subject: [PATCH 613/860] Update types-pytz requirement from ~=2023.2 to
 ~=2023.3 (#640)

Updates the requirements on [types-pytz](https://github.com/python/typeshed) to permit the latest version.
- [Release notes](https://github.com/python/typeshed/releases)
- [Commits](https://github.com/python/typeshed/commits)

---
updated-dependencies:
- dependency-name: types-pytz
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index 2b59c6535..7ece3c349 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -26,7 +26,7 @@ pytest-xdist~=3.2
 pytz~=2023.2
 tox~=3.0;python_version=="3.7"
 tox~=4.4;python_version>="3.8"
-types-pytz~=2023.2
+types-pytz~=2023.3
 types-requests~=2.28
 types-protobuf~=4.22
 twine~=4.0

From d2bef8672a395b1aba6a673d506f5a7ae6e9a483 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 21 Apr 2023 11:48:10 -0400
Subject: [PATCH 614/860] Update pytz requirement from ~=2023.2 to ~=2023.3
 (#638)

Updates the requirements on [pytz](https://github.com/stub42/pytz) to permit the latest version.
- [Release notes](https://github.com/stub42/pytz/releases)
- [Commits](https://github.com/stub42/pytz/compare/release_2023.2...release_2023.3)

---
updated-dependencies:
- dependency-name: pytz
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index 7ece3c349..1e0d5d8c3 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -23,7 +23,7 @@ pytest-csv~=3.0
 pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
 pytest-xdist~=3.2
-pytz~=2023.2
+pytz~=2023.3
 tox~=3.0;python_version=="3.7"
 tox~=4.4;python_version>="3.8"
 types-pytz~=2023.3

From 5b00086d59b20ce982f96121ea3803f68d81797b Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Mon, 24 Apr 2023 12:36:05 -0400
Subject: [PATCH 615/860] ADAP-382: Address TestAccessGrantSucceeds sporadic
 failure (#674)

* try waiting between consecutive calls to run_dbt to see if it is a concurrency issue
---
 .changes/unreleased/Fixes-20230421-120450.yaml   | 6 ++++++
 tests/functional/adapter/test_grant_access_to.py | 5 +++++
 2 files changed, 11 insertions(+)
 create mode 100644 .changes/unreleased/Fixes-20230421-120450.yaml

diff --git a/.changes/unreleased/Fixes-20230421-120450.yaml b/.changes/unreleased/Fixes-20230421-120450.yaml
new file mode 100644
index 000000000..64cee345f
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230421-120450.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: ' Fix issue of sporadic failure to apply grants during high transaction volumes'
+time: 2023-04-21T12:04:50.391534-04:00
+custom:
+  Author: mikealfare
+  Issue: "614"
diff --git a/tests/functional/adapter/test_grant_access_to.py b/tests/functional/adapter/test_grant_access_to.py
index 89ec5a6c8..109cab125 100644
--- a/tests/functional/adapter/test_grant_access_to.py
+++ b/tests/functional/adapter/test_grant_access_to.py
@@ -1,6 +1,10 @@
+import time
+
 import pytest
+
 from dbt.tests.util import run_dbt
 
+
 SELECT_1 = """
 {{ config(
     materialized='view',
@@ -43,6 +47,7 @@ def test_grant_access_succeeds(self, project):
         # Need to run twice to validate idempotency
         results = run_dbt(["run"])
         assert len(results) == 2
+        time.sleep(10)
         results = run_dbt(["run"])
         assert len(results) == 2
 

From 1972d72b53f897f85188b1218431c6610602bf5c Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 24 Apr 2023 14:15:23 -0400
Subject: [PATCH 616/860] Update pip-tools requirement from ~=6.12 to ~=6.13
 (#675)

Updates the requirements on [pip-tools](https://github.com/jazzband/pip-tools) to permit the latest version.
- [Release notes](https://github.com/jazzband/pip-tools/releases)
- [Changelog](https://github.com/jazzband/pip-tools/blob/main/CHANGELOG.md)
- [Commits](https://github.com/jazzband/pip-tools/compare/6.12.0...6.13.0)

---
updated-dependencies:
- dependency-name: pip-tools
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index 1e0d5d8c3..7d6994a6c 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -14,7 +14,7 @@ flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
 mypy==1.2.0  # patch updates have historically introduced breaking changes
-pip-tools~=6.12
+pip-tools~=6.13
 pre-commit~=2.21;python_version=="3.7"
 pre-commit~=3.2;python_version>="3.8"
 pre-commit-hooks~=4.4

From 7c216445f8009baa9cec4d61dd56693be1dd79fa Mon Sep 17 00:00:00 2001
From: Christophe Oudar <kayrnt@gmail.com>
Date: Wed, 26 Apr 2023 13:14:36 +0200
Subject: [PATCH 617/860] Support all types of data_type using time ingestion
 partitioning (#496)

* Support all types of data_type using time ingestion partitioning

* rework bq_create_table_as & fix partitions

* touchups after verifying no bug

* change case of test field because the parse routine now sanitizes the config val

---------

Co-authored-by: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .../unreleased/Fixes-20230202-010912.yaml     |  8 ++
 dbt/adapters/bigquery/impl.py                 | 74 ++++++++++++++-----
 dbt/include/bigquery/macros/adapters.sql      | 16 ++++
 .../macros/materializations/incremental.sql   | 31 ++++----
 .../incremental_strategy/common.sql           |  9 ---
 .../incremental_strategy/insert_overwrite.sql | 14 ++--
 .../incremental_strategy/merge.sql            |  4 +-
 .../time_ingestion_tables.sql                 | 35 ++-------
 .../incremental_strategy_fixtures.py          | 46 +++++++++++-
 .../test_incremental_strategies.py            | 20 ++++-
 tests/unit/test_bigquery_adapter.py           | 16 ++--
 11 files changed, 179 insertions(+), 94 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230202-010912.yaml

diff --git a/.changes/unreleased/Fixes-20230202-010912.yaml b/.changes/unreleased/Fixes-20230202-010912.yaml
new file mode 100644
index 000000000..d85c2ed84
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230202-010912.yaml
@@ -0,0 +1,8 @@
+kind: Fixes
+body: Support all types of data_type using time ingestion partitioning as previously
+  `date` was failing
+time: 2023-02-02T01:09:12.013631+01:00
+custom:
+  Author: Kayrnt
+  Issue: "486"
+  PR: "496"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index fd99927a2..2bf6ddf84 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -77,9 +77,17 @@ class PartitionConfig(dbtClassMixin):
     time_ingestion_partitioning: bool = False
     copy_partitions: bool = False
 
+    PARTITION_DATE = "_PARTITIONDATE"
+    PARTITION_TIME = "_PARTITIONTIME"
+
     def data_type_for_partition(self):
-        """Return the data type of partitions for replacement."""
-        return self.data_type if not self.time_ingestion_partitioning else "timestamp"
+        """Return the data type of partitions for replacement.
+        When time_ingestion_partitioning is enabled, the data type supported are date & timestamp.
+        """
+        if not self.time_ingestion_partitioning:
+            return self.data_type
+
+        return "date" if self.data_type == "date" else "timestamp"
 
     def reject_partition_field_column(self, columns: List[Any]) -> List[str]:
         return [c for c in columns if not c.name.upper() == self.field.upper()]
@@ -87,12 +95,28 @@ def reject_partition_field_column(self, columns: List[Any]) -> List[str]:
     def data_type_should_be_truncated(self):
         """Return true if the data type should be truncated instead of cast to the data type."""
         return not (
-            self.data_type.lower() == "int64"
-            or (self.data_type.lower() == "date" and self.granularity.lower() == "day")
+            self.data_type == "int64" or (self.data_type == "date" and self.granularity == "day")
         )
 
+    def time_partitioning_field(self) -> str:
+        """Return the time partitioning field name based on the data type.
+        The default is _PARTITIONTIME, but for date it is _PARTITIONDATE
+        else it will fail statements for type mismatch."""
+        if self.data_type == "date":
+            return self.PARTITION_DATE
+        else:
+            return self.PARTITION_TIME
+
+    def insertable_time_partitioning_field(self) -> str:
+        """Return the insertable time partitioning field name based on the data type.
+        Practically, only _PARTITIONTIME works so far.
+        The function is meant to keep the call sites consistent as it might evolve."""
+        return self.PARTITION_TIME
+
     def render(self, alias: Optional[str] = None):
-        column: str = self.field if not self.time_ingestion_partitioning else "_PARTITIONTIME"
+        column: str = (
+            self.field if not self.time_ingestion_partitioning else self.time_partitioning_field()
+        )
         if alias:
             column = f"{alias}.{column}"
 
@@ -107,6 +131,9 @@ def render_wrapped(self, alias: Optional[str] = None):
         if (
             self.data_type in ("date", "timestamp", "datetime")
             and not self.data_type_should_be_truncated()
+            and not (
+                self.time_ingestion_partitioning and self.data_type == "date"
+            )  # _PARTITIONDATE is already a date
         ):
             return f"{self.data_type}({self.render(alias)})"
         else:
@@ -118,7 +145,12 @@ def parse(cls, raw_partition_by) -> Optional["PartitionConfig"]:
             return None
         try:
             cls.validate(raw_partition_by)
-            return cls.from_dict(raw_partition_by)
+            return cls.from_dict(
+                {
+                    key: (value.lower() if isinstance(value, str) else value)
+                    for key, value in raw_partition_by.items()
+                }
+            )
         except ValidationError as exc:
             raise dbt.exceptions.DbtValidationError("Could not parse partition config") from exc
         except TypeError:
@@ -273,9 +305,16 @@ def get_columns_in_relation(self, relation: BigQueryRelation) -> List[BigQueryCo
             return []
 
     @available.parse(lambda *a, **k: [])
-    def add_time_ingestion_partition_column(self, columns) -> List[BigQueryColumn]:
-        "Add time ingestion partition column to columns list"
-        columns.append(self.Column("_PARTITIONTIME", "TIMESTAMP", None, "NULLABLE"))
+    def add_time_ingestion_partition_column(self, partition_by, columns) -> List[BigQueryColumn]:
+        """Add time ingestion partition column to columns list"""
+        columns.append(
+            self.Column(
+                partition_by.insertable_time_partitioning_field(),
+                partition_by.data_type,
+                None,
+                "NULLABLE",
+            )
+        )
         return columns
 
     def expand_column_types(self, goal: BigQueryRelation, current: BigQueryRelation) -> None:  # type: ignore[override]
@@ -564,18 +603,15 @@ def _partitions_match(table, conf_partition: Optional[PartitionConfig]) -> bool:
         if not is_partitioned and not conf_partition:
             return True
         elif conf_partition and table.time_partitioning is not None:
-            partitioning_field = table.time_partitioning.field or "_PARTITIONTIME"
-            table_field = partitioning_field.lower()
-            table_granularity = table.partitioning_type.lower()
-            conf_table_field = (
-                conf_partition.field
-                if not conf_partition.time_ingestion_partitioning
-                else "_PARTITIONTIME"
+            table_field = (
+                table.time_partitioning.field.lower() if table.time_partitioning.field else None
             )
+            table_granularity = table.partitioning_type
+            conf_table_field = conf_partition.field
             return (
-                table_field == conf_table_field.lower()
-                and table_granularity == conf_partition.granularity.lower()
-            )
+                table_field == conf_table_field
+                or (conf_partition.time_ingestion_partitioning and table_field is not None)
+            ) and table_granularity == conf_partition.granularity
         elif conf_partition and table.range_partitioning is not None:
             dest_part = table.range_partitioning
             conf_part = conf_partition.range or {}
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index ed9359bee..23a3f3bf6 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -2,6 +2,8 @@
 {% macro partition_by(partition_config) -%}
     {%- if partition_config is none -%}
       {% do return('') %}
+    {%- elif partition_config.time_ingestion_partitioning -%}
+        partition by {{ partition_config.render_wrapped() }}
     {%- elif partition_config.data_type | lower in ('date','timestamp','datetime') -%}
         partition by {{ partition_config.render() }}
     {%- elif partition_config.data_type | lower in ('int64') -%}
@@ -48,6 +50,11 @@
     {%- set sql_header = config.get('sql_header', none) -%}
 
     {%- set partition_config = adapter.parse_partition_by(raw_partition_by) -%}
+    {%- if partition_config.time_ingestion_partitioning -%}
+    {%- set columns = get_columns_with_types_in_query_sql(sql) -%}
+    {%- set table_dest_columns_csv = columns_without_partition_fields_csv(partition_config, columns) -%}
+    {%- set columns = '(' ~ table_dest_columns_csv ~ ')' -%}
+    {%- endif -%}
 
     {{ sql_header if sql_header is not none }}
 
@@ -57,14 +64,23 @@
         {{ get_assert_columns_equivalent(compiled_code) }}
         {{ get_table_columns_and_constraints() }}
         {%- set compiled_code = get_select_subquery(compiled_code) %}
+      {% else %}
+        {#-- cannot do contracts at the same time as time ingestion partitioning -#}
+        {{ columns }}
       {% endif %}
     {{ partition_by(partition_config) }}
     {{ cluster_by(raw_cluster_by) }}
 
     {{ bigquery_table_options(config, model, temporary) }}
+
+    {#-- PARTITION BY cannot be used with the AS query_statement clause.
+         https://cloud.google.com/bigquery/docs/reference/standard-sql/data-definition-language#partition_expression
+    -#}
+    {%- if not partition_config.time_ingestion_partitioning %}
     as (
       {{ compiled_code }}
     );
+    {%- endif %}
   {%- elif language == 'python' -%}
     {#--
     N.B. Python models _can_ write to temp views HOWEVER they use a different session
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index 9f1479749..2cbb14d9b 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -16,24 +16,25 @@
 {% macro source_sql_with_partition(partition_by, source_sql) %}
 
   {%- if partition_by.time_ingestion_partitioning %}
-    {{ return(wrap_with_time_ingestion_partitioning_sql(build_partition_time_exp(partition_by.field), source_sql, False))  }}
+    {{ return(wrap_with_time_ingestion_partitioning_sql(partition_by, source_sql, False))  }}
   {% else %}
     {{ return(source_sql)  }}
   {%- endif -%}
 
 {% endmacro %}
-{% macro bq_create_table_as(is_time_ingestion_partitioning, temporary, relation, compiled_code, language='sql') %}
-  {% if is_time_ingestion_partitioning and language == 'python' %}
+
+{% macro bq_create_table_as(partition_by, temporary, relation, compiled_code, language='sql') %}
+  {%- set _dbt_max_partition = declare_dbt_max_partition(this, partition_by, compiled_code, language) -%}
+  {% if partition_by.time_ingestion_partitioning and language == 'python' %}
     {% do exceptions.raise_compiler_error(
       "Python models do not support ingestion time partitioning"
     ) %}
-  {% endif %}
-  {% if is_time_ingestion_partitioning and language == 'sql' %}
+  {% elif partition_by.time_ingestion_partitioning and language == 'sql' %}
     {#-- Create the table before inserting data as ingestion time partitioned tables can't be created with the transformed data --#}
-    {% do run_query(create_ingestion_time_partitioned_table_as_sql(temporary, relation, compiled_code)) %}
-    {{ return(bq_insert_into_ingestion_time_partitioned_table_sql(relation, compiled_code)) }}
+    {% do run_query(create_table_as(temporary, relation, compiled_code)) %}
+    {{ return(_dbt_max_partition + bq_insert_into_ingestion_time_partitioned_table_sql(relation, compiled_code)) }}
   {% else %}
-    {{ return(create_table_as(temporary, relation, compiled_code, language)) }}
+    {{ return(_dbt_max_partition + create_table_as(temporary, relation, compiled_code, language)) }}
   {% endif %}
 {% endmacro %}
 
@@ -93,14 +94,14 @@
 
   {% elif existing_relation is none %}
       {%- call statement('main', language=language) -%}
-        {{ bq_create_table_as(partition_by.time_ingestion_partitioning, False, target_relation, compiled_code, language) }}
+        {{ bq_create_table_as(partition_by, False, target_relation, compiled_code, language) }}
       {%- endcall -%}
 
   {% elif existing_relation.is_view %}
       {#-- There's no way to atomically replace a view with a table on BQ --#}
       {{ adapter.drop_relation(existing_relation) }}
       {%- call statement('main', language=language) -%}
-        {{ bq_create_table_as(partition_by.time_ingestion_partitioning, False, target_relation, compiled_code, language) }}
+        {{ bq_create_table_as(partition_by, False, target_relation, compiled_code, language) }}
       {%- endcall -%}
 
   {% elif full_refresh_mode %}
@@ -110,7 +111,7 @@
           {{ adapter.drop_relation(existing_relation) }}
       {% endif %}
       {%- call statement('main', language=language) -%}
-        {{ bq_create_table_as(partition_by.time_ingestion_partitioning, False, target_relation, compiled_code, language) }}
+        {{ bq_create_table_as(partition_by, False, target_relation, compiled_code, language) }}
       {%- endcall -%}
 
   {% else %}
@@ -127,9 +128,7 @@
       {#-- Check first, since otherwise we may not build a temp table --#}
       {#-- Python always needs to create a temp table --#}
       {%- call statement('create_tmp_relation', language=language) -%}
-        {{ declare_dbt_max_partition(this, partition_by, compiled_code, language) +
-           bq_create_table_as(partition_by.time_ingestion_partitioning, True, tmp_relation, compiled_code, language)
-        }}
+        {{ bq_create_table_as(partition_by, True, tmp_relation, compiled_code, language) }}
       {%- endcall -%}
       {% set tmp_relation_exists = true %}
       {#-- Process schema changes. Returns dict of changes if successful. Use source columns for upserting/merging --#}
@@ -139,9 +138,11 @@
     {% if not dest_columns %}
       {% set dest_columns = adapter.get_columns_in_relation(existing_relation) %}
     {% endif %}
+    {#--  Add time ingestion pseudo column to destination column as not part of the 'schema' but still need it for actual data insertion --#}
     {% if partition_by.time_ingestion_partitioning %}
-      {% set dest_columns = adapter.add_time_ingestion_partition_column(dest_columns) %}
+      {% set dest_columns = adapter.add_time_ingestion_partition_column(partition_by, dest_columns) %}
     {% endif %}
+
     {% set build_sql = bq_generate_incremental_build_sql(
         strategy, tmp_relation, target_relation, compiled_code, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, partition_by.copy_partitions, incremental_predicates
     ) %}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
index b9f8560d9..9d71ba7c0 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
@@ -1,12 +1,3 @@
-{% macro build_partition_time_exp(partition_by) %}
-  {% if partition_by.data_type == 'timestamp' %}
-    {% set partition_value = partition_by.field %}
-  {% else %}
-    {% set partition_value = 'timestamp(' + partition_by.field + ')' %}
-  {% endif %}
-  {{ return({'value': partition_value, 'field': partition_by.field}) }}
-{% endmacro %}
-
 {% macro declare_dbt_max_partition(relation, partition_by, compiled_code, language='sql') %}
 
   {#-- TODO: revisit partitioning with python models --#}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
index 3153f49d0..41c11c15d 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
@@ -58,7 +58,7 @@
       {%- set source_sql -%}
         (
           {%- if partition_by.time_ingestion_partitioning -%}
-          {{ wrap_with_time_ingestion_partitioning_sql(build_partition_time_exp(partition_by), sql, True) }}
+          {{ wrap_with_time_ingestion_partitioning_sql(partition_by, sql, True) }}
           {%- else -%}
           {{sql}}
           {%- endif -%}
@@ -85,8 +85,7 @@
   ) %}
   {# We run temp table creation in a separated script to move to partitions copy #}
   {%- call statement('create_tmp_relation_for_copy', language='sql') -%}
-    {{ declare_dbt_max_partition(this, partition_by, sql, 'sql') +
-     bq_create_table_as(partition_by.time_ingestion_partitioning, True, tmp_relation, sql, 'sql')
+    {{ bq_create_table_as(partition_by, True, tmp_relation, sql, 'sql')
   }}
   {%- endcall %}
   {%- set partitions_sql -%}
@@ -112,7 +111,7 @@
       (
         select
         {% if partition_by.time_ingestion_partitioning -%}
-        _PARTITIONTIME,
+        {{ partition_by.insertable_time_partitioning_field() }},
         {%- endif -%}
         * from {{ tmp_relation }}
       )
@@ -123,19 +122,18 @@
 
       {# have we already created the temp table to check for schema changes? #}
       {% if not tmp_relation_exists %}
-        {{ declare_dbt_max_partition(this, partition_by, sql) }}
-
        -- 1. create a temp table with model data
-        {{ bq_create_table_as(partition_by.time_ingestion_partitioning, True, tmp_relation, sql, 'sql') }}
+        {{ bq_create_table_as(partition_by, True, tmp_relation, sql, 'sql') }}
       {% else %}
         -- 1. temp table already exists, we used it to check for schema changes
       {% endif %}
+      {%- set partition_field = partition_by.time_partitioning_field() if partition_by.time_ingestion_partitioning else partition_by.render_wrapped() -%}
 
       -- 2. define partitions to update
       set (dbt_partitions_for_replacement) = (
           select as struct
               -- IGNORE NULLS: this needs to be aligned to _dbt_max_partition, which ignores null
-              array_agg(distinct {{ partition_by.render_wrapped() }} IGNORE NULLS)
+              array_agg(distinct {{ partition_field }} IGNORE NULLS)
           from {{ tmp_relation }}
       );
 
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
index 57c88dbc8..90af66f52 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
@@ -6,14 +6,14 @@
         (
         select
         {% if partition_by.time_ingestion_partitioning -%}
-        _PARTITIONTIME,
+        {{ partition_by.insertable_time_partitioning_field() }},
         {%- endif -%}
         * from {{ tmp_relation }}
         )
         {%- else -%} {#-- wrap sql in parens to make it a subquery --#}
         (
             {%- if partition_by.time_ingestion_partitioning -%}
-            {{ wrap_with_time_ingestion_partitioning_sql(build_partition_time_exp(partition_by), sql, True) }}
+            {{ wrap_with_time_ingestion_partitioning_sql(partition_by, sql, True) }}
             {%- else -%}
             {{sql}}
             {%- endif %}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
index 79d6a74eb..0a118dab6 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
@@ -1,35 +1,11 @@
-{% macro wrap_with_time_ingestion_partitioning_sql(partition_time_exp, sql, is_nested) %}
+{% macro wrap_with_time_ingestion_partitioning_sql(partition_by, sql, is_nested) %}
 
-  select {{ partition_time_exp['value'] }} as _partitiontime, * EXCEPT({{ partition_time_exp['field'] }}) from (
+  select TIMESTAMP({{ partition_by.field }}) as {{ partition_by.insertable_time_partitioning_field() }}, * EXCEPT({{ partition_by.field }}) from (
     {{ sql }}
   ){%- if not is_nested -%};{%- endif -%}
 
 {% endmacro %}
 
-{% macro create_ingestion_time_partitioned_table_as_sql(temporary, relation, sql) -%}
-  {%- set raw_partition_by = config.get('partition_by', none) -%}
-  {%- set raw_cluster_by = config.get('cluster_by', none) -%}
-  {%- set sql_header = config.get('sql_header', none) -%}
-
-  {%- set partition_config = adapter.parse_partition_by(raw_partition_by) -%}
-
-  {%- set columns = get_columns_with_types_in_query_sql(sql) -%}
-  {%- set table_dest_columns_csv = columns_without_partition_fields_csv(partition_config, columns) -%}
-
-  {{ sql_header if sql_header is not none }}
-
-  {% set ingestion_time_partition_config_raw = fromjson(tojson(raw_partition_by)) %}
-  {% do ingestion_time_partition_config_raw.update({'field':'_PARTITIONTIME'}) %}
-
-  {%- set ingestion_time_partition_config = adapter.parse_partition_by(ingestion_time_partition_config_raw) -%}
-
-  create or replace table {{ relation }} ({{table_dest_columns_csv}})
-  {{ partition_by(ingestion_time_partition_config) }}
-  {{ cluster_by(raw_cluster_by) }}
-  {{ bigquery_table_options(config, model, temporary) }}
-
-{%- endmacro -%}
-
 {% macro get_quoted_with_types_csv(columns) %}
     {% set quoted = [] %}
     {% for col in columns -%}
@@ -48,12 +24,13 @@
 {%- endmacro -%}
 
 {% macro bq_insert_into_ingestion_time_partitioned_table_sql(target_relation, sql) -%}
-  {%- set partition_by = config.get('partition_by', none) -%}
+  {%- set raw_partition_by = config.get('partition_by', none) -%}
+  {%- set partition_by = adapter.parse_partition_by(raw_partition_by) -%}
   {% set dest_columns = adapter.get_columns_in_relation(target_relation) %}
   {%- set dest_columns_csv = get_quoted_csv(dest_columns | map(attribute="name")) -%}
 
-  insert into {{ target_relation }} (_partitiontime, {{ dest_columns_csv }})
-    {{ wrap_with_time_ingestion_partitioning_sql(build_partition_time_exp(partition_by), sql, False) }}
+  insert into {{ target_relation }} ({{ partition_by.insertable_time_partitioning_field() }}, {{ dest_columns_csv }})
+    {{ wrap_with_time_ingestion_partitioning_sql(partition_by, sql, False) }}
 
 {%- endmacro -%}
 
diff --git a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
index b3d45ae80..6bb429833 100644
--- a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
+++ b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
@@ -54,7 +54,7 @@
         cluster_by="id",
         partition_by={
             "field": "date_time",
-            "data_type": "datetime"
+            "data_type": "dateTime"
         }
     )
 }}
@@ -220,6 +220,50 @@
 {% endif %}
 """.lstrip()
 
+overwrite_day_with_time_partition_datetime_sql = """
+{{
+    config(
+        materialized="incremental",
+        incremental_strategy='insert_overwrite',
+        cluster_by="id",
+        partition_by={
+            "field": "date_day",
+            "data_type": "date",
+            "time_ingestion_partitioning": true
+        }
+    )
+}}
+
+
+with data as (
+
+    {% if not is_incremental() %}
+
+        select 1 as id, cast('2020-01-01' as date) as date_day union all
+        select 2 as id, cast('2020-01-01' as date) as date_day union all
+        select 3 as id, cast('2020-01-01' as date) as date_day union all
+        select 4 as id, cast('2020-01-01' as date) as date_day
+
+    {% else %}
+
+        -- we want to overwrite the 4 records in the 2020-01-01 partition
+        -- with the 2 records below, but add two more in the 2020-01-02 partition
+        select 10 as id, cast('2020-01-01' as date) as date_day union all
+        select 20 as id, cast('2020-01-01' as date) as date_day union all
+        select 30 as id, cast('2020-01-02' as date) as date_day union all
+        select 40 as id, cast('2020-01-02' as date) as date_day
+
+    {% endif %}
+
+)
+
+select * from data
+
+{% if is_incremental() %}
+where date_day >= '2020-01-01'
+{% endif %}
+""".lstrip()
+
 overwrite_partitions_sql = """
 {{
     config(
diff --git a/tests/functional/adapter/incremental/test_incremental_strategies.py b/tests/functional/adapter/incremental/test_incremental_strategies.py
index a49872310..8a90b98ab 100644
--- a/tests/functional/adapter/incremental/test_incremental_strategies.py
+++ b/tests/functional/adapter/incremental/test_incremental_strategies.py
@@ -24,6 +24,7 @@
     overwrite_range_sql,
     overwrite_time_sql,
     overwrite_day_with_time_ingestion_sql,
+    overwrite_day_with_time_partition_datetime_sql,
 )
 
 
@@ -43,7 +44,8 @@ def models(self):
             "incremental_overwrite_partitions.sql": overwrite_partitions_sql,
             "incremental_overwrite_range.sql": overwrite_range_sql,
             "incremental_overwrite_time.sql": overwrite_time_sql,
-            "incremental_overwrite_day_with_time_ingestion.sql": overwrite_day_with_time_ingestion_sql,
+            "incremental_overwrite_day_with_time_partition.sql": overwrite_day_with_time_ingestion_sql,
+            "incremental_overwrite_day_with_time_partition_datetime.sql": overwrite_day_with_time_partition_datetime_sql,
         }
 
     @pytest.fixture(scope="class")
@@ -61,10 +63,10 @@ def seeds(self):
     def test__bigquery_assert_incremental_configurations_apply_the_right_strategy(self, project):
         run_dbt(["seed"])
         results = run_dbt()
-        assert len(results) == 9
+        assert len(results) == 10
 
         results = run_dbt()
-        assert len(results) == 9
+        assert len(results) == 10
         incremental_strategies = [
             ("incremental_merge_range", "merge_expected"),
             ("incremental_merge_time", "merge_expected"),
@@ -73,6 +75,10 @@ def test__bigquery_assert_incremental_configurations_apply_the_right_strategy(se
             ("incremental_overwrite_partitions", "incremental_overwrite_date_expected"),
             ("incremental_overwrite_day", "incremental_overwrite_day_expected"),
             ("incremental_overwrite_range", "incremental_overwrite_range_expected"),
+            (
+                "incremental_overwrite_day_with_time_partition_datetime",
+                "incremental_overwrite_day_with_time_partition_expected",
+            ),
         ]
         db_with_schema = f"{project.database}.{project.test_schema}"
         for incremental_strategy in incremental_strategies:
@@ -87,3 +93,11 @@ def test__bigquery_assert_incremental_configurations_apply_the_right_strategy(se
         )
         expected = get_relation_columns(project.adapter, "incremental_overwrite_day_expected")
         assert created == expected
+
+        created = get_relation_columns(
+            project.adapter, "incremental_overwrite_day_with_time_partition"
+        )
+        expected = get_relation_columns(
+            project.adapter, "incremental_overwrite_day_with_time_partition_expected"
+        )
+        assert created == expected
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 4dece85a6..56b8e07d7 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -752,7 +752,7 @@ def test_parse_partition_by(self):
             {
                 "field": "ts",
                 "data_type": "date",
-                "granularity": "MONTH",
+                "granularity": "month",
                 "time_ingestion_partitioning": False,
                 "copy_partitions": False,
             },
@@ -765,7 +765,7 @@ def test_parse_partition_by(self):
             {
                 "field": "ts",
                 "data_type": "date",
-                "granularity": "YEAR",
+                "granularity": "year",
                 "time_ingestion_partitioning": False,
                 "copy_partitions": False,
             },
@@ -778,7 +778,7 @@ def test_parse_partition_by(self):
             {
                 "field": "ts",
                 "data_type": "timestamp",
-                "granularity": "HOUR",
+                "granularity": "hour",
                 "time_ingestion_partitioning": False,
                 "copy_partitions": False,
             },
@@ -791,7 +791,7 @@ def test_parse_partition_by(self):
             {
                 "field": "ts",
                 "data_type": "timestamp",
-                "granularity": "MONTH",
+                "granularity": "month",
                 "time_ingestion_partitioning": False,
                 "copy_partitions": False,
             },
@@ -804,7 +804,7 @@ def test_parse_partition_by(self):
             {
                 "field": "ts",
                 "data_type": "timestamp",
-                "granularity": "YEAR",
+                "granularity": "year",
                 "time_ingestion_partitioning": False,
                 "copy_partitions": False,
             },
@@ -817,7 +817,7 @@ def test_parse_partition_by(self):
             {
                 "field": "ts",
                 "data_type": "datetime",
-                "granularity": "HOUR",
+                "granularity": "hour",
                 "time_ingestion_partitioning": False,
                 "copy_partitions": False,
             },
@@ -830,7 +830,7 @@ def test_parse_partition_by(self):
             {
                 "field": "ts",
                 "data_type": "datetime",
-                "granularity": "MONTH",
+                "granularity": "month",
                 "time_ingestion_partitioning": False,
                 "copy_partitions": False,
             },
@@ -843,7 +843,7 @@ def test_parse_partition_by(self):
             {
                 "field": "ts",
                 "data_type": "datetime",
-                "granularity": "YEAR",
+                "granularity": "year",
                 "time_ingestion_partitioning": False,
                 "copy_partitions": False,
             },

From 6f08ce1ca3c243f63af77eda66c4420b2ac02207 Mon Sep 17 00:00:00 2001
From: Christophe Oudar <kayrnt@gmail.com>
Date: Fri, 28 Apr 2023 18:18:51 +0200
Subject: [PATCH 618/860] Fix UDF usage with time ingestion (#683)

---
 .changes/unreleased/Fixes-20230427-141957.yaml |  6 ++++++
 .../time_ingestion_tables.sql                  |  4 ++++
 .../incremental_strategy_fixtures.py           | 18 ++++++++++++------
 3 files changed, 22 insertions(+), 6 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230427-141957.yaml

diff --git a/.changes/unreleased/Fixes-20230427-141957.yaml b/.changes/unreleased/Fixes-20230427-141957.yaml
new file mode 100644
index 000000000..7b5bf598d
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230427-141957.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: 'Fix UDF usage with time ingestion'
+time: 2023-04-27T14:19:57.518037+02:00
+custom:
+  Author: Kayrnt
+  Issue: "684"
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
index 0a118dab6..e44a6a94f 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
@@ -24,6 +24,8 @@
 {%- endmacro -%}
 
 {% macro bq_insert_into_ingestion_time_partitioned_table_sql(target_relation, sql) -%}
+  {%- set sql_header = config.get('sql_header', none) -%}
+  {{ sql_header if sql_header is not none }}
   {%- set raw_partition_by = config.get('partition_by', none) -%}
   {%- set partition_by = adapter.parse_partition_by(raw_partition_by) -%}
   {% set dest_columns = adapter.get_columns_in_relation(target_relation) %}
@@ -36,6 +38,8 @@
 
 {% macro get_columns_with_types_in_query_sql(select_sql) %}
   {% set sql %}
+    {%- set sql_header = config.get('sql_header', none) -%}
+    {{ sql_header if sql_header is not none }}
     select * from (
       {{ select_sql }}
     ) as __dbt_sbq
diff --git a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
index 6bb429833..162ede883 100644
--- a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
+++ b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
@@ -416,21 +416,27 @@
 }}
 
 
+{%- call set_sql_header(config) %}
+ CREATE TEMP FUNCTION asDateTime(date STRING) AS (
+   cast(date as datetime)
+ );
+{%- endcall %}
+
 with data as (
 
     {% if not is_incremental() %}
 
-        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 4 as id, cast('2020-01-01' as datetime) as date_time
+        select 1 as id, asDateTime('2020-01-01') as date_time union all
+        select 2 as id, asDateTime('2020-01-01') as date_time union all
+        select 3 as id, asDateTime('2020-01-01') as date_time union all
+        select 4 as id, asDateTime('2020-01-01') as date_time
 
     {% else %}
 
         -- we want to overwrite the 4 records in the 2020-01-01 partition
         -- with the 2 records below, but add two more in the 2020-01-02 partition
-        select 10 as id, cast('2020-01-01' as datetime) as date_time union all
-        select 20 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 10 as id, asDateTime('2020-01-01') as date_time union all
+        select 20 as id, asDateTime('2020-01-01') as date_time union all
         select 30 as id, cast('2020-01-02' as datetime) as date_time union all
         select 40 as id, cast('2020-01-02' as datetime) as date_time
 

From 971f48ccc054084a6bbdce2d9f04a8cd9e6070c6 Mon Sep 17 00:00:00 2001
From: Daniel Green <43223669+dgreen161@users.noreply.github.com>
Date: Tue, 2 May 2023 23:09:26 +0100
Subject: [PATCH 619/860] Fixes adding policy tags when a struct is defined in
 the yml (#688)

* Fixes adding policy tags when a struct is defined in the yml

* Fix changie entry

* Update delete_column_policy test for structs
---
 .changes/unreleased/Fixes-20230428-232904.yaml |  6 ++++++
 dbt/adapters/bigquery/impl.py                  |  3 ++-
 tests/functional/test_delete_column_policy.py  | 10 +++++++---
 3 files changed, 15 insertions(+), 4 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230428-232904.yaml

diff --git a/.changes/unreleased/Fixes-20230428-232904.yaml b/.changes/unreleased/Fixes-20230428-232904.yaml
new file mode 100644
index 000000000..db9a5e20c
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230428-232904.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fixes adding policy tags when a struct is defined in the yml
+time: 2023-04-28T23:29:04.08966+01:00
+custom:
+  Author: dgreen161
+  Issue: "687"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 2bf6ddf84..cad7d0072 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -693,7 +693,8 @@ def _update_column_dict(self, bq_column_dict, dbt_columns, parent=""):
         if dotted_column_name in dbt_columns:
             column_config = dbt_columns[dotted_column_name]
             bq_column_dict["description"] = column_config.get("description")
-            bq_column_dict["policyTags"] = {"names": column_config.get("policy_tags", list())}
+            if bq_column_dict["type"] != "RECORD":
+                bq_column_dict["policyTags"] = {"names": column_config.get("policy_tags", list())}
 
         new_fields = []
         for child_col_dict in bq_column_dict.get("fields", list()):
diff --git a/tests/functional/test_delete_column_policy.py b/tests/functional/test_delete_column_policy.py
index ae2c6bba7..73b1c1f28 100644
--- a/tests/functional/test_delete_column_policy.py
+++ b/tests/functional/test_delete_column_policy.py
@@ -11,7 +11,9 @@
 }}
 
 select
-  1 field
+  struct(
+    1 as field
+  ) as first_struct
 """
 
 _POLICY_TAG_YML = """version: 2
@@ -19,7 +21,8 @@
 models:
 - name: policy_tag_table
   columns:
-  - name: field
+  - name: first_struct
+  - name: first_struct.field
     policy_tags:
       - '{{ var("policy_tag") }}'
 """
@@ -29,7 +32,8 @@
 models:
 - name: policy_tag_table
   columns:
-  - name: field
+  - name: first_struct
+  - name: first_struct.field
 """
 
 # Manually generated https://console.cloud.google.com/bigquery/policy-tags?project=dbt-test-env

From 237cdc48fadc3c143fd852eb0e06ba7856827f34 Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Tue, 2 May 2023 23:23:26 -0700
Subject: [PATCH 620/860] Now that pr 688 has been merged, these old tests
 work. (#694)

Stemming from a lack of documentation, I didn't realize the patch 400 error was happening because policy tags were being applied (but only an empty list of them!) to non leaf fields, namely the inner struct (which itself has an inner struct with leaf fields). In pr 542, this bug was introduced -- but why did it initially pass CI? That's a mystery I have no answer for: Guess is a cloud change that exposed this or a BQ spec change.

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 tests/functional/adapter/test_persist_docs.py | 19 +++++++++++++++++++
 1 file changed, 19 insertions(+)

diff --git a/tests/functional/adapter/test_persist_docs.py b/tests/functional/adapter/test_persist_docs.py
index 2e32b200d..d2c7509f5 100644
--- a/tests/functional/adapter/test_persist_docs.py
+++ b/tests/functional/adapter/test_persist_docs.py
@@ -36,10 +36,18 @@
 models:
   - name: table_model_nested
     columns:
+      - name: level_1
+        description: level_1 column description
+      - name: level_1.level_2
+        description: level_2 column description
       - name: level_1.level_2.level_3_a
         description: level_3 column description
   - name: view_model_nested
     columns:
+      - name: level_1
+        description: level_1 column description
+      - name: level_1.level_2
+        description: level_2 column description
       - name: level_1.level_2.level_3_a
         description: level_3 column description
 """
@@ -183,6 +191,11 @@ def test_persist_docs(self, project):
                 bq_schema = client.get_table(table_id).schema
 
                 level_1_field = bq_schema[0]
+                assert level_1_field.description == "level_1 column description"
+
+                level_2_field = level_1_field.fields[0]
+                assert level_2_field.description == "level_2 column description"
+
                 level_2_field = level_1_field.fields[0]
                 level_3_field = level_2_field.fields[0]
                 assert level_3_field.description == "level_3 column description"
@@ -190,5 +203,11 @@ def test_persist_docs(self, project):
             # check the descriptions in the catalog
             node = catalog_data["nodes"]["model.test.{}".format(node_id)]
 
+            level_1_column = node["columns"]["level_1"]
+            assert level_1_column["comment"] == "level_1 column description"
+
+            level_2_column = node["columns"]["level_1.level_2"]
+            assert level_2_column["comment"] == "level_2 column description"
+
             level_3_column = node["columns"]["level_1.level_2.level_3_a"]
             assert level_3_column["comment"] == "level_3 column description"

From f717e83927e5ac71c4a085985ecc95e7c0c70099 Mon Sep 17 00:00:00 2001
From: David Bloss <david.bloss@dbtlabs.com>
Date: Fri, 5 May 2023 14:03:17 -0500
Subject: [PATCH 621/860] update used gh actions ahead of set-output, node12
 deprecation (#695)

---
 .github/workflows/integration.yml          | 14 +++++++-------
 .github/workflows/main.yml                 | 12 ++++++------
 .github/workflows/release-branch-tests.yml |  2 +-
 3 files changed, 14 insertions(+), 14 deletions(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 7be34358f..7a6890566 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -63,13 +63,13 @@ jobs:
     steps:
       - name: Check out the repository (non-PR)
         if: github.event_name != 'pull_request_target'
-        uses: actions/checkout@v2
+        uses: actions/checkout@v3
         with:
           persist-credentials: false
 
       - name: Check out the repository (PR)
         if: github.event_name == 'pull_request_target'
-        uses: actions/checkout@v2
+        uses: actions/checkout@v3
         with:
           persist-credentials: false
           ref: ${{ github.event.pull_request.head.sha }}
@@ -93,7 +93,7 @@ jobs:
               - 'dev-requirements.txt'
       - name: Generate integration test matrix
         id: generate-matrix
-        uses: actions/github-script@v4
+        uses: actions/github-script@v6
         env:
           CHANGES: ${{ steps.get-changes.outputs.changes }}
         with:
@@ -131,7 +131,7 @@ jobs:
     steps:
       - name: Check out the repository
         if: github.event_name != 'pull_request_target'
-        uses: actions/checkout@v2
+        uses: actions/checkout@v3
         with:
           persist-credentials: false
 
@@ -139,7 +139,7 @@ jobs:
       # this is necessary for the `pull_request_target` event
       - name: Check out the repository (PR)
         if: github.event_name == 'pull_request_target'
-        uses: actions/checkout@v2
+        uses: actions/checkout@v3
         with:
           persist-credentials: false
           ref: ${{ github.event.pull_request.head.sha }}
@@ -174,7 +174,7 @@ jobs:
           GCS_BUCKET: dbt-ci
         run: tox
 
-      - uses: actions/upload-artifact@v2
+      - uses: actions/upload-artifact@v3
         if: always()
         with:
           name: logs
@@ -185,7 +185,7 @@ jobs:
         id: date
         run: echo "::set-output name=date::$(date +'%Y-%m-%dT%H_%M_%S')" #no colons allowed for artifacts
 
-      - uses: actions/upload-artifact@v2
+      - uses: actions/upload-artifact@v3
         if: always()
         with:
           name: integration_results_${{ matrix.python-version }}_${{ matrix.os }}_${{ matrix.adapter }}-${{ steps.date.outputs.date }}.csv
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 140557beb..cc499fa28 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -43,7 +43,7 @@ jobs:
 
     steps:
       - name: Check out the repository
-        uses: actions/checkout@v2
+        uses: actions/checkout@v3
         with:
           persist-credentials: false
 
@@ -79,7 +79,7 @@ jobs:
 
     steps:
       - name: Check out the repository
-        uses: actions/checkout@v2
+        uses: actions/checkout@v3
         with:
           persist-credentials: false
 
@@ -105,7 +105,7 @@ jobs:
         id: date
         run: echo "::set-output name=date::$(date +'%Y-%m-%dT%H_%M_%S')" #no colons allowed for artifacts
 
-      - uses: actions/upload-artifact@v2
+      - uses: actions/upload-artifact@v3
         if: always()
         with:
           name: unit_results_${{ matrix.python-version }}-${{ steps.date.outputs.date }}.csv
@@ -121,7 +121,7 @@ jobs:
 
     steps:
       - name: Check out the repository
-        uses: actions/checkout@v2
+        uses: actions/checkout@v3
         with:
           persist-credentials: false
 
@@ -157,7 +157,7 @@ jobs:
           if [[ "$(ls -lh dist/)" == *"a1"* ]]; then export is_alpha=1; fi
           echo "::set-output name=is_alpha::$is_alpha"
 
-      - uses: actions/upload-artifact@v2
+      - uses: actions/upload-artifact@v3
         with:
           name: dist
           path: dist/
@@ -187,7 +187,7 @@ jobs:
           python -m pip install --user --upgrade pip
           python -m pip install --upgrade wheel setuptools twine check-wheel-contents
           python -m pip --version
-      - uses: actions/download-artifact@v2
+      - uses: actions/download-artifact@v3
         with:
           name: dist
           path: dist/
diff --git a/.github/workflows/release-branch-tests.yml b/.github/workflows/release-branch-tests.yml
index 282efdf2c..2297d979e 100644
--- a/.github/workflows/release-branch-tests.yml
+++ b/.github/workflows/release-branch-tests.yml
@@ -53,7 +53,7 @@ jobs:
         token: ${{ secrets.FISHTOWN_BOT_PAT }}
 
     - name: Post failure to Slack
-      uses: ravsamhq/notify-slack-action@v1
+      uses: ravsamhq/notify-slack-action@v2
       if: ${{ always() && !contains(steps.trigger-step.outputs.workflow-conclusion,'success') }}
       with:
         status: ${{ job.status }}

From 043b2ea1e851b716df2ddc59f619fa1b67106c24 Mon Sep 17 00:00:00 2001
From: Kshitij Aranke <equal.lamp5206@aranke.org>
Date: Wed, 10 May 2023 08:52:27 -0700
Subject: [PATCH 622/860] Add limit to execute signature for consistency with
 dbt-core (#707)

* Add limit to execute signature for consistency with dbt-core

* add changie
---
 .changes/unreleased/Fixes-20230509-222705.yaml |  6 ++++++
 dbt/adapters/bigquery/connections.py           | 16 +++++++++-------
 2 files changed, 15 insertions(+), 7 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230509-222705.yaml

diff --git a/.changes/unreleased/Fixes-20230509-222705.yaml b/.changes/unreleased/Fixes-20230509-222705.yaml
new file mode 100644
index 000000000..f86e3d245
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230509-222705.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Update signature for execute method
+time: 2023-05-09T22:27:05.976611-07:00
+custom:
+  Author: aranke
+  Issue: ' '
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index a561eb5b9..47acceda9 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -410,7 +410,7 @@ def get_table_from_response(cls, resp):
         column_names = [field.name for field in resp.schema]
         return agate_helper.table_from_data_flat(resp, column_names)
 
-    def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False):
+    def raw_execute(self, sql, use_legacy_sql=False, limit: Optional[int] = None):
         conn = self.get_thread_connection()
         client = conn.handle
 
@@ -420,8 +420,8 @@ def raw_execute(self, sql, fetch=False, *, use_legacy_sql=False):
             and self.profile.query_comment
             and self.profile.query_comment.job_label
         ):
-            query_comment = self.query_header.comment.query_comment
-            labels = self._labels_from_query_comment(query_comment)
+            query_comment = self.profile.query_comment
+            labels = self._labels_from_query_comment(query_comment.comment)
         else:
             labels = {}
 
@@ -450,6 +450,7 @@ def fn():
                 job_params,
                 job_creation_timeout=job_creation_timeout,
                 job_execution_timeout=job_execution_timeout,
+                limit=limit,
             )
 
         query_job, iterator = self._retry_and_handle(msg=sql, conn=conn, fn=fn)
@@ -457,11 +458,11 @@ def fn():
         return query_job, iterator
 
     def execute(
-        self, sql, auto_begin=False, fetch=None
+        self, sql, auto_begin=False, fetch=None, limit: Optional[int] = None
     ) -> Tuple[BigQueryAdapterResponse, agate.Table]:
         sql = self._add_query_comment(sql)
         # auto_begin is ignored on bigquery, and only included for consistency
-        query_job, iterator = self.raw_execute(sql, fetch=fetch)
+        query_job, iterator = self.raw_execute(sql, limit=limit)
 
         if fetch:
             table = self.get_table_from_response(iterator)
@@ -550,7 +551,7 @@ def standard_to_legacy(table):
 
         sql = self._add_query_comment(legacy_sql)
         # auto_begin is ignored on bigquery, and only included for consistency
-        _, iterator = self.raw_execute(sql, fetch="fetch_result", use_legacy_sql=True)
+        _, iterator = self.raw_execute(sql, use_legacy_sql=True)
         return self.get_table_from_response(iterator)
 
     def copy_bq_table(self, source, destination, write_disposition):
@@ -644,12 +645,13 @@ def _query_and_results(
         job_params,
         job_creation_timeout=None,
         job_execution_timeout=None,
+        limit: Optional[int] = None,
     ):
         """Query the client and wait for results."""
         # Cannot reuse job_config if destination is set and ddl is used
         job_config = google.cloud.bigquery.QueryJobConfig(**job_params)
         query_job = client.query(query=sql, job_config=job_config, timeout=job_creation_timeout)
-        iterator = query_job.result(timeout=job_execution_timeout)
+        iterator = query_job.result(max_results=limit, timeout=job_execution_timeout)
 
         return query_job, iterator
 

From 6402e3a430dbcd4a99ebfe3c4956166c87d0c61d Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Fri, 12 May 2023 19:11:20 +0000
Subject: [PATCH 623/860] Bumping version to 1.6.0b1 and generate changelog

---
 .bumpversion.cfg                                  |  2 +-
 .changes/1.6.0-b1.md                              | 13 +++++++++++++
 .../Fixes-20230202-010912.yaml                    |  0
 .../Fixes-20230421-120450.yaml                    |  0
 .../Fixes-20230427-141957.yaml                    |  0
 .../Fixes-20230428-232904.yaml                    |  0
 .../Fixes-20230509-222705.yaml                    |  0
 CHANGELOG.md                                      | 15 ++++++++++++++-
 dbt/adapters/bigquery/__version__.py              |  2 +-
 setup.py                                          |  2 +-
 10 files changed, 30 insertions(+), 4 deletions(-)
 create mode 100644 .changes/1.6.0-b1.md
 rename .changes/{unreleased => 1.6.0}/Fixes-20230202-010912.yaml (100%)
 rename .changes/{unreleased => 1.6.0}/Fixes-20230421-120450.yaml (100%)
 rename .changes/{unreleased => 1.6.0}/Fixes-20230427-141957.yaml (100%)
 rename .changes/{unreleased => 1.6.0}/Fixes-20230428-232904.yaml (100%)
 rename .changes/{unreleased => 1.6.0}/Fixes-20230509-222705.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 4fef75693..a0294846b 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.6.0a1
+current_version = 1.6.0b1
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.6.0-b1.md b/.changes/1.6.0-b1.md
new file mode 100644
index 000000000..8593f2dad
--- /dev/null
+++ b/.changes/1.6.0-b1.md
@@ -0,0 +1,13 @@
+## dbt-bigquery 1.6.0-b1 - May 12, 2023
+
+### Fixes
+
+- Support all types of data_type using time ingestion partitioning as previously `date` was failing ([#486](https://github.com/dbt-labs/dbt-bigquery/issues/486))
+-  Fix issue of sporadic failure to apply grants during high transaction volumes ([#614](https://github.com/dbt-labs/dbt-bigquery/issues/614))
+- Fix UDF usage with time ingestion ([#684](https://github.com/dbt-labs/dbt-bigquery/issues/684))
+- Fixes adding policy tags when a struct is defined in the yml ([#687](https://github.com/dbt-labs/dbt-bigquery/issues/687))
+- Update signature for execute method ([#](https://github.com/dbt-labs/dbt-bigquery/issues/), [#](https://github.com/dbt-labs/dbt-bigquery/issues/))
+
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#486](https://github.com/dbt-labs/dbt-bigquery/issues/486), [#684](https://github.com/dbt-labs/dbt-bigquery/issues/684))
+- [@dgreen161](https://github.com/dgreen161) ([#687](https://github.com/dbt-labs/dbt-bigquery/issues/687))
diff --git a/.changes/unreleased/Fixes-20230202-010912.yaml b/.changes/1.6.0/Fixes-20230202-010912.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230202-010912.yaml
rename to .changes/1.6.0/Fixes-20230202-010912.yaml
diff --git a/.changes/unreleased/Fixes-20230421-120450.yaml b/.changes/1.6.0/Fixes-20230421-120450.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230421-120450.yaml
rename to .changes/1.6.0/Fixes-20230421-120450.yaml
diff --git a/.changes/unreleased/Fixes-20230427-141957.yaml b/.changes/1.6.0/Fixes-20230427-141957.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230427-141957.yaml
rename to .changes/1.6.0/Fixes-20230427-141957.yaml
diff --git a/.changes/unreleased/Fixes-20230428-232904.yaml b/.changes/1.6.0/Fixes-20230428-232904.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230428-232904.yaml
rename to .changes/1.6.0/Fixes-20230428-232904.yaml
diff --git a/.changes/unreleased/Fixes-20230509-222705.yaml b/.changes/1.6.0/Fixes-20230509-222705.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230509-222705.yaml
rename to .changes/1.6.0/Fixes-20230509-222705.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 065c8d636..2c39a245d 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,10 +5,23 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
-## dbt-bigquery 1.6.0-a1 - April 17, 2023
+## dbt-bigquery 1.6.0-b1 - May 12, 2023
+
+### Fixes
+
+- Support all types of data_type using time ingestion partitioning as previously `date` was failing ([#486](https://github.com/dbt-labs/dbt-bigquery/issues/486))
+-  Fix issue of sporadic failure to apply grants during high transaction volumes ([#614](https://github.com/dbt-labs/dbt-bigquery/issues/614))
+- Fix UDF usage with time ingestion ([#684](https://github.com/dbt-labs/dbt-bigquery/issues/684))
+- Fixes adding policy tags when a struct is defined in the yml ([#687](https://github.com/dbt-labs/dbt-bigquery/issues/687))
+- Update signature for execute method ([#](https://github.com/dbt-labs/dbt-bigquery/issues/), [#](https://github.com/dbt-labs/dbt-bigquery/issues/))
 
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#486](https://github.com/dbt-labs/dbt-bigquery/issues/486), [#684](https://github.com/dbt-labs/dbt-bigquery/issues/684))
+- [@dgreen161](https://github.com/dgreen161) ([#687](https://github.com/dbt-labs/dbt-bigquery/issues/687))
 
 
+## dbt-bigquery 1.6.0-a1 - April 17, 2023
+
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 * [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 07fc02eef..cafa91966 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.6.0a1"
+version = "1.6.0b1"
diff --git a/setup.py b/setup.py
index c70e08f7e..90a5d7c49 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.6.0a1"
+package_version = "1.6.0b1"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From 495566a24e9e8a1f35ee366ae1ecf8e944a6f966 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Mon, 15 May 2023 09:29:43 -0500
Subject: [PATCH 624/860] more github deprecations fixes (#714)

* more github deprecations fixes

* swap out abandonded action
---
 .github/workflows/bot-changelog.yml |  2 +-
 .github/workflows/triage-labels.yml | 13 +++++--------
 2 files changed, 6 insertions(+), 9 deletions(-)

diff --git a/.github/workflows/bot-changelog.yml b/.github/workflows/bot-changelog.yml
index 8122ab8b4..9938c51e5 100644
--- a/.github/workflows/bot-changelog.yml
+++ b/.github/workflows/bot-changelog.yml
@@ -49,7 +49,7 @@ jobs:
     - name: Create and commit changelog on bot PR
       if: "contains(github.event.pull_request.labels.*.name, ${{ matrix.label }})"
       id: bot_changelog
-      uses: emmyoop/changie_bot@v1.0
+      uses: emmyoop/changie_bot@v1.1.0
       with:
         GITHUB_TOKEN: ${{ secrets.FISHTOWN_BOT_PAT }}
         commit_author_name: "Github Build Bot"
diff --git a/.github/workflows/triage-labels.yml b/.github/workflows/triage-labels.yml
index a71dc5e1f..97649a522 100644
--- a/.github/workflows/triage-labels.yml
+++ b/.github/workflows/triage-labels.yml
@@ -23,11 +23,8 @@ permissions:
 
 jobs:
   triage_label:
-    if: contains(github.event.issue.labels.*.name, 'awaiting_response')
-    runs-on: ubuntu-latest
-    steps:
-      - name: initial labeling
-        uses: andymckay/labeler@master
-        with:
-          add-labels: "triage"
-          remove-labels: "awaiting_response"
+    uses: dbt-labs/actions/.github/workflows/swap-labels.yml@main
+    with:
+      add_label: "triage"
+      remove_label: "awaiting_response"
+    secrets: inherit # this is only acceptable because we own the action we're calling

From 22b3c750b03edaa5d8e8f3aed26f57c852c5fbf5 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Tue, 16 May 2023 09:30:00 -0500
Subject: [PATCH 625/860] updating set-output (#725)

* updating set-output

* add pipe
---
 .github/workflows/integration.yml | 3 ++-
 .github/workflows/main.yml        | 5 +++--
 2 files changed, 5 insertions(+), 3 deletions(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 7a6890566..b36090ced 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -183,7 +183,8 @@ jobs:
       - name: Get current date
         if: always()
         id: date
-        run: echo "::set-output name=date::$(date +'%Y-%m-%dT%H_%M_%S')" #no colons allowed for artifacts
+        run: |
+          echo "date=$(date +'%Y-%m-%dT%H_%M_%S')" >> $GITHUB_OUTPUT #no colons allowed for artifacts
 
       - uses: actions/upload-artifact@v3
         if: always()
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index cc499fa28..1d7e7e8c6 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -103,7 +103,8 @@ jobs:
       - name: Get current date
         if: always()
         id: date
-        run: echo "::set-output name=date::$(date +'%Y-%m-%dT%H_%M_%S')" #no colons allowed for artifacts
+        run: |
+          echo "date=$(date +'%Y-%m-%dT%H_%M_%S')" >> $GITHUB_OUTPUT #no colons allowed for artifacts
 
       - uses: actions/upload-artifact@v3
         if: always()
@@ -155,7 +156,7 @@ jobs:
         run: |
           export is_alpha=0
           if [[ "$(ls -lh dist/)" == *"a1"* ]]; then export is_alpha=1; fi
-          echo "::set-output name=is_alpha::$is_alpha"
+          echo "is_alpha=$is_alpha" >> $GITHUB_OUTPUT
 
       - uses: actions/upload-artifact@v3
         with:

From 7bc2af0cef1e42b625cee6d996b48cb91dc2e49c Mon Sep 17 00:00:00 2001
From: David Bloss <david.bloss@dbtlabs.com>
Date: Wed, 17 May 2023 15:46:25 -0500
Subject: [PATCH 626/860] update used gh actions ahead of node12 deprecation
 (#729)

---
 .github/workflows/backport.yml             | 2 +-
 .github/workflows/bot-changelog.yml        | 2 +-
 .github/workflows/integration.yml          | 2 +-
 .github/workflows/main.yml                 | 8 ++++----
 .github/workflows/release-branch-tests.yml | 2 +-
 5 files changed, 8 insertions(+), 8 deletions(-)

diff --git a/.github/workflows/backport.yml b/.github/workflows/backport.yml
index d5c7fffed..46f240347 100644
--- a/.github/workflows/backport.yml
+++ b/.github/workflows/backport.yml
@@ -35,6 +35,6 @@ jobs:
       github.event.pull_request.merged
       && contains(github.event.label.name, 'backport')
     steps:
-      - uses: tibdex/backport@v2.0.2
+      - uses: tibdex/backport@v2
         with:
           github_token: ${{ secrets.GITHUB_TOKEN }}
diff --git a/.github/workflows/bot-changelog.yml b/.github/workflows/bot-changelog.yml
index 9938c51e5..94498d25f 100644
--- a/.github/workflows/bot-changelog.yml
+++ b/.github/workflows/bot-changelog.yml
@@ -49,7 +49,7 @@ jobs:
     - name: Create and commit changelog on bot PR
       if: "contains(github.event.pull_request.labels.*.name, ${{ matrix.label }})"
       id: bot_changelog
-      uses: emmyoop/changie_bot@v1.1.0
+      uses: emmyoop/changie_bot@v1
       with:
         GITHUB_TOKEN: ${{ secrets.FISHTOWN_BOT_PAT }}
         commit_author_name: "Github Build Bot"
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index b36090ced..472baab31 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -145,7 +145,7 @@ jobs:
           ref: ${{ github.event.pull_request.head.sha }}
 
       - name: Set up Python ${{ matrix.python-version }}
-        uses: actions/setup-python@v4.3.0
+        uses: actions/setup-python@v4
         with:
           python-version: ${{ matrix.python-version }}
 
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 1d7e7e8c6..243d3aaa4 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -48,7 +48,7 @@ jobs:
           persist-credentials: false
 
       - name: Set up Python
-        uses: actions/setup-python@v4.3.0
+        uses: actions/setup-python@v4
         with:
           python-version: '3.8'
 
@@ -84,7 +84,7 @@ jobs:
           persist-credentials: false
 
       - name: Set up Python ${{ matrix.python-version }}
-        uses: actions/setup-python@v4.3.0
+        uses: actions/setup-python@v4
         with:
           python-version: ${{ matrix.python-version }}
 
@@ -127,7 +127,7 @@ jobs:
           persist-credentials: false
 
       - name: Set up Python
-        uses: actions/setup-python@v4.3.0
+        uses: actions/setup-python@v4
         with:
           python-version: '3.8'
 
@@ -180,7 +180,7 @@ jobs:
 
     steps:
       - name: Set up Python ${{ matrix.python-version }}
-        uses: actions/setup-python@v4.3.0
+        uses: actions/setup-python@v4
         with:
           python-version: ${{ matrix.python-version }}
       - name: Install python dependencies
diff --git a/.github/workflows/release-branch-tests.yml b/.github/workflows/release-branch-tests.yml
index 2297d979e..d20cd05ad 100644
--- a/.github/workflows/release-branch-tests.yml
+++ b/.github/workflows/release-branch-tests.yml
@@ -46,7 +46,7 @@ jobs:
     - name: Call CI workflow for ${{ matrix.branch }} branch
       id: trigger-step
       continue-on-error: true
-      uses: aurelien-baudet/workflow-dispatch@v2.1.1
+      uses: aurelien-baudet/workflow-dispatch@v2
       with:
         workflow: ${{ matrix.workflow_name }}
         ref: ${{ matrix.branch }}

From 3f3cfdc9e46a06eb59fd38a939a3d5b46bd5bebf Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 25 May 2023 15:14:51 -0500
Subject: [PATCH 627/860] [Fix] small change to test pathing for assert clause
 (#739)

* changing assert path to reach success sate of RunResult

* adding more to assert clause
---
 .../adapter/upload_file_tests/test_upload_file.py          | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)

diff --git a/tests/functional/adapter/upload_file_tests/test_upload_file.py b/tests/functional/adapter/upload_file_tests/test_upload_file.py
index 474db11ed..ba9a2289b 100644
--- a/tests/functional/adapter/upload_file_tests/test_upload_file.py
+++ b/tests/functional/adapter/upload_file_tests/test_upload_file.py
@@ -1,5 +1,6 @@
 import pytest
 from dbt.tests.util import get_relation_columns, run_dbt, run_sql_with_adapter
+from dbt.contracts.results import NodeStatus
 import datetime
 import yaml
 
@@ -75,7 +76,7 @@ def test_bigquery_upload_file_csv(self, project):
             }
         )
         upload_result = run_dbt(["run-operation", "upload_file", "--args", upload_args])
-        assert upload_result.success
+        assert upload_result.results[0].status == NodeStatus.Success
 
         # Check if the uploaded table contains expected values and schema
         self.perform_uploaded_table_checks(project.test_schema, "TestUploadFileCSV", project)
@@ -94,7 +95,7 @@ def test_bigquery_upload_file_ndjson(self, project):
             }
         )
         upload_result = run_dbt(["run-operation", "upload_file", "--args", upload_args])
-        assert upload_result.success
+        assert upload_result.results[0].status == NodeStatus.Success
 
         # Check if the uploaded table contains expected values and schema
         self.perform_uploaded_table_checks(project.test_schema, "TestUploadFileNDJSON", project)
@@ -112,7 +113,7 @@ def test_bigquery_upload_file_parquet(self, project):
             }
         )
         upload_result = run_dbt(["run-operation", "upload_file", "--args", upload_args])
-        assert upload_result.success
+        assert upload_result.results[0].status == NodeStatus.Success
 
         # Check if the uploaded table contains expected values and schema
         self.perform_uploaded_table_checks(project.test_schema, "TestUploadFileParquet", project)

From 56851b93953612e6b4f5eb5bc3a5060383564913 Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Thu, 25 May 2023 21:05:08 +0000
Subject: [PATCH 628/860] Bumping version to 1.6.0b2 and generate changelog

---
 .bumpversion.cfg                     | 2 +-
 .changes/1.6.0-b2.md                 | 1 +
 CHANGELOG.md                         | 5 ++++-
 dbt/adapters/bigquery/__version__.py | 2 +-
 setup.py                             | 2 +-
 5 files changed, 8 insertions(+), 4 deletions(-)
 create mode 100644 .changes/1.6.0-b2.md

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index a0294846b..45829e5a3 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.6.0b1
+current_version = 1.6.0b2
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.6.0-b2.md b/.changes/1.6.0-b2.md
new file mode 100644
index 000000000..41f646609
--- /dev/null
+++ b/.changes/1.6.0-b2.md
@@ -0,0 +1 @@
+## dbt-bigquery 1.6.0-b2 - May 25, 2023
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 2c39a245d..990a14d1c 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,6 +5,10 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.6.0-b2 - May 25, 2023
+
+
+
 ## dbt-bigquery 1.6.0-b1 - May 12, 2023
 
 ### Fixes
@@ -19,7 +23,6 @@
 - [@Kayrnt](https://github.com/Kayrnt) ([#486](https://github.com/dbt-labs/dbt-bigquery/issues/486), [#684](https://github.com/dbt-labs/dbt-bigquery/issues/684))
 - [@dgreen161](https://github.com/dgreen161) ([#687](https://github.com/dbt-labs/dbt-bigquery/issues/687))
 
-
 ## dbt-bigquery 1.6.0-a1 - April 17, 2023
 
 ## Previous Releases
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index cafa91966..21c2b2836 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.6.0b1"
+version = "1.6.0b2"
diff --git a/setup.py b/setup.py
index 90a5d7c49..9dfabbbe7 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.6.0b1"
+package_version = "1.6.0b2"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From c09168f2e3739e369edfa632e095db0e90df4679 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Fri, 26 May 2023 11:08:00 -0500
Subject: [PATCH 629/860] replace with resusable workflow (#732)

* replace with resusable workflow

* fix triage label workflow

---------

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .github/workflows/release-branch-tests.yml | 43 +++-------------------
 .github/workflows/triage-labels.yml        |  1 +
 2 files changed, 6 insertions(+), 38 deletions(-)

diff --git a/.github/workflows/release-branch-tests.yml b/.github/workflows/release-branch-tests.yml
index d20cd05ad..a56bf22d2 100644
--- a/.github/workflows/release-branch-tests.yml
+++ b/.github/workflows/release-branch-tests.yml
@@ -2,10 +2,6 @@
 # The purpose of this workflow is to trigger CI to run for each
 # release branch and main branch on a regular cadence. If the CI workflow
 # fails for a branch, it will post to dev-core-alerts to raise awareness.
-# The 'aurelien-baudet/workflow-dispatch' Action triggers the existing
-# CI worklow file on the given branch to run so that even if we change the
-# CI workflow file in the future, the one that is tailored for the given
-# release branch will be used.
 
 # **why?**
 # Ensures release branches and main are always shippable and not broken.
@@ -28,37 +24,8 @@ on:
 permissions: read-all
 
 jobs:
-  kick-off-ci:
-    name: Kick-off CI
-    runs-on: ubuntu-latest
-
-    strategy:
-      # must run CI 1 branch at a time b/c the workflow-dispatch Action polls for
-      # latest run for results and it gets confused when we kick off multiple runs
-      # at once. There is a race condition so we will just run in sequential order.
-      max-parallel: 1
-      fail-fast: false
-      matrix:
-        branch: [1.0.latest, 1.1.latest, 1.2.latest, 1.3.latest, 1.4.latest, main]
-        workflow_name: [main.yml, integration.yml]
-
-    steps:
-    - name: Call CI workflow for ${{ matrix.branch }} branch
-      id: trigger-step
-      continue-on-error: true
-      uses: aurelien-baudet/workflow-dispatch@v2
-      with:
-        workflow: ${{ matrix.workflow_name }}
-        ref: ${{ matrix.branch }}
-        token: ${{ secrets.FISHTOWN_BOT_PAT }}
-
-    - name: Post failure to Slack
-      uses: ravsamhq/notify-slack-action@v2
-      if: ${{ always() && !contains(steps.trigger-step.outputs.workflow-conclusion,'success') }}
-      with:
-        status: ${{ job.status }}
-        notification_title: 'dbt-bigquery scheduled run of ${{ matrix.workflow_name }} on "${{ matrix.branch }}" branch not successful'
-        message_format: ':x: ${{ matrix.workflow_name }} CI on branch "${{ matrix.branch }}" ${{ steps.trigger-step.outputs.workflow-conclusion }}'
-        footer: 'Linked failed CI run ${{ steps.trigger-step.outputs.workflow-url }}'
-      env:
-        SLACK_WEBHOOK_URL: ${{ secrets.SLACK_DEV_CORE_ALERTS }}
+  run_tests:
+    uses: dbt-labs/actions/.github/workflows/release-branch-tests.yml@er/centralize/release-branch-tests
+    with:
+      workflows_to_run: '["main.yml", "integration.yml"]'
+    secrets: inherit
diff --git a/.github/workflows/triage-labels.yml b/.github/workflows/triage-labels.yml
index 97649a522..91f529e3e 100644
--- a/.github/workflows/triage-labels.yml
+++ b/.github/workflows/triage-labels.yml
@@ -23,6 +23,7 @@ permissions:
 
 jobs:
   triage_label:
+    if: contains(github.event.issue.labels.*.name, 'awaiting_response')
     uses: dbt-labs/actions/.github/workflows/swap-labels.yml@main
     with:
       add_label: "triage"

From 8d23ca4bdf945621baa4302fa10866aa4d1ba957 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Fri, 26 May 2023 11:08:25 -0500
Subject: [PATCH 630/860] replace with resusable workflow (#732)

* replace with resusable workflow

* fix triage label workflow

---------

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>

From 5f9b1bcf9d327640301943800a535b0599410ff4 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Fri, 26 May 2023 15:13:55 -0500
Subject: [PATCH 631/860] point to main (#741)

---
 .github/workflows/release-branch-tests.yml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/.github/workflows/release-branch-tests.yml b/.github/workflows/release-branch-tests.yml
index a56bf22d2..acffbab39 100644
--- a/.github/workflows/release-branch-tests.yml
+++ b/.github/workflows/release-branch-tests.yml
@@ -25,7 +25,7 @@ permissions: read-all
 
 jobs:
   run_tests:
-    uses: dbt-labs/actions/.github/workflows/release-branch-tests.yml@er/centralize/release-branch-tests
+    uses: dbt-labs/actions/.github/workflows/release-branch-tests.yml@main
     with:
       workflows_to_run: '["main.yml", "integration.yml"]'
     secrets: inherit

From 863c2f833c92c3edabb6a3504985294e56fd29d0 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 31 May 2023 11:22:54 -0400
Subject: [PATCH 632/860] Drop support for py37 (#742)

* drop support for py37

* drop support for py37

* drop support for py37
---
 .../unreleased/Breaking Changes-20230530-174051.yaml     | 6 ++++++
 .github/scripts/integration-test-matrix.js               | 2 +-
 .github/workflows/main.yml                               | 4 ++--
 .github/workflows/nightly-release.yml                    | 2 +-
 CONTRIBUTING.md                                          | 2 +-
 dev-requirements.txt                                     | 9 +++------
 setup.py                                                 | 5 ++---
 tox.ini                                                  | 8 ++++----
 8 files changed, 20 insertions(+), 18 deletions(-)
 create mode 100644 .changes/unreleased/Breaking Changes-20230530-174051.yaml

diff --git a/.changes/unreleased/Breaking Changes-20230530-174051.yaml b/.changes/unreleased/Breaking Changes-20230530-174051.yaml
new file mode 100644
index 000000000..c4d90db97
--- /dev/null
+++ b/.changes/unreleased/Breaking Changes-20230530-174051.yaml	
@@ -0,0 +1,6 @@
+kind: Breaking Changes
+body: Drop support for python 3.7
+time: 2023-05-30T17:40:51.510639-04:00
+custom:
+  Author: mikealfare
+  Issue: dbt-labs/dbt-core/7082
diff --git a/.github/scripts/integration-test-matrix.js b/.github/scripts/integration-test-matrix.js
index 58acf364d..bf7fd2ef7 100644
--- a/.github/scripts/integration-test-matrix.js
+++ b/.github/scripts/integration-test-matrix.js
@@ -1,6 +1,6 @@
 module.exports = ({ context }) => {
   const defaultPythonVersion = "3.8";
-  const supportedPythonVersions = ["3.7", "3.8", "3.9", "3.10", "3.11"];
+  const supportedPythonVersions = ["3.8", "3.9", "3.10", "3.11"];
   const supportedAdapters = ["bigquery"];
 
   // if PR, generate matrix based on files changed and PR labels
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 243d3aaa4..6b32c7f7f 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -71,7 +71,7 @@ jobs:
     strategy:
       fail-fast: false
       matrix:
-        python-version: ['3.7', '3.8', '3.9', '3.10', '3.11']
+        python-version: ['3.8', '3.9', '3.10', '3.11']
 
     env:
       TOXENV: "unit"
@@ -176,7 +176,7 @@ jobs:
       fail-fast: false
       matrix:
         os: [ubuntu-latest, macos-latest, windows-latest]
-        python-version: ['3.7', '3.8', '3.9', '3.10', '3.11']
+        python-version: ['3.8', '3.9', '3.10', '3.11']
 
     steps:
       - name: Set up Python ${{ matrix.python-version }}
diff --git a/.github/workflows/nightly-release.yml b/.github/workflows/nightly-release.yml
index 54c5fdc69..f552a04eb 100644
--- a/.github/workflows/nightly-release.yml
+++ b/.github/workflows/nightly-release.yml
@@ -26,7 +26,7 @@ defaults:
     shell: bash
 
 env:
-  RELEASE_BRANCH: "1.4.latest"
+  RELEASE_BRANCH: "main"
 
 jobs:
   aggregate-release-data:
diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md
index e0c905b39..e9432d363 100644
--- a/CONTRIBUTING.md
+++ b/CONTRIBUTING.md
@@ -67,7 +67,7 @@ $EDITOR test.env
 There are a few methods for running tests locally.
 
 #### `tox`
-`tox` takes care of managing Python virtualenvs and installing dependencies in order to run tests. You can also run tests in parallel, for example you can run unit tests for Python 3.7, Python 3.8, Python 3.9, and `flake8` checks in parallel with `tox -p`. Also, you can run unit tests for specific python versions with `tox -e py37`. The configuration of these tests are located in `tox.ini`.
+`tox` takes care of managing Python virtualenvs and installing dependencies in order to run tests. You can also run tests in parallel, for example you can run unit tests for Python 3.8, Python 3.9, and `flake8` checks in parallel with `tox -p`. Also, you can run unit tests for specific python versions with `tox -e py38`. The configuration of these tests are located in `tox.ini`.
 
 #### `pytest`
 Finally, you can also run a specific test or group of tests using `pytest` directly. With a Python virtualenv active and dev dependencies installed you can do things like:
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 7d6994a6c..0d914bd7e 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -8,15 +8,13 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 black~=23.3
 bumpversion~=0.6.0
 click~=8.1
-flake8~=5.0;python_version=="3.7"
-flake8~=6.0;python_version>="3.8"
+flake8~=6.0
 flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
 mypy==1.2.0  # patch updates have historically introduced breaking changes
 pip-tools~=6.13
-pre-commit~=2.21;python_version=="3.7"
-pre-commit~=3.2;python_version>="3.8"
+pre-commit~=3.2
 pre-commit-hooks~=4.4
 pytest~=7.3
 pytest-csv~=3.0
@@ -24,8 +22,7 @@ pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
 pytest-xdist~=3.2
 pytz~=2023.3
-tox~=3.0;python_version=="3.7"
-tox~=4.4;python_version>="3.8"
+tox~=4.4
 types-pytz~=2023.3
 types-requests~=2.28
 types-protobuf~=4.22
diff --git a/setup.py b/setup.py
index 9dfabbbe7..55fad9c5c 100644
--- a/setup.py
+++ b/setup.py
@@ -2,9 +2,9 @@
 import sys
 
 # require python 3.7 or newer
-if sys.version_info < (3, 7):
+if sys.version_info < (3, 8):
     print("Error: dbt does not support this version of Python.")
-    print("Please upgrade to Python 3.7 or higher.")
+    print("Please upgrade to Python 3.8 or higher.")
     sys.exit(1)
 
 try:
@@ -87,7 +87,6 @@ def _dbt_core_version(plugin_version: str) -> str:
         "Operating System :: Microsoft :: Windows",
         "Operating System :: MacOS :: MacOS X",
         "Operating System :: POSIX :: Linux",
-        "Programming Language :: Python :: 3.7",
         "Programming Language :: Python :: 3.8",
         "Programming Language :: Python :: 3.9",
         "Programming Language :: Python :: 3.10",
diff --git a/tox.ini b/tox.ini
index 4d552ab44..efa18083d 100644
--- a/tox.ini
+++ b/tox.ini
@@ -1,8 +1,8 @@
 [tox]
 skipsdist = True
-envlist = py37,py38,py39,py310,py311
+envlist = py38,py39,py310,py311
 
-[testenv:{unit,py37,py38,py39,py310,py311,py}]
+[testenv:{unit,py38,py39,py310,py311,py}]
 description = unit testing
 skip_install = true
 passenv =
@@ -13,7 +13,7 @@ deps =
   -rdev-requirements.txt
   -e.
 
-[testenv:{integration,py37,py38,py39,py310,py311,py}-{bigquery}]
+[testenv:{integration,py38,py39,py310,py311,py}-{bigquery}]
 description = adapter plugin integration testing
 skip_install = true
 passenv =
@@ -28,7 +28,7 @@ deps =
   -rdev-requirements.txt
   -e.
 
-[testenv:{python-tests,py37,py38,py39,py310,py311,py}]
+[testenv:{python-tests,py38,py39,py310,py311,py}]
 description = python integration testing
 skip_install = true
 passenv =

From 934e6ae176cb9e9c6d95ddd5117c059e8b91e361 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 31 May 2023 13:29:01 -0500
Subject: [PATCH 633/860] adding link to 1.5 release notes (#740)

* adding link to 1.5 release notes

* run changie merge
---
 .changes/0.0.0.md | 3 ++-
 CHANGELOG.md      | 5 ++---
 2 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/.changes/0.0.0.md b/.changes/0.0.0.md
index b719cfe93..e754a7f1a 100644
--- a/.changes/0.0.0.md
+++ b/.changes/0.0.0.md
@@ -1,6 +1,7 @@
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
-* [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
+- [1.5](https://github.com/dbt-labs/dbt-bigquery/blob/1.5.latest/CHANGELOG.md)
+- [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
 - [1.3](https://github.com/dbt-labs/dbt-bigquery/blob/1.3.latest/CHANGELOG.md)
 - [1.2](https://github.com/dbt-labs/dbt-bigquery/blob/1.2.latest/CHANGELOG.md)
 - [1.1](https://github.com/dbt-labs/dbt-bigquery/blob/1.1.latest/CHANGELOG.md)
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 990a14d1c..c944431c5 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -7,8 +7,6 @@
 
 ## dbt-bigquery 1.6.0-b2 - May 25, 2023
 
-
-
 ## dbt-bigquery 1.6.0-b1 - May 12, 2023
 
 ### Fixes
@@ -27,7 +25,8 @@
 
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
-* [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
+- [1.5](https://github.com/dbt-labs/dbt-bigquery/blob/1.5.latest/CHANGELOG.md)
+- [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
 - [1.3](https://github.com/dbt-labs/dbt-bigquery/blob/1.3.latest/CHANGELOG.md)
 - [1.2](https://github.com/dbt-labs/dbt-bigquery/blob/1.2.latest/CHANGELOG.md)
 - [1.1](https://github.com/dbt-labs/dbt-bigquery/blob/1.1.latest/CHANGELOG.md)

From 79517b2d03f682a623b8996dd0902c05b81e06e3 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 31 May 2023 15:58:46 -0500
Subject: [PATCH 634/860] remove unneeded install of core (#744)

---
 .github/workflows/integration.yml | 4 ----
 .github/workflows/main.yml        | 4 +---
 2 files changed, 1 insertion(+), 7 deletions(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 472baab31..1639058fa 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -156,10 +156,6 @@ jobs:
           python -m pip --version
           tox --version
 
-      - name: Install dbt-core latest
-        run: |
-          python -m pip install "git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core"
-
       - name: Run tox (bigquery)
         if: matrix.adapter == 'bigquery'
         env:
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 6b32c7f7f..b13c53e9f 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -94,9 +94,7 @@ jobs:
           python -m pip install tox
           python -m pip --version
           tox --version
-      - name: Install dbt-core latest
-        run: |
-          python -m pip install "git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core"
+
       - name: Run tox
         run: tox
 

From 389aceeff8e8e699c04d132c2c5b7af4c514c22c Mon Sep 17 00:00:00 2001
From: Michelle Ark <MichelleArk@users.noreply.github.com>
Date: Fri, 2 Jun 2023 12:35:52 -0700
Subject: [PATCH 635/860] add tests for rendering foreign key constraints
 (#698)

---
 .../unreleased/Fixes-20230511-143217.yaml     |  6 ++++
 tests/functional/adapter/test_constraints.py  | 31 +++++++++++++------
 2 files changed, 28 insertions(+), 9 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230511-143217.yaml

diff --git a/.changes/unreleased/Fixes-20230511-143217.yaml b/.changes/unreleased/Fixes-20230511-143217.yaml
new file mode 100644
index 000000000..ff56ce5d8
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230511-143217.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: test foreign key constraint rendering
+time: 2023-05-11T14:32:17.364819-04:00
+custom:
+  Author: michelleark
+  Issue: "7512"
diff --git a/tests/functional/adapter/test_constraints.py b/tests/functional/adapter/test_constraints.py
index 6b6189092..58a6f03cc 100644
--- a/tests/functional/adapter/test_constraints.py
+++ b/tests/functional/adapter/test_constraints.py
@@ -20,11 +20,15 @@
     my_model_incremental_wrong_name_sql,
     model_schema_yml,
     constrained_model_schema_yml,
+    model_fk_constraint_schema_yml,
+    my_model_wrong_order_depends_on_fk_sql,
+    foreign_key_model_sql,
+    my_model_incremental_wrong_order_depends_on_fk_sql,
 )
 
 _expected_sql_bigquery = """
 create or replace table <model_identifier> (
-    id integer not null primary key not enforced,
+    id integer not null primary key not enforced references <foreign_key_model_identifier> (id) not enforced,
     color string,
     date_day string
 )
@@ -34,6 +38,7 @@
     color,
     date_day from
   (
+    -- depends_on: <foreign_key_model_identifier>
     select 'blue' as color,
     1 as id,
     '2019-01-01' as date_day
@@ -45,6 +50,8 @@
 # - does not support a data type named 'text' (TODO handle this via type translation/aliasing!)
 constraints_yml = model_schema_yml.replace("text", "string")
 model_constraints_yml = constrained_model_schema_yml.replace("text", "string")
+model_fk_constraint_schema_yml = model_fk_constraint_schema_yml.replace("text", "string")
+constrained_model_schema_yml = constrained_model_schema_yml.replace("text", "string")
 
 
 class BigQueryColumnEqualSetup:
@@ -117,8 +124,9 @@ class TestBigQueryTableConstraintsRuntimeDdlEnforcement(BaseConstraintsRuntimeDd
     @pytest.fixture(scope="class")
     def models(self):
         return {
-            "my_model.sql": my_model_wrong_order_sql,
-            "constraints_schema.yml": constraints_yml,
+            "my_model.sql": my_model_wrong_order_depends_on_fk_sql,
+            "foreign_key_model.sql": foreign_key_model_sql,
+            "constraints_schema.yml": model_fk_constraint_schema_yml,
         }
 
     @pytest.fixture(scope="class")
@@ -145,8 +153,9 @@ class TestBigQueryIncrementalConstraintsRuntimeDdlEnforcement(
     @pytest.fixture(scope="class")
     def models(self):
         return {
-            "my_model.sql": my_model_incremental_wrong_order_sql,
-            "constraints_schema.yml": constraints_yml,
+            "my_model.sql": my_model_incremental_wrong_order_depends_on_fk_sql,
+            "foreign_key_model.sql": foreign_key_model_sql,
+            "constraints_schema.yml": model_fk_constraint_schema_yml,
         }
 
     @pytest.fixture(scope="class")
@@ -171,8 +180,9 @@ class TestBigQueryModelConstraintsRuntimeEnforcement(BaseModelConstraintsRuntime
     @pytest.fixture(scope="class")
     def models(self):
         return {
-            "my_model.sql": my_incremental_model_sql,
-            "constraints_schema.yml": model_constraints_yml,
+            "my_model.sql": my_model_wrong_order_depends_on_fk_sql,
+            "foreign_key_model.sql": foreign_key_model_sql,
+            "constraints_schema.yml": constrained_model_schema_yml,
         }
 
     @pytest.fixture(scope="class")
@@ -182,7 +192,8 @@ def expected_sql(self):
     id integer not null,
     color string,
     date_day string,
-    primary key (id) not enforced
+    primary key (id) not enforced,
+    foreign key (id) references <foreign_key_model_identifier> (id) not enforced
 )
 OPTIONS()
 as (
@@ -190,8 +201,10 @@ def expected_sql(self):
     color,
     date_day from
   (
-    select 1 as id,
+    -- depends_on: <foreign_key_model_identifier>
+    select
     'blue' as color,
+    1 as id,
     '2019-01-01' as date_day
   ) as model_subq
 );

From aa7bea8dbe4cdca0d6f98b3502083d2c677af116 Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Wed, 7 Jun 2023 09:57:03 -0700
Subject: [PATCH 636/860] Standardize the adapter for core debug changes.
 (#754)

* Standardize the adapter for core debug changes.

* add changelog

* Tweak fields

---------

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .changes/unreleased/Features-20230604-034603.yaml |  6 ++++++
 dbt/adapters/bigquery/connections.py              | 15 +++++++++++++--
 dbt/adapters/bigquery/impl.py                     |  4 ++++
 3 files changed, 23 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Features-20230604-034603.yaml

diff --git a/.changes/unreleased/Features-20230604-034603.yaml b/.changes/unreleased/Features-20230604-034603.yaml
new file mode 100644
index 000000000..a1eebc327
--- /dev/null
+++ b/.changes/unreleased/Features-20230604-034603.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Standardize the _connection_keys and debug_query for `dbt debug`.
+time: 2023-06-04T03:46:03.065575-07:00
+custom:
+  Author: versusfacit
+  Issue: PR754
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 47acceda9..c9c6055ac 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -169,17 +169,28 @@ def _connection_keys(self):
         return (
             "method",
             "database",
+            "execution_project",
             "schema",
             "location",
             "priority",
-            "timeout_seconds",
             "maximum_bytes_billed",
-            "execution_project",
+            "impersonate_service_account",
             "job_retry_deadline_seconds",
             "job_retries",
             "job_creation_timeout_seconds",
             "job_execution_timeout_seconds",
+            "keyfile",
+            "keyfile_json",
+            "timeout_seconds",
+            "token",
+            "refresh_token",
+            "client_id",
+            "client_secret",
+            "token_uri",
+            "dataproc_region",
+            "dataproc_cluster_name",
             "gcs_bucket",
+            "dataproc_batch",
         )
 
     @classmethod
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index cad7d0072..f5a369ffc 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -978,3 +978,7 @@ def render_model_constraint(cls, constraint: ModelLevelConstraint) -> Optional[s
             return f"{c} not enforced" if c else None
 
         return c
+
+    def debug_query(self):
+        """Override for DebugTask method"""
+        self.execute("select 1 as id")

From f65a8a3129c27f3e004d62f15ae47b6207b48802 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 7 Jun 2023 13:05:06 -0500
Subject: [PATCH 637/860] change group for env in bigquery (#758)

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .github/workflows/integration.yml | 2 +-
 scripts/env-setup.sh              | 2 +-
 test.env.example                  | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 1639058fa..a155701f2 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -163,7 +163,7 @@ jobs:
           BIGQUERY_TEST_ALT_DATABASE: ${{ secrets.BIGQUERY_TEST_ALT_DATABASE }}
           BIGQUERY_TEST_NO_ACCESS_DATABASE: ${{ secrets.BIGQUERY_TEST_NO_ACCESS_DATABASE }}
           DBT_TEST_USER_1: group:buildbot@dbtlabs.com
-          DBT_TEST_USER_2: group:dev-core@dbtlabs.com
+          DBT_TEST_USER_2: group:engineering-core-team@dbtlabs.com
           DBT_TEST_USER_3: serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserviceaccount.com
           DATAPROC_REGION: us-central1
           DATAPROC_CLUSTER_NAME: dbt-test-1
diff --git a/scripts/env-setup.sh b/scripts/env-setup.sh
index 726a00cf8..ca873b3aa 100644
--- a/scripts/env-setup.sh
+++ b/scripts/env-setup.sh
@@ -6,7 +6,7 @@ echo "TOXENV=integration-bigquery" >> $GITHUB_ENV
 echo "INTEGRATION_TESTS_SECRETS_PREFIX=BIGQUERY_TEST" >> $GITHUB_ENV
 # Set environment variables required for integration tests
 echo "DBT_TEST_USER_1=group:buildbot@dbtlabs.com" >> $GITHUB_ENV
-echo "DBT_TEST_USER_2=group:dev-core@dbtlabs.com" >> $GITHUB_ENV
+echo "DBT_TEST_USER_2=group:engineering-core-team@dbtlabs.com" >> $GITHUB_ENV
 echo "DBT_TEST_USER_3=serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserviceaccount.com" >> $GITHUB_ENV
 echo "DATAPROC_REGION=us-central1" >> $GITHUB_ENV
 echo "DATAPROC_CLUSTER_NAME=dbt-test-1" >> $GITHUB_ENV
diff --git a/test.env.example b/test.env.example
index 58893f5af..ffe9ee060 100644
--- a/test.env.example
+++ b/test.env.example
@@ -8,7 +8,7 @@ BIGQUERY_TEST_SERVICE_ACCOUNT_JSON='{}'
 
 # tests for local ci/cd
 DBT_TEST_USER_1="group:buildbot@dbtlabs.com"
-DBT_TEST_USER_2="group:dev-core@dbtlabs.com"
+DBT_TEST_USER_2="group:engineering-core-team@dbtlabs.com"
 DBT_TEST_USER_3="serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserviceaccount.com"
 
 # only needed for python model

From e5b456ade8ddfa55f71fe28ffaab215a45dcf3c6 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 7 Jun 2023 13:40:31 -0500
Subject: [PATCH 638/860] re-pointing reelease branch env for nightly releases
 to a .latest branch (#759)

* re-pointing reelease branch env for nightly releases to a .latest branch

* adding comment to explain why variable doesn't test against main

---------

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .github/workflows/nightly-release.yml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/.github/workflows/nightly-release.yml b/.github/workflows/nightly-release.yml
index f552a04eb..46db5b749 100644
--- a/.github/workflows/nightly-release.yml
+++ b/.github/workflows/nightly-release.yml
@@ -26,7 +26,7 @@ defaults:
     shell: bash
 
 env:
-  RELEASE_BRANCH: "main"
+  RELEASE_BRANCH: "1.5.latest" # must test against most recent .latest branch to have parity for dependency with core
 
 jobs:
   aggregate-release-data:

From f8b49a1b7d264d7a75c22f1c68fc617133d67e7b Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Thu, 8 Jun 2023 22:49:20 +0000
Subject: [PATCH 639/860] Bumping version to v1.6.0b3 and generate changelog

---
 .bumpversion.cfg                                 |  2 +-
 .changes/1.6.0-b3.md                             | 13 +++++++++++++
 .../Breaking Changes-20230530-174051.yaml        |  0
 .../Features-20230604-034603.yaml                |  0
 .../Fixes-20230511-143217.yaml                   |  0
 CHANGELOG.md                                     | 16 ++++++++++++++++
 dbt/adapters/bigquery/__version__.py             |  2 +-
 setup.py                                         |  2 +-
 8 files changed, 32 insertions(+), 3 deletions(-)
 create mode 100644 .changes/1.6.0-b3.md
 rename .changes/{unreleased => 1.6.0}/Breaking Changes-20230530-174051.yaml (100%)
 rename .changes/{unreleased => 1.6.0}/Features-20230604-034603.yaml (100%)
 rename .changes/{unreleased => 1.6.0}/Fixes-20230511-143217.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 45829e5a3..d90289e31 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.6.0b2
+current_version = v1.6.0b3
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.6.0-b3.md b/.changes/1.6.0-b3.md
new file mode 100644
index 000000000..966e9f2f2
--- /dev/null
+++ b/.changes/1.6.0-b3.md
@@ -0,0 +1,13 @@
+## dbt-bigquery 1.6.0-b3 - June 08, 2023
+
+### Breaking Changes
+
+- Drop support for python 3.7 ([#dbt-labs/dbt-core/7082](https://github.com/dbt-labs/dbt-bigquery/issues/dbt-labs/dbt-core/7082))
+
+### Features
+
+- Standardize the _connection_keys and debug_query for `dbt debug`. ([#PR754](https://github.com/dbt-labs/dbt-bigquery/issues/PR754))
+
+### Fixes
+
+- test foreign key constraint rendering ([#7512](https://github.com/dbt-labs/dbt-bigquery/issues/7512))
diff --git a/.changes/unreleased/Breaking Changes-20230530-174051.yaml b/.changes/1.6.0/Breaking Changes-20230530-174051.yaml
similarity index 100%
rename from .changes/unreleased/Breaking Changes-20230530-174051.yaml
rename to .changes/1.6.0/Breaking Changes-20230530-174051.yaml
diff --git a/.changes/unreleased/Features-20230604-034603.yaml b/.changes/1.6.0/Features-20230604-034603.yaml
similarity index 100%
rename from .changes/unreleased/Features-20230604-034603.yaml
rename to .changes/1.6.0/Features-20230604-034603.yaml
diff --git a/.changes/unreleased/Fixes-20230511-143217.yaml b/.changes/1.6.0/Fixes-20230511-143217.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230511-143217.yaml
rename to .changes/1.6.0/Fixes-20230511-143217.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index c944431c5..776cbfe8c 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,6 +5,22 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.6.0-b3 - June 08, 2023
+
+### Breaking Changes
+
+- Drop support for python 3.7 ([#dbt-labs/dbt-core/7082](https://github.com/dbt-labs/dbt-bigquery/issues/dbt-labs/dbt-core/7082))
+
+### Features
+
+- Standardize the _connection_keys and debug_query for `dbt debug`. ([#PR754](https://github.com/dbt-labs/dbt-bigquery/issues/PR754))
+
+### Fixes
+
+- test foreign key constraint rendering ([#7512](https://github.com/dbt-labs/dbt-bigquery/issues/7512))
+
+
+
 ## dbt-bigquery 1.6.0-b2 - May 25, 2023
 
 ## dbt-bigquery 1.6.0-b1 - May 12, 2023
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 21c2b2836..0c2870f87 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.6.0b2"
+version = "1.6.0b3"
diff --git a/setup.py b/setup.py
index 55fad9c5c..eefb4da35 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.6.0b2"
+package_version = "1.6.0b3"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From 580ad7649ffb00f7220938cca4cc4bc533b8d124 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen <jeremy@dbtlabs.com>
Date: Tue, 13 Jun 2023 23:48:21 -0400
Subject: [PATCH 640/860] Test constraint rendering with column quoting (#713)

---
 tests/functional/adapter/test_constraints.py | 32 ++++++++++++++++++++
 1 file changed, 32 insertions(+)

diff --git a/tests/functional/adapter/test_constraints.py b/tests/functional/adapter/test_constraints.py
index 58a6f03cc..dcefae7b9 100644
--- a/tests/functional/adapter/test_constraints.py
+++ b/tests/functional/adapter/test_constraints.py
@@ -8,6 +8,7 @@
     BaseIncrementalConstraintsRuntimeDdlEnforcement,
     BaseIncrementalConstraintsRollback,
     BaseModelConstraintsRuntimeEnforcement,
+    BaseConstraintQuotedColumn,
 )
 from dbt.tests.adapter.constraints.fixtures import (
     my_model_sql,
@@ -18,8 +19,10 @@
     my_model_wrong_name_sql,
     my_model_view_wrong_name_sql,
     my_model_incremental_wrong_name_sql,
+    my_model_with_quoted_column_name_sql,
     model_schema_yml,
     constrained_model_schema_yml,
+    model_quoted_column_schema_yml,
     model_fk_constraint_schema_yml,
     my_model_wrong_order_depends_on_fk_sql,
     foreign_key_model_sql,
@@ -209,3 +212,32 @@ def expected_sql(self):
   ) as model_subq
 );
 """
+
+
+class TestBigQueryConstraintQuotedColumn(BaseConstraintQuotedColumn):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model.sql": my_model_with_quoted_column_name_sql,
+            "constraints_schema.yml": model_quoted_column_schema_yml.replace("text", "string"),
+        }
+
+    @pytest.fixture(scope="class")
+    def expected_sql(self):
+        return """
+create or replace table <model_identifier> (
+    id integer not null,
+    `from` string not null,
+    date_day string
+)
+options()
+as (
+    select id, `from`, date_day
+    from (
+        select
+          'blue' as `from`,
+          1 as id,
+          '2019-01-01' as date_day
+    ) as model_subq
+);
+"""

From b1e950bf6f8f6c3528b65080e257b746839ef87c Mon Sep 17 00:00:00 2001
From: Michelle Ark <MichelleArk@users.noreply.github.com>
Date: Wed, 14 Jun 2023 10:34:49 -0700
Subject: [PATCH 641/860] model contracts + constraints with nested fields
 (#738)

---
 .../unreleased/Features-20230601-141255.yaml  |   6 +
 dbt/adapters/bigquery/column.py               | 133 ++++++++++++++++-
 dbt/adapters/bigquery/impl.py                 |  40 +++++-
 .../macros/utils/get_columns_spec_ddl.sql     |  22 +++
 .../adapter/constraints/fixtures.py           | 118 +++++++++++++++
 .../{ => constraints}/test_constraints.py     | 101 ++++++++++++-
 tests/unit/test_column.py                     | 135 ++++++++++++++++++
 7 files changed, 548 insertions(+), 7 deletions(-)
 create mode 100644 .changes/unreleased/Features-20230601-141255.yaml
 create mode 100644 tests/functional/adapter/constraints/fixtures.py
 rename tests/functional/adapter/{ => constraints}/test_constraints.py (71%)
 create mode 100644 tests/unit/test_column.py

diff --git a/.changes/unreleased/Features-20230601-141255.yaml b/.changes/unreleased/Features-20230601-141255.yaml
new file mode 100644
index 000000000..cfb428e3f
--- /dev/null
+++ b/.changes/unreleased/Features-20230601-141255.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Support model contracts + constraints on nested columns
+time: 2023-06-01T14:12:55.433346-04:00
+custom:
+  Author: MichelleArk
+  Issue: "673"
diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py
index 158f6650c..cfa298b82 100644
--- a/dbt/adapters/bigquery/column.py
+++ b/dbt/adapters/bigquery/column.py
@@ -1,5 +1,5 @@
 from dataclasses import dataclass
-from typing import Optional, List, TypeVar, Iterable, Type, Any
+from typing import Optional, List, TypeVar, Iterable, Type, Any, Dict, Union
 
 from dbt.adapters.base.column import Column
 
@@ -126,3 +126,134 @@ def column_to_bq_schema(self) -> SchemaField:
             kwargs = {"fields": fields}
 
         return SchemaField(self.name, self.dtype, self.mode, **kwargs)  # type: ignore[arg-type]
+
+
+def get_nested_column_data_types(
+    columns: Dict[str, Dict[str, Any]],
+    constraints: Optional[Dict[str, str]] = None,
+) -> Dict[str, Dict[str, str]]:
+    """
+    columns:
+        * Dictionary where keys are of flat columns names and values are dictionary of column attributes
+        * column names with "." indicate a nested column within a STRUCT type
+        * e.g. {"a": {"name": "a", "data_type": "string", ...}}
+    constraints:
+        * Dictionary where keys are flat column names and values are rendered constraints for the column
+        * If provided, rendered column is included in returned "data_type" values.
+    returns:
+        * Dictionary where keys are root column names and values are corresponding nested data_type values.
+        * Fields other than "name" and "data_type" are __not__ preserved in the return value for nested columns.
+        * Fields other than "name" and "data_type" are preserved in the return value for flat columns.
+
+    Example:
+    columns: {
+        "a": {"name": "a", "data_type": "string", "description": ...},
+        "b.nested": {"name": "b.nested", "data_type": "string"},
+        "b.nested2": {"name": "b.nested2", "data_type": "string"}
+        }
+
+    returns: {
+        "a": {"name": "a", "data_type": "string"},
+        "b": {"name": "b": "data_type": "struct<nested string, nested2 string>}
+    }
+    """
+    constraints = constraints or {}
+
+    nested_column_data_types: Dict[str, Union[str, Dict]] = {}
+    for column in columns.values():
+        _update_nested_column_data_types(
+            column["name"],
+            column["data_type"],
+            constraints.get(column["name"]),
+            nested_column_data_types,
+        )
+
+    formatted_nested_column_data_types: Dict[str, Dict[str, str]] = {}
+    for column_name, unformatted_column_type in nested_column_data_types.items():
+        formatted_nested_column_data_types[column_name] = {
+            "name": column_name,
+            "data_type": _format_nested_data_type(unformatted_column_type),
+        }
+
+    # add column configs back to flat columns
+    for column_name in formatted_nested_column_data_types:
+        if column_name in columns:
+            formatted_nested_column_data_types[column_name].update(
+                {
+                    k: v
+                    for k, v in columns[column_name].items()
+                    if k not in formatted_nested_column_data_types[column_name]
+                }
+            )
+
+    return formatted_nested_column_data_types
+
+
+def _update_nested_column_data_types(
+    column_name: str,
+    column_data_type: str,
+    column_rendered_constraint: Optional[str],
+    nested_column_data_types: Dict[str, Union[str, Dict]],
+) -> None:
+    """
+    Recursively update nested_column_data_types given a column_name, column_data_type, and optional column_rendered_constraint.
+
+    Examples:
+    >>> nested_column_data_types = {}
+    >>> BigQueryAdapter._update_nested_column_data_types("a", "string", "not_null", nested_column_data_types)
+    >>> nested_column_data_types
+    {"a": "string not null"}
+    >>> BigQueryAdapter._update_nested_column_data_types("b.c", "string", "not_null", nested_column_data_types)
+    >>> nested_column_data_types
+    {"a": "string not null", "b": {"c": "string not null"}}
+    >>> BigQueryAdapter._update_nested_column_data_types("b.d", "string", None, nested_column_data_types)
+    >>> nested_column_data_types
+    {"a": "string not null", "b": {"c": "string not null", "d": "string"}}
+    """
+    column_name_parts = column_name.split(".")
+    root_column_name = column_name_parts[0]
+
+    if len(column_name_parts) == 1:
+        # Base case: column is not nested - store its data_type concatenated with constraint if provided.
+        nested_column_data_types[root_column_name] = (
+            column_data_type
+            if column_rendered_constraint is None
+            else f"{column_data_type} {column_rendered_constraint}"
+        )
+    else:
+        # Initialize nested dictionary
+        if root_column_name not in nested_column_data_types:
+            nested_column_data_types[root_column_name] = {}
+
+        # Recursively process rest of remaining column name
+        remaining_column_name = ".".join(column_name_parts[1:])
+        remaining_column_data_types = nested_column_data_types[root_column_name]
+        assert isinstance(remaining_column_data_types, dict)  # keeping mypy happy
+        _update_nested_column_data_types(
+            remaining_column_name,
+            column_data_type,
+            column_rendered_constraint,
+            remaining_column_data_types,
+        )
+
+
+def _format_nested_data_type(unformatted_nested_data_type: Union[str, Dict[str, Any]]) -> str:
+    """
+    Recursively format a (STRUCT) data type given an arbitrarily nested data type structure.
+
+    Examples:
+    >>> BigQueryAdapter._format_nested_data_type("string")
+    'string'
+    >>> BigQueryAdapter._format_nested_data_type({'c': 'string not_null', 'd': 'string'})
+    'struct<c string not_null, d string>'
+    >>> BigQueryAdapter._format_nested_data_type({'c': 'string not_null', 'd': {'e': 'string'}})
+    'struct<c string not_null, d struct<e string>>'
+    """
+    if isinstance(unformatted_nested_data_type, str):
+        return unformatted_nested_data_type
+    else:
+        formatted_nested_types = [
+            f"{column_name} {_format_nested_data_type(column_type)}"
+            for column_name, column_type in unformatted_nested_data_type.items()
+        ]
+        return f"""struct<{", ".join(formatted_nested_types)}>"""
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index f5a369ffc..37ed2c2f1 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -23,6 +23,7 @@
 
 from dbt.adapters.cache import _make_ref_key_dict  # type: ignore
 
+from dbt.adapters.bigquery.column import get_nested_column_data_types
 from dbt.adapters.bigquery.relation import BigQueryRelation
 from dbt.adapters.bigquery.dataset import add_access_entry_to_dataset
 from dbt.adapters.bigquery import BigQueryColumn
@@ -293,6 +294,15 @@ def check_schema_exists(self, database: str, schema: str) -> bool:
             return False
         return True
 
+    @available.parse(lambda *a, **k: {})
+    @classmethod
+    def nest_column_data_types(
+        cls,
+        columns: Dict[str, Dict[str, Any]],
+        constraints: Optional[Dict[str, str]] = None,
+    ) -> Dict[str, Dict[str, str]]:
+        return get_nested_column_data_types(columns, constraints)
+
     def get_columns_in_relation(self, relation: BigQueryRelation) -> List[BigQueryColumn]:
         try:
             table = self.connections.get_bq_table(
@@ -526,7 +536,10 @@ def get_column_schema_from_query(self, sql: str) -> List[BigQueryColumn]:
         """
         _, iterator = self.connections.raw_execute(sql)
         columns = [self.Column.create_from_field(field) for field in iterator.schema]
-        return columns
+        flattened_columns = []
+        for column in columns:
+            flattened_columns += column.flatten()
+        return flattened_columns
 
     @available.parse(lambda *a, **k: False)
     def get_columns_in_select_sql(self, select_sql: str) -> List[BigQueryColumn]:
@@ -958,6 +971,31 @@ def python_submission_helpers(self) -> Dict[str, Type[PythonJobHelper]]:
             "serverless": ServerlessDataProcHelper,
         }
 
+    @available
+    @classmethod
+    def render_raw_columns_constraints(cls, raw_columns: Dict[str, Dict[str, Any]]) -> List:
+        rendered_constraints: Dict[str, str] = {}
+        for raw_column in raw_columns.values():
+            for con in raw_column.get("constraints", None):
+                constraint = cls._parse_column_constraint(con)
+                rendered_constraint = cls.process_parsed_constraint(
+                    constraint, cls.render_column_constraint
+                )
+
+                if rendered_constraint:
+                    column_name = raw_column["name"]
+                    if column_name not in rendered_constraints:
+                        rendered_constraints[column_name] = rendered_constraint
+                    else:
+                        rendered_constraints[column_name] += f" {rendered_constraint}"
+
+        nested_columns = cls.nest_column_data_types(raw_columns, rendered_constraints)
+        rendered_column_constraints = [
+            f"{cls.quote(column['name']) if column.get('quote') else column['name']} {column['data_type']}"
+            for column in nested_columns.values()
+        ]
+        return rendered_column_constraints
+
     @classmethod
     def render_column_constraint(cls, constraint: ColumnLevelConstraint) -> Optional[str]:
         c = super().render_column_constraint(constraint)  # type: ignore
diff --git a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
index d078e8ad9..a1247dcd6 100644
--- a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
+++ b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
@@ -3,3 +3,25 @@
   {% set formatted = column.column.lower() ~ " " ~ data_type %}
   {{ return({'name': column.name, 'data_type': data_type, 'formatted': formatted}) }}
 {%- endmacro -%}
+
+{% macro bigquery__get_empty_schema_sql(columns) %}
+    {%- set columns = adapter.nest_column_data_types(columns) -%}
+    {{ return(dbt.default__get_empty_schema_sql(columns)) }}
+{% endmacro %}
+
+{% macro bigquery__get_select_subquery(sql) %}
+    select {{ adapter.dispatch('get_column_names')() }}
+    from (
+        {{ sql }}
+    ) as model_subq
+{%- endmacro %}
+
+{% macro bigquery__get_column_names() %}
+  {#- loop through nested user_provided_columns to get column names -#}
+    {%- set user_provided_columns = adapter.nest_column_data_types(model['columns']) -%}
+    {%- for i in user_provided_columns %}
+      {%- set col = user_provided_columns[i] -%}
+      {%- set col_name = adapter.quote(col['name']) if col.get('quote') else col['name'] -%}
+      {{ col_name }}{{ ", " if not loop.last }}
+    {%- endfor -%}
+{% endmacro %}
diff --git a/tests/functional/adapter/constraints/fixtures.py b/tests/functional/adapter/constraints/fixtures.py
new file mode 100644
index 000000000..415043403
--- /dev/null
+++ b/tests/functional/adapter/constraints/fixtures.py
@@ -0,0 +1,118 @@
+my_model_struct_wrong_data_type_sql = """
+{{ config(materialized = "table") }}
+
+select
+  STRUCT(1 AS struct_column_being_tested, "test" AS another_struct_column) as a
+"""
+
+my_model_struct_correct_data_type_sql = """
+{{ config(materialized = "table")}}
+
+select
+  STRUCT("test" AS struct_column_being_tested, "test" AS b) as a
+"""
+
+model_struct_data_type_schema_yml = """
+version: 2
+models:
+  - name: contract_struct_wrong
+    config:
+      contract:
+        enforced: true
+    columns:
+      - name: a.struct_column_being_tested
+        data_type: string
+      - name: a.b
+        data_type: string
+
+  - name: contract_struct_correct
+    config:
+      contract:
+        enforced: true
+    columns:
+      - name: a.struct_column_being_tested
+        data_type: string
+      - name: a.b
+        data_type: string
+"""
+
+my_model_double_struct_wrong_data_type_sql = """
+{{ config(materialized = "table") }}
+
+select
+  STRUCT(
+    STRUCT(1 AS struct_column_being_tested, "test" AS c) as b,
+    "test" as d
+    ) as a
+"""
+
+my_model_double_struct_correct_data_type_sql = """
+{{ config(materialized = "table") }}
+
+select
+  STRUCT(
+    STRUCT("test" AS struct_column_being_tested, "test" AS c) as b,
+    "test" as d
+    ) as a
+"""
+
+model_double_struct_data_type_schema_yml = """
+version: 2
+models:
+  - name: contract_struct_wrong
+    config:
+      contract:
+        enforced: true
+    columns:
+      - name: a.b.struct_column_being_tested
+        data_type: string
+      - name: a.b.c
+        data_type: string
+      - name: a.d
+        data_type: string
+
+  - name: contract_struct_correct
+    config:
+      contract:
+        enforced: true
+    columns:
+      - name: a.b.struct_column_being_tested
+        data_type: string
+      - name: a.b.c
+        data_type: string
+      - name: a.d
+        data_type: string
+"""
+
+
+my_model_struct_sql = """
+{{
+  config(
+    materialized = "table"
+  )
+}}
+
+select STRUCT("test" as nested_column, "test" as nested_column2) as id
+"""
+
+
+model_struct_schema_yml = """
+version: 2
+models:
+  - name: my_model
+    config:
+      contract:
+        enforced: true
+    columns:
+      - name: id.nested_column
+        quote: true
+        data_type: string
+        description: hello
+        constraints:
+          - type: not_null
+          - type: unique
+      - name: id.nested_column2
+        data_type: string
+        constraints:
+          - type: unique
+"""
diff --git a/tests/functional/adapter/test_constraints.py b/tests/functional/adapter/constraints/test_constraints.py
similarity index 71%
rename from tests/functional/adapter/test_constraints.py
rename to tests/functional/adapter/constraints/test_constraints.py
index dcefae7b9..e1ab87f35 100644
--- a/tests/functional/adapter/test_constraints.py
+++ b/tests/functional/adapter/constraints/test_constraints.py
@@ -29,6 +29,19 @@
     my_model_incremental_wrong_order_depends_on_fk_sql,
 )
 
+from tests.functional.adapter.constraints.fixtures import (
+    my_model_struct_wrong_data_type_sql,
+    my_model_struct_correct_data_type_sql,
+    my_model_double_struct_wrong_data_type_sql,
+    my_model_double_struct_correct_data_type_sql,
+    model_struct_data_type_schema_yml,
+    model_double_struct_data_type_schema_yml,
+    my_model_struct_sql,
+    model_struct_schema_yml,
+)
+
+from dbt.tests.util import run_dbt_and_capture, run_dbt
+
 _expected_sql_bigquery = """
 create or replace table <model_identifier> (
     id integer not null primary key not enforced references <foreign_key_model_identifier> (id) not enforced,
@@ -49,6 +62,19 @@
 );
 """
 
+_expected_struct_sql_bigquery = """
+create or replace table <model_identifier> (
+    id struct<nested_column string not null, nested_column2 string>
+)
+OPTIONS()
+as (
+    select id from
+  (
+    select STRUCT("test" as nested_column, "test" as nested_column2) as id
+  ) as model_subq
+);
+"""
+
 # Different on BigQuery:
 # - does not support a data type named 'text' (TODO handle this via type translation/aliasing!)
 constraints_yml = model_schema_yml.replace("text", "string")
@@ -79,11 +105,6 @@ def data_types(self, int_type, string_type):
             ["[1,2,3]", f"ARRAY<{int_type}>", f"ARRAY<{int_type}>"],
             ["cast(1 as NUMERIC)", "numeric", "NUMERIC"],
             ["""JSON '{"name": "Cooper", "forname": "Alice"}'""", "json", "JSON"],
-            [
-                'STRUCT("Rudisha" AS name, [23.4, 26.3, 26.4, 26.1] AS laps)',
-                "STRUCT<name STRING, laps ARRAY<FLOAT64>>",
-                "STRUCT<name STRING, laps ARRAY<FLOAT64>>",
-            ],
         ]
 
 
@@ -123,6 +144,61 @@ def models(self):
         }
 
 
+class BaseStructContract:
+    @pytest.fixture
+    def wrong_schema_data_type(self):
+        return "INT64"
+
+    @pytest.fixture
+    def correct_schema_data_type(self):
+        return "STRING"
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "contract_struct_schema.yml": model_struct_data_type_schema_yml,
+            "contract_struct_wrong.sql": my_model_struct_wrong_data_type_sql,
+            "contract_struct_correct.sql": my_model_struct_correct_data_type_sql,
+        }
+
+    def test__struct_contract_wrong_data_type(
+        self, project, correct_schema_data_type, wrong_schema_data_type
+    ):
+        results, log_output = run_dbt_and_capture(
+            ["run", "-s", "contract_struct_wrong"], expect_pass=False
+        )
+        assert len(results) == 1
+        assert results[0].node.config.contract.enforced
+
+        expected = [
+            "struct_column_being_tested",
+            wrong_schema_data_type,
+            correct_schema_data_type,
+            "data type mismatch",
+        ]
+        assert all([(exp in log_output or exp.upper() in log_output) for exp in expected])
+
+    def test__struct_contract_correct_data_type(self, project):
+        results = run_dbt(["run", "-s", "contract_struct_correct"])
+
+        assert len(results) == 1
+        assert results[0].node.config.contract.enforced
+
+
+class TestBigQueryStructContract(BaseStructContract):
+    pass
+
+
+class TestBigQueryDoubleStructContract(BaseStructContract):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "contract_struct_schema.yml": model_double_struct_data_type_schema_yml,
+            "contract_struct_wrong.sql": my_model_double_struct_wrong_data_type_sql,
+            "contract_struct_correct.sql": my_model_double_struct_correct_data_type_sql,
+        }
+
+
 class TestBigQueryTableConstraintsRuntimeDdlEnforcement(BaseConstraintsRuntimeDdlEnforcement):
     @pytest.fixture(scope="class")
     def models(self):
@@ -137,6 +213,21 @@ def expected_sql(self, project):
         return _expected_sql_bigquery
 
 
+class TestBigQueryStructTableConstraintsRuntimeDdlEnforcement(
+    BaseConstraintsRuntimeDdlEnforcement
+):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model.sql": my_model_struct_sql,
+            "constraints_schema.yml": model_struct_schema_yml,
+        }
+
+    @pytest.fixture(scope="class")
+    def expected_sql(self, project):
+        return _expected_struct_sql_bigquery
+
+
 class TestBigQueryTableConstraintsRollback(BaseConstraintsRollback):
     @pytest.fixture(scope="class")
     def models(self):
diff --git a/tests/unit/test_column.py b/tests/unit/test_column.py
new file mode 100644
index 000000000..8c3c645bb
--- /dev/null
+++ b/tests/unit/test_column.py
@@ -0,0 +1,135 @@
+import pytest
+
+from dbt.adapters.bigquery.column import get_nested_column_data_types
+
+
+@pytest.mark.parametrize(
+    ["columns", "constraints", "expected_nested_columns"],
+    [
+        ({}, None, {}),
+        ({}, {"not_in_columns": "unique"}, {}),
+        # Flat column
+        (
+            {"a": {"name": "a", "data_type": "string"}},
+            None,
+            {"a": {"name": "a", "data_type": "string"}},
+        ),
+        # Flat column - with constraints
+        (
+            {"a": {"name": "a", "data_type": "string"}},
+            {"a": "not null"},
+            {"a": {"name": "a", "data_type": "string not null"}},
+        ),
+        # Flat column - with constraints + other keys
+        (
+            {"a": {"name": "a", "data_type": "string", "quote": True}},
+            {"a": "not null"},
+            {"a": {"name": "a", "data_type": "string not null", "quote": True}},
+        ),
+        # Single nested column, 1 level
+        (
+            {"b.nested": {"name": "b.nested", "data_type": "string"}},
+            None,
+            {"b": {"name": "b", "data_type": "struct<nested string>"}},
+        ),
+        # Single nested column, 1 level - with constraints
+        (
+            {"b.nested": {"name": "b.nested", "data_type": "string"}},
+            {"b.nested": "not null"},
+            {"b": {"name": "b", "data_type": "struct<nested string not null>"}},
+        ),
+        # Single nested column, 1 level - with constraints + other keys
+        (
+            {"b.nested": {"name": "b.nested", "data_type": "string", "other": "unpreserved"}},
+            {"b.nested": "not null"},
+            {"b": {"name": "b", "data_type": "struct<nested string not null>"}},
+        ),
+        # Multiple nested columns, 1 level
+        (
+            {
+                "b.nested": {"name": "b.nested", "data_type": "string"},
+                "b.nested2": {"name": "b.nested2", "data_type": "int64"},
+            },
+            None,
+            {"b": {"name": "b", "data_type": "struct<nested string, nested2 int64>"}},
+        ),
+        # Multiple nested columns, 1 level - with constraints
+        (
+            {
+                "b.nested": {"name": "b.nested", "data_type": "string"},
+                "b.nested2": {"name": "b.nested2", "data_type": "int64"},
+            },
+            {"b.nested": "not null"},
+            {"b": {"name": "b", "data_type": "struct<nested string not null, nested2 int64>"}},
+        ),
+        # Multiple nested columns, 1 level - with constraints
+        (
+            {
+                "b.nested": {"name": "b.nested", "data_type": "string"},
+                "b.nested2": {"name": "b.nested2", "data_type": "int64"},
+            },
+            {"b.nested": "not null"},
+            {"b": {"name": "b", "data_type": "struct<nested string not null, nested2 int64>"}},
+        ),
+        # Mix of flat and nested columns, 1 level
+        (
+            {
+                "a": {"name": "a", "data_type": "string"},
+                "b.nested": {"name": "b.nested", "data_type": "string"},
+                "b.nested2": {"name": "b.nested2", "data_type": "int64"},
+            },
+            None,
+            {
+                "b": {"name": "b", "data_type": "struct<nested string, nested2 int64>"},
+                "a": {"name": "a", "data_type": "string"},
+            },
+        ),
+        # Nested columns, multiple levels
+        (
+            {
+                "b.user.name.first": {
+                    "name": "b.user.name.first",
+                    "data_type": "string",
+                },
+                "b.user.name.last": {
+                    "name": "b.user.name.last",
+                    "data_type": "string",
+                },
+                "b.user.id": {"name": "b.user.id", "data_type": "int64"},
+                "b.user.country": {"name": "b.user.country", "data_type": "string"},
+            },
+            None,
+            {
+                "b": {
+                    "name": "b",
+                    "data_type": "struct<user struct<name struct<first string, last string>, id int64, country string>>",
+                },
+            },
+        ),
+        # Nested columns, multiple levels - with constraints!
+        (
+            {
+                "b.user.name.first": {
+                    "name": "b.user.name.first",
+                    "data_type": "string",
+                },
+                "b.user.name.last": {
+                    "name": "b.user.name.last",
+                    "data_type": "string",
+                },
+                "b.user.id": {"name": "b.user.id", "data_type": "int64"},
+                "b.user.country": {"name": "b.user.country", "data_type": "string"},
+            },
+            {"b.user.name.first": "not null", "b.user.id": "unique"},
+            {
+                "b": {
+                    "name": "b",
+                    "data_type": "struct<user struct<name struct<first string not null, last string>, id int64 unique, country string>>",
+                },
+            },
+        ),
+    ],
+)
+def test_get_nested_column_data_types(columns, constraints, expected_nested_columns):
+    actual_nested_columns = get_nested_column_data_types(columns, constraints)
+    assert expected_nested_columns == actual_nested_columns

From a1401efb97de2cc16fa381b9872f720e51e54dc6 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Fri, 16 Jun 2023 11:43:37 -0700
Subject: [PATCH 642/860] fix utils.py (#776)

---
 tests/unit/utils.py | 1 +
 1 file changed, 1 insertion(+)

diff --git a/tests/unit/utils.py b/tests/unit/utils.py
index 7cd2bb9ab..4d6dee451 100644
--- a/tests/unit/utils.py
+++ b/tests/unit/utils.py
@@ -75,6 +75,7 @@ def project_from_dict(project, profile, packages=None, selectors=None, cli_vars=
         project_root=project_root,
         project_dict=project,
         packages_dict=packages,
+        dependent_projects_dict={},
         selectors_dict=selectors,
     )
     return partial.render(renderer)

From b8d127fbb67df30cd1190050e01200cf953d9ff3 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen <jeremy@dbtlabs.com>
Date: Tue, 20 Jun 2023 23:03:54 +0200
Subject: [PATCH 643/860] Rm agate as explicit dep (#778)

* Rm agate as explicit dep

* Add changelog entry
---
 .changes/unreleased/Dependencies-20230618-220517.yaml | 6 ++++++
 setup.py                                              | 1 -
 2 files changed, 6 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230618-220517.yaml

diff --git a/.changes/unreleased/Dependencies-20230618-220517.yaml b/.changes/unreleased/Dependencies-20230618-220517.yaml
new file mode 100644
index 000000000..359ae9b7c
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230618-220517.yaml
@@ -0,0 +1,6 @@
+kind: Dependencies
+body: Rm explicit agate pin, in favor of transitive dependency from dbt-core
+time: 2023-06-18T22:05:17.020743-04:00
+custom:
+  Author: jtcohen6
+  PR: "777"
diff --git a/setup.py b/setup.py
index eefb4da35..b4526be7a 100644
--- a/setup.py
+++ b/setup.py
@@ -78,7 +78,6 @@ def _dbt_core_version(plugin_version: str) -> str:
         "google-cloud-bigquery~=3.0",
         "google-cloud-storage~=2.4",
         "google-cloud-dataproc~=5.0",
-        "agate~=1.6.3",
     ],
     zip_safe=False,
     classifiers=[

From 5dd68e266ebd185b9239ee4df94afbf07eecbb80 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Wed, 21 Jun 2023 10:40:55 -0700
Subject: [PATCH 644/860] Pass python model timeout to polling instead of retry
 (#766)

* Pass python model timeout to polling instead of retry

* rename retry to result_polling_policy
---
 .../unreleased/Fixes-20230609-132727.yaml     |  7 +++++
 dbt/adapters/bigquery/python_submissions.py   | 10 +++++--
 tests/functional/adapter/test_python_model.py | 28 +++++++++++++++++--
 tox.ini                                       |  2 +-
 4 files changed, 41 insertions(+), 6 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230609-132727.yaml

diff --git a/.changes/unreleased/Fixes-20230609-132727.yaml b/.changes/unreleased/Fixes-20230609-132727.yaml
new file mode 100644
index 000000000..c37d2e72a
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230609-132727.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: Pass python model timeout to polling operation so model execution times out
+  as expected.
+time: 2023-06-09T13:27:27.279842-07:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "577"
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index e5fbf037e..0c7ce1917 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -1,6 +1,8 @@
 from typing import Dict, Union
 
 from dbt.adapters.base import PythonJobHelper
+from google.api_core.future.polling import POLLING_PREDICATE
+
 from dbt.adapters.bigquery import BigQueryConnectionManager, BigQueryCredentials
 from dbt.adapters.bigquery.connections import DataprocBatchConfig
 from google.api_core import retry
@@ -43,7 +45,9 @@ def __init__(self, parsed_model: Dict, credential: BigQueryCredentials) -> None:
         self.timeout = self.parsed_model["config"].get(
             "timeout", self.credential.job_execution_timeout_seconds or 60 * 60 * 24
         )
-        self.retry = retry.Retry(maximum=10.0, deadline=self.timeout)
+        self.result_polling_policy = retry.Retry(
+            predicate=POLLING_PREDICATE, maximum=10.0, timeout=self.timeout
+        )
         self.client_options = ClientOptions(
             api_endpoint="{}-dataproc.googleapis.com:443".format(self.credential.dataproc_region)
         )
@@ -98,7 +102,7 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
                 "job": job,
             }
         )
-        response = operation.result(retry=self.retry)
+        response = operation.result(polling=self.result_polling_policy)
         # check if job failed
         if response.status.state == 6:
             raise ValueError(response.status.details)
@@ -123,7 +127,7 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         operation = self.job_client.create_batch(request=request)  # type: ignore
         # this takes quite a while, waiting on GCP response to resolve
         # (not a google-api-core issue, more likely a dataproc serverless issue)
-        response = operation.result(retry=self.retry)
+        response = operation.result(polling=self.result_polling_policy)
         return response
         # there might be useful results here that we can parse and return
         # Dataproc job output is saved to the Cloud Storage bucket
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
index 955bc6be0..7f17429f1 100644
--- a/tests/functional/adapter/test_python_model.py
+++ b/tests/functional/adapter/test_python_model.py
@@ -1,14 +1,38 @@
 import os
 import pytest
-from dbt.tests.util import run_dbt, write_file
+from dbt.tests.util import run_dbt, run_dbt_and_capture, write_file
 import dbt.tests.adapter.python_model.test_python_model as dbt_tests
 
 TEST_SKIP_MESSAGE = (
     "Skipping the Tests since Dataproc serverless is not stable. " "TODO: Fix later"
 )
 
+blocks_for_thirty_sec = """
+def model(dbt, _):
+    dbt.config(
+        materialized='table',
+        timeout=5
+    )
+    import pandas as pd
+    data = {'col_1': [3, 2, 1, 0], 'col_2': ['a', 'b', 'c', 'd']}
+    df = pd.DataFrame.from_dict(data)
+    import time
+    time.sleep(30)
+    return df
+"""
+
+
+class TestPythonModelDataprocTimeoutTest:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {"30_sec_python_model.py": blocks_for_thirty_sec}
+
+    def test_model_times_out(self, project):
+        result, output = run_dbt_and_capture(["run"], expect_pass=False)
+        assert len(result) == 1
+        assert "Operation did not complete within the designated timeout of 5 seconds." in output
+
 
-@pytest.mark.skip(reason=TEST_SKIP_MESSAGE)
 class TestPythonModelDataproc(dbt_tests.BasePythonModelTests):
     pass
 
diff --git a/tox.ini b/tox.ini
index efa18083d..e0342e04d 100644
--- a/tox.ini
+++ b/tox.ini
@@ -23,7 +23,7 @@ passenv =
     DATAPROC_*
     GCS_BUCKET
 commands =
-  bigquery: {envpython} -m pytest {posargs} -vv tests/functional --profile service_account
+  bigquery: {envpython} -m pytest {posargs} -vv tests/functional -k "not TestPython" --profile service_account
 deps =
   -rdev-requirements.txt
   -e.

From 9b3243b6dfe77f9c75ee684b3672c355b0098dbc Mon Sep 17 00:00:00 2001
From: Michelle Ark <MichelleArk@users.noreply.github.com>
Date: Thu, 22 Jun 2023 16:04:40 -0700
Subject: [PATCH 645/860] add sql_header tests for constraints (#750)

* add sql_header tests for constraints

* changelog entry

* revert dev-requirements.txt changes
---
 .../unreleased/Fixes-20230601-120430.yaml     |  6 +++
 .../adapter/constraints/test_constraints.py   | 51 +++++++++++++++++++
 2 files changed, 57 insertions(+)
 create mode 100644 .changes/unreleased/Fixes-20230601-120430.yaml

diff --git a/.changes/unreleased/Fixes-20230601-120430.yaml b/.changes/unreleased/Fixes-20230601-120430.yaml
new file mode 100644
index 000000000..715ac26a6
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230601-120430.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: test model constraints with sql headers
+time: 2023-06-01T12:04:30.876751-04:00
+custom:
+  Author: michelleark
+  Issue: "7714"
diff --git a/tests/functional/adapter/constraints/test_constraints.py b/tests/functional/adapter/constraints/test_constraints.py
index e1ab87f35..c8c0c11e2 100644
--- a/tests/functional/adapter/constraints/test_constraints.py
+++ b/tests/functional/adapter/constraints/test_constraints.py
@@ -3,6 +3,8 @@
     BaseTableConstraintsColumnsEqual,
     BaseViewConstraintsColumnsEqual,
     BaseIncrementalConstraintsColumnsEqual,
+    BaseTableContractSqlHeader,
+    BaseIncrementalContractSqlHeader,
     BaseConstraintsRuntimeDdlEnforcement,
     BaseConstraintsRollback,
     BaseIncrementalConstraintsRuntimeDdlEnforcement,
@@ -22,6 +24,7 @@
     my_model_with_quoted_column_name_sql,
     model_schema_yml,
     constrained_model_schema_yml,
+    model_contract_header_schema_yml,
     model_quoted_column_schema_yml,
     model_fk_constraint_schema_yml,
     my_model_wrong_order_depends_on_fk_sql,
@@ -79,9 +82,39 @@
 # - does not support a data type named 'text' (TODO handle this via type translation/aliasing!)
 constraints_yml = model_schema_yml.replace("text", "string")
 model_constraints_yml = constrained_model_schema_yml.replace("text", "string")
+model_contract_header_schema_yml = model_contract_header_schema_yml.replace("text", "string")
 model_fk_constraint_schema_yml = model_fk_constraint_schema_yml.replace("text", "string")
 constrained_model_schema_yml = constrained_model_schema_yml.replace("text", "string")
 
+my_model_contract_sql_header_sql = """
+{{
+  config(
+    materialized = "table"
+  )
+}}
+
+{% call set_sql_header(config) %}
+DECLARE DEMO STRING DEFAULT 'hello world';
+{% endcall %}
+
+SELECT DEMO as column_name
+"""
+
+my_model_incremental_contract_sql_header_sql = """
+{{
+  config(
+    materialized = "incremental",
+    on_schema_change="append_new_columns"
+  )
+}}
+
+{% call set_sql_header(config) %}
+DECLARE DEMO STRING DEFAULT 'hello world';
+{% endcall %}
+
+SELECT DEMO as column_name
+"""
+
 
 class BigQueryColumnEqualSetup:
     @pytest.fixture
@@ -144,6 +177,24 @@ def models(self):
         }
 
 
+class TestBigQueryTableContractsSqlHeader(BaseTableContractSqlHeader):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model_contract_sql_header.sql": my_model_contract_sql_header_sql,
+            "constraints_schema.yml": model_contract_header_schema_yml,
+        }
+
+
+class TestBigQueryIncrementalContractsSqlHeader(BaseIncrementalContractSqlHeader):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "my_model_contract_sql_header.sql": my_model_incremental_contract_sql_header_sql,
+            "constraints_schema.yml": model_contract_header_schema_yml,
+        }
+
+
 class BaseStructContract:
     @pytest.fixture
     def wrong_schema_data_type(self):

From 3bf21a545233b477f7a4749d6a95f56b59c9391a Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Fri, 23 Jun 2023 01:13:52 +0000
Subject: [PATCH 646/860] Bumping version to 1.6.0b4 and generate changelog

---
 .bumpversion.cfg                              |  2 +-
 .changes/1.6.0-b4.md                          | 14 ++++++++++++++
 .../Dependencies-20230618-220517.yaml         |  0
 .../Features-20230601-141255.yaml             |  0
 .../Fixes-20230601-120430.yaml                |  0
 .../Fixes-20230609-132727.yaml                |  0
 CHANGELOG.md                                  | 19 +++++++++++++++++--
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 9 files changed, 34 insertions(+), 5 deletions(-)
 create mode 100644 .changes/1.6.0-b4.md
 rename .changes/{unreleased => 1.6.0}/Dependencies-20230618-220517.yaml (100%)
 rename .changes/{unreleased => 1.6.0}/Features-20230601-141255.yaml (100%)
 rename .changes/{unreleased => 1.6.0}/Fixes-20230601-120430.yaml (100%)
 rename .changes/{unreleased => 1.6.0}/Fixes-20230609-132727.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index d90289e31..d529ec377 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = v1.6.0b3
+current_version = 1.6.0b4
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.6.0-b4.md b/.changes/1.6.0-b4.md
new file mode 100644
index 000000000..47310f31d
--- /dev/null
+++ b/.changes/1.6.0-b4.md
@@ -0,0 +1,14 @@
+## dbt-bigquery 1.6.0-b4 - June 23, 2023
+
+### Features
+
+- Support model contracts + constraints on nested columns ([#673](https://github.com/dbt-labs/dbt-bigquery/issues/673))
+
+### Fixes
+
+- test model constraints with sql headers ([#7714](https://github.com/dbt-labs/dbt-bigquery/issues/7714))
+- Pass python model timeout to polling operation so model execution times out as expected. ([#577](https://github.com/dbt-labs/dbt-bigquery/issues/577))
+
+### Dependencies
+
+- Rm explicit agate pin, in favor of transitive dependency from dbt-core ([#777](https://github.com/dbt-labs/dbt-bigquery/pull/777))
diff --git a/.changes/unreleased/Dependencies-20230618-220517.yaml b/.changes/1.6.0/Dependencies-20230618-220517.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230618-220517.yaml
rename to .changes/1.6.0/Dependencies-20230618-220517.yaml
diff --git a/.changes/unreleased/Features-20230601-141255.yaml b/.changes/1.6.0/Features-20230601-141255.yaml
similarity index 100%
rename from .changes/unreleased/Features-20230601-141255.yaml
rename to .changes/1.6.0/Features-20230601-141255.yaml
diff --git a/.changes/unreleased/Fixes-20230601-120430.yaml b/.changes/1.6.0/Fixes-20230601-120430.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230601-120430.yaml
rename to .changes/1.6.0/Fixes-20230601-120430.yaml
diff --git a/.changes/unreleased/Fixes-20230609-132727.yaml b/.changes/1.6.0/Fixes-20230609-132727.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230609-132727.yaml
rename to .changes/1.6.0/Fixes-20230609-132727.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 776cbfe8c..c43286a02 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,6 +5,23 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.6.0-b4 - June 23, 2023
+
+### Features
+
+- Support model contracts + constraints on nested columns ([#673](https://github.com/dbt-labs/dbt-bigquery/issues/673))
+
+### Fixes
+
+- test model constraints with sql headers ([#7714](https://github.com/dbt-labs/dbt-bigquery/issues/7714))
+- Pass python model timeout to polling operation so model execution times out as expected. ([#577](https://github.com/dbt-labs/dbt-bigquery/issues/577))
+
+### Dependencies
+
+- Rm explicit agate pin, in favor of transitive dependency from dbt-core ([#777](https://github.com/dbt-labs/dbt-bigquery/pull/777))
+
+
+
 ## dbt-bigquery 1.6.0-b3 - June 08, 2023
 
 ### Breaking Changes
@@ -19,8 +36,6 @@
 
 - test foreign key constraint rendering ([#7512](https://github.com/dbt-labs/dbt-bigquery/issues/7512))
 
-
-
 ## dbt-bigquery 1.6.0-b2 - May 25, 2023
 
 ## dbt-bigquery 1.6.0-b1 - May 12, 2023
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 0c2870f87..091852496 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.6.0b3"
+version = "1.6.0b4"
diff --git a/setup.py b/setup.py
index b4526be7a..05bda5c87 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.6.0b3"
+package_version = "1.6.0b4"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From a04caa696d4ef275a42700afe7d8e68a41d5cfe5 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 23 Jun 2023 17:07:27 -0400
Subject: [PATCH 647/860] Bump mypy from 1.2.0 to 1.4.0 (#780)

* Bump mypy from 1.2.0 to 1.4.0

Bumps [mypy](https://github.com/python/mypy) from 1.2.0 to 1.4.0.
- [Commits](https://github.com/python/mypy/compare/v1.2.0...v1.4.0)

---
updated-dependencies:
- dependency-name: mypy
  dependency-type: direct:development
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230621-005934.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230621-005934.yaml

diff --git a/.changes/unreleased/Dependencies-20230621-005934.yaml b/.changes/unreleased/Dependencies-20230621-005934.yaml
new file mode 100644
index 000000000..4730a2462
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230621-005934.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump mypy from 1.2.0 to 1.4.0"
+time: 2023-06-21T00:59:34.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 780
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 0d914bd7e..a2b09e23f 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -12,7 +12,7 @@ flake8~=6.0
 flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
-mypy==1.2.0  # patch updates have historically introduced breaking changes
+mypy==1.4.0  # patch updates have historically introduced breaking changes
 pip-tools~=6.13
 pre-commit~=3.2
 pre-commit-hooks~=4.4

From aa8bef82e3810c9ef4b70f77bf22b20e117e0237 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 23 Jun 2023 17:39:55 -0400
Subject: [PATCH 648/860] Update types-requests requirement from ~=2.28 to
 ~=2.31 (#737)

* Update types-requests requirement from ~=2.28 to ~=2.31

Updates the requirements on [types-requests](https://github.com/python/typeshed) to permit the latest version.
- [Commits](https://github.com/python/typeshed/commits)

---
updated-dependencies:
- dependency-name: types-requests
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230606-003859.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230606-003859.yaml

diff --git a/.changes/unreleased/Dependencies-20230606-003859.yaml b/.changes/unreleased/Dependencies-20230606-003859.yaml
new file mode 100644
index 000000000..d362c9c8e
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230606-003859.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update types-requests requirement from ~=2.28 to ~=2.31"
+time: 2023-06-06T00:38:59.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 737
diff --git a/dev-requirements.txt b/dev-requirements.txt
index a2b09e23f..348b90de8 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -24,7 +24,7 @@ pytest-xdist~=3.2
 pytz~=2023.3
 tox~=4.4
 types-pytz~=2023.3
-types-requests~=2.28
+types-requests~=2.31
 types-protobuf~=4.22
 twine~=4.0
 wheel~=0.40

From 94eed37c52569a2dc39f7e477bd5d42d93630595 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Sat, 24 Jun 2023 13:22:15 -0400
Subject: [PATCH 649/860] Update tox requirement from ~=4.4 to ~=4.6 (#757)

* Update tox requirement from ~=4.4 to ~=4.6

Updates the requirements on [tox](https://github.com/tox-dev/tox) to permit the latest version.
- [Release notes](https://github.com/tox-dev/tox/releases)
- [Changelog](https://github.com/tox-dev/tox/blob/main/docs/changelog.rst)
- [Commits](https://github.com/tox-dev/tox/compare/4.4.0...4.6.0)

---
updated-dependencies:
- dependency-name: tox
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230606-005912.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230606-005912.yaml

diff --git a/.changes/unreleased/Dependencies-20230606-005912.yaml b/.changes/unreleased/Dependencies-20230606-005912.yaml
new file mode 100644
index 000000000..4e2ccab63
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230606-005912.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update tox requirement from ~=4.4 to ~=4.6"
+time: 2023-06-06T00:59:12.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 757
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 348b90de8..182c97dc6 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -22,7 +22,7 @@ pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
 pytest-xdist~=3.2
 pytz~=2023.3
-tox~=4.4
+tox~=4.6
 types-pytz~=2023.3
 types-requests~=2.31
 types-protobuf~=4.22

From 0b08610258a809c19c7011a26ba40ca3a2d183e0 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Sat, 24 Jun 2023 14:18:57 -0400
Subject: [PATCH 650/860] Update types-protobuf requirement from ~=4.22 to
 ~=4.23 (#706)

* Update types-protobuf requirement from ~=4.22 to ~=4.23

Updates the requirements on [types-protobuf](https://github.com/python/typeshed) to permit the latest version.
- [Commits](https://github.com/python/typeshed/commits)

---
updated-dependencies:
- dependency-name: types-protobuf
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230624-172518.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230624-172518.yaml

diff --git a/.changes/unreleased/Dependencies-20230624-172518.yaml b/.changes/unreleased/Dependencies-20230624-172518.yaml
new file mode 100644
index 000000000..817bc17c4
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230624-172518.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update types-protobuf requirement from ~=4.22 to ~=4.23"
+time: 2023-06-24T17:25:18.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 706
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 182c97dc6..a5b619358 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -24,7 +24,7 @@ pytest-xdist~=3.2
 pytz~=2023.3
 tox~=4.6
 types-pytz~=2023.3
+types-protobuf~=4.23
 types-requests~=2.31
-types-protobuf~=4.22
 twine~=4.0
 wheel~=0.40

From a42e13d8023123102c4d1aeefdfa6cd6246c4768 Mon Sep 17 00:00:00 2001
From: Nicholas Zeolla <nick.zeolla@starlingbank.com>
Date: Mon, 26 Jun 2023 19:23:35 +0100
Subject: [PATCH 651/860] Add `batch_id` param to profile  to pass along with
 `dataproc_v1.CreateBatchRequest` (#727)

* Add new param to profile batch_id to pass along with google.cloud.dataproc.v1.CreateBatchRequest

* add changie

* Add to credential fixtures

* implement suggestion on PR

---------

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Features-20230517-092205.yaml | 6 ++++++
 dbt/adapters/bigquery/connections.py              | 1 +
 dbt/adapters/bigquery/python_submissions.py       | 1 +
 test.env.example                                  | 1 +
 tests/conftest.py                                 | 2 ++
 5 files changed, 11 insertions(+)
 create mode 100644 .changes/unreleased/Features-20230517-092205.yaml

diff --git a/.changes/unreleased/Features-20230517-092205.yaml b/.changes/unreleased/Features-20230517-092205.yaml
new file mode 100644
index 000000000..9afde587d
--- /dev/null
+++ b/.changes/unreleased/Features-20230517-092205.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Add batch_id param to profile
+time: 2023-05-17T09:22:05.264368+01:00
+custom:
+  Author: nickozilla
+  Issue: "671"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index c9c6055ac..f726481e7 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -133,6 +133,7 @@ class BigQueryCredentials(Credentials):
     dataproc_region: Optional[str] = None
     dataproc_cluster_name: Optional[str] = None
     gcs_bucket: Optional[str] = None
+    batch_id: Optional[str] = None
 
     dataproc_batch: Optional[DataprocBatchConfig] = field(
         metadata={
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 0c7ce1917..29e33032d 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -122,6 +122,7 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         request = dataproc_v1.CreateBatchRequest(
             parent=parent,
             batch=batch,
+            batch_id=self.credential.batch_id,
         )
         # make the request
         operation = self.job_client.create_batch(request=request)  # type: ignore
diff --git a/test.env.example b/test.env.example
index ffe9ee060..70befbbbb 100644
--- a/test.env.example
+++ b/test.env.example
@@ -15,3 +15,4 @@ DBT_TEST_USER_3="serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserv
 DATAPROC_REGION=us-
 DATAPROC_CLUSTER_NAME=
 GCS_BUCKET=
+BATCH_ID=
diff --git a/tests/conftest.py b/tests/conftest.py
index 0ba0091fb..b5a967a09 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -33,6 +33,7 @@ def oauth_target():
         "dataproc_region": os.getenv("DATAPROC_REGION"),
         "dataproc_cluster_name": os.getenv("DATAPROC_CLUSTER_NAME"),
         "gcs_bucket": os.getenv("GCS_BUCKET"),
+        "batch_id": os.getenv("BATCH_ID"),
     }
 
 
@@ -53,4 +54,5 @@ def service_account_target():
             "DATAPROC_CLUSTER_NAME"
         ),  # only needed for cluster submission method
         "gcs_bucket": os.getenv("GCS_BUCKET"),
+        "batch_id": os.getenv("BATCH_ID"),
     }

From 1552cee1c8dfdeac10de88824fd97468908e9e55 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 26 Jun 2023 15:27:54 -0400
Subject: [PATCH 652/860] Update pre-commit requirement from ~=3.2 to ~=3.3
 (#787)

* Update pre-commit requirement from ~=3.2 to ~=3.3

Updates the requirements on [pre-commit](https://github.com/pre-commit/pre-commit) to permit the latest version.
- [Release notes](https://github.com/pre-commit/pre-commit/releases)
- [Changelog](https://github.com/pre-commit/pre-commit/blob/main/CHANGELOG.md)
- [Commits](https://github.com/pre-commit/pre-commit/compare/v3.2.0...v3.3.3)

---
updated-dependencies:
- dependency-name: pre-commit
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20230626-011525.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230626-011525.yaml

diff --git a/.changes/unreleased/Dependencies-20230626-011525.yaml b/.changes/unreleased/Dependencies-20230626-011525.yaml
new file mode 100644
index 000000000..fc8261ec1
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230626-011525.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pre-commit requirement from ~=3.2 to ~=3.3"
+time: 2023-06-26T01:15:25.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 787
diff --git a/dev-requirements.txt b/dev-requirements.txt
index a5b619358..f8d3aa591 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -14,7 +14,7 @@ freezegun~=1.2
 ipdb~=0.13.13
 mypy==1.4.0  # patch updates have historically introduced breaking changes
 pip-tools~=6.13
-pre-commit~=3.2
+pre-commit~=3.3
 pre-commit-hooks~=4.4
 pytest~=7.3
 pytest-csv~=3.0

From 93fd439fabade42c3049ccdb277739ea0e933a48 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 26 Jun 2023 16:48:13 -0400
Subject: [PATCH 653/860] Bump mypy from 1.4.0 to 1.4.1 (#789)

* Bump mypy from 1.4.0 to 1.4.1

Bumps [mypy](https://github.com/python/mypy) from 1.4.0 to 1.4.1.
- [Commits](https://github.com/python/mypy/compare/v1.4.0...v1.4.1)

---
updated-dependencies:
- dependency-name: mypy
  dependency-type: direct:development
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230626-011544.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230626-011544.yaml

diff --git a/.changes/unreleased/Dependencies-20230626-011544.yaml b/.changes/unreleased/Dependencies-20230626-011544.yaml
new file mode 100644
index 000000000..c5f297a41
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230626-011544.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump mypy from 1.4.0 to 1.4.1"
+time: 2023-06-26T01:15:45.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 789
diff --git a/dev-requirements.txt b/dev-requirements.txt
index f8d3aa591..2e24a53cb 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -12,7 +12,7 @@ flake8~=6.0
 flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
-mypy==1.4.0  # patch updates have historically introduced breaking changes
+mypy==1.4.1  # patch updates have historically introduced breaking changes
 pip-tools~=6.13
 pre-commit~=3.3
 pre-commit-hooks~=4.4

From 6238749a514617b5160a51cd7f3f25b8f7f744ab Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 26 Jun 2023 17:19:54 -0400
Subject: [PATCH 654/860] Update pytest requirement from ~=7.3 to ~=7.4 (#788)

* Update pytest requirement from ~=7.3 to ~=7.4

Updates the requirements on [pytest](https://github.com/pytest-dev/pytest) to permit the latest version.
- [Release notes](https://github.com/pytest-dev/pytest/releases)
- [Changelog](https://github.com/pytest-dev/pytest/blob/main/CHANGELOG.rst)
- [Commits](https://github.com/pytest-dev/pytest/compare/7.3.0...7.4.0)

---
updated-dependencies:
- dependency-name: pytest
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230626-011535.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230626-011535.yaml

diff --git a/.changes/unreleased/Dependencies-20230626-011535.yaml b/.changes/unreleased/Dependencies-20230626-011535.yaml
new file mode 100644
index 000000000..150d56534
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230626-011535.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pytest requirement from ~=7.3 to ~=7.4"
+time: 2023-06-26T01:15:35.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 788
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 2e24a53cb..ae7e43a12 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -16,7 +16,7 @@ mypy==1.4.1  # patch updates have historically introduced breaking changes
 pip-tools~=6.13
 pre-commit~=3.3
 pre-commit-hooks~=4.4
-pytest~=7.3
+pytest~=7.4
 pytest-csv~=3.0
 pytest-dotenv~=0.5.2
 pytest-logbook~=1.2

From 85efa2a9bb803a070c4ff3a78d30c31f3fc6eb97 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Mon, 26 Jun 2023 15:40:35 -0700
Subject: [PATCH 655/860] Avoid unnecessary grant call (#791)

* remove call to dataset update if dataset has not changed

* add changie

* fix unit test naming

* seperate entry check from update

---------

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .../unreleased/Fixes-20230626-105156.yaml     |  6 +++
 dbt/adapters/bigquery/dataset.py              | 24 +++++++--
 dbt/adapters/bigquery/impl.py                 | 10 ++--
 tests/unit/test_dataset.py                    | 51 ++++++++++++++++---
 4 files changed, 77 insertions(+), 14 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230626-105156.yaml

diff --git a/.changes/unreleased/Fixes-20230626-105156.yaml b/.changes/unreleased/Fixes-20230626-105156.yaml
new file mode 100644
index 000000000..d1c6b9e25
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230626-105156.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: remove call to dataset update if dataset has not changed
+time: 2023-06-26T10:51:56.698483-07:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "770"
diff --git a/dbt/adapters/bigquery/dataset.py b/dbt/adapters/bigquery/dataset.py
index ebffe1072..c886637d7 100644
--- a/dbt/adapters/bigquery/dataset.py
+++ b/dbt/adapters/bigquery/dataset.py
@@ -6,15 +6,15 @@
 logger = AdapterLogger("BigQuery")
 
 
-def add_access_entry_to_dataset(dataset: Dataset, access_entry: AccessEntry) -> Dataset:
-    """Idempotently adds an access entry to a dataset
+def is_access_entry_in_dataset(dataset: Dataset, access_entry: AccessEntry) -> bool:
+    """Check if the access entry already exists in the dataset.
 
     Args:
         dataset (Dataset): the dataset to be updated
         access_entry (AccessEntry): the access entry to be added to the dataset
 
     Returns:
-        Dataset
+        bool: True if entry exists in dataset, False otherwise
     """
     access_entries: List[AccessEntry] = dataset.access_entries
     # we can't simply check if an access entry is in the list as the current equality check
@@ -24,8 +24,22 @@ def add_access_entry_to_dataset(dataset: Dataset, access_entry: AccessEntry) ->
         entity_type_match = existing_entry.entity_type == access_entry.entity_type
         property_match = existing_entry._properties.items() <= access_entry._properties.items()
         if role_match and entity_type_match and property_match:
-            logger.warning(f"Access entry {access_entry} " f"already exists in dataset")
-            return dataset
+            return True
+    return False
+
+
+def add_access_entry_to_dataset(dataset: Dataset, access_entry: AccessEntry) -> Dataset:
+    """Adds an access entry to a dataset, always use access_entry_present_in_dataset to check
+    if the access entry already exists before calling this function.
+
+    Args:
+        dataset (Dataset): the dataset to be updated
+        access_entry (AccessEntry): the access entry to be added to the dataset
+
+    Returns:
+        Dataset: the updated dataset
+    """
+    access_entries: List[AccessEntry] = dataset.access_entries
     access_entries.append(access_entry)
     dataset.access_entries = access_entries
     return dataset
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 37ed2c2f1..5c965ca7c 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -25,7 +25,7 @@
 
 from dbt.adapters.bigquery.column import get_nested_column_data_types
 from dbt.adapters.bigquery.relation import BigQueryRelation
-from dbt.adapters.bigquery.dataset import add_access_entry_to_dataset
+from dbt.adapters.bigquery.dataset import add_access_entry_to_dataset, is_access_entry_in_dataset
 from dbt.adapters.bigquery import BigQueryColumn
 from dbt.adapters.bigquery import BigQueryConnectionManager
 from dbt.adapters.bigquery.python_submissions import (
@@ -899,8 +899,12 @@ def grant_access_to(self, entity, entity_type, role, grant_target_dict):
             dataset_ref = self.connections.dataset_ref(grant_target.project, grant_target.dataset)
             dataset = client.get_dataset(dataset_ref)
             access_entry = AccessEntry(role, entity_type, entity)
-            dataset = add_access_entry_to_dataset(dataset, access_entry)
-            client.update_dataset(dataset, ["access_entries"])
+            # only perform update if access entry not in dataset
+            if is_access_entry_in_dataset(dataset, access_entry):
+                logger.warning(f"Access entry {access_entry} " f"already exists in dataset")
+            else:
+                dataset = add_access_entry_to_dataset(dataset, access_entry)
+                client.update_dataset(dataset, ["access_entries"])
 
     @available.parse_none
     def get_dataset_location(self, relation):
diff --git a/tests/unit/test_dataset.py b/tests/unit/test_dataset.py
index 681593185..adb3964c6 100644
--- a/tests/unit/test_dataset.py
+++ b/tests/unit/test_dataset.py
@@ -1,10 +1,10 @@
-from dbt.adapters.bigquery.dataset import add_access_entry_to_dataset
+from dbt.adapters.bigquery.dataset import add_access_entry_to_dataset, is_access_entry_in_dataset
 from dbt.adapters.bigquery import BigQueryRelation
 
 from google.cloud.bigquery import Dataset, AccessEntry, DatasetReference
 
 
-def test_add_access_entry_to_dataset_idempotently_adds_entries():
+def test_add_access_entry_to_dataset_updates_dataset():
     database = "someDb"
     dataset = "someDataset"
     entity = BigQueryRelation.from_dict(
@@ -23,11 +23,9 @@ def test_add_access_entry_to_dataset_idempotently_adds_entries():
     access_entry = AccessEntry(None, "table", entity)
     dataset = add_access_entry_to_dataset(dataset, access_entry)
     assert access_entry in dataset.access_entries
-    dataset = add_access_entry_to_dataset(dataset, access_entry)
-    assert len(dataset.access_entries) == 1
 
 
-def test_add_access_entry_to_dataset_does_not_add_with_pre_existing_entries():
+def test_add_access_entry_to_dataset_updates_with_pre_existing_entries():
     database = "someOtherDb"
     dataset = "someOtherDataset"
     entity_2 = BigQueryRelation.from_dict(
@@ -48,4 +46,45 @@ def test_add_access_entry_to_dataset_does_not_add_with_pre_existing_entries():
     dataset.access_entries = [initial_entry]
     access_entry = AccessEntry(None, "view", entity_2)
     dataset = add_access_entry_to_dataset(dataset, access_entry)
-    assert len(dataset.access_entries) == 1
+    assert len(dataset.access_entries) == 2
+
+
+def test_is_access_entry_in_dataset_returns_true_if_entry_in_dataset():
+    database = "someDb"
+    dataset = "someDataset"
+    entity = BigQueryRelation.from_dict(
+        {
+            "type": None,
+            "path": {
+                "database": "test-project",
+                "schema": "test_schema",
+                "identifier": "my_table",
+            },
+            "quote_policy": {"identifier": False},
+        }
+    ).to_dict()
+    dataset_ref = DatasetReference(project=database, dataset_id=dataset)
+    dataset = Dataset(dataset_ref)
+    access_entry = AccessEntry(None, "table", entity)
+    dataset = add_access_entry_to_dataset(dataset, access_entry)
+    assert is_access_entry_in_dataset(dataset, access_entry)
+
+
+def test_is_access_entry_in_dataset_returns_false_if_entry_not_in_dataset():
+    database = "someDb"
+    dataset = "someDataset"
+    entity = BigQueryRelation.from_dict(
+        {
+            "type": None,
+            "path": {
+                "database": "test-project",
+                "schema": "test_schema",
+                "identifier": "my_table",
+            },
+            "quote_policy": {"identifier": False},
+        }
+    ).to_dict()
+    dataset_ref = DatasetReference(project=database, dataset_id=dataset)
+    dataset = Dataset(dataset_ref)
+    access_entry = AccessEntry(None, "table", entity)
+    assert not is_access_entry_in_dataset(dataset, access_entry)

From 030324b7b3d0b14c6895b109fc5ddd6c6f4f006b Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 27 Jun 2023 00:02:15 -0400
Subject: [PATCH 656/860] Update pytest-xdist requirement from ~=3.2 to ~=3.3
 (#790)

* Update pytest-xdist requirement from ~=3.2 to ~=3.3

Updates the requirements on [pytest-xdist](https://github.com/pytest-dev/pytest-xdist) to permit the latest version.
- [Changelog](https://github.com/pytest-dev/pytest-xdist/blob/master/CHANGELOG.rst)
- [Commits](https://github.com/pytest-dev/pytest-xdist/compare/v3.2.0...v3.3.1)

---
updated-dependencies:
- dependency-name: pytest-xdist
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230626-011550.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230626-011550.yaml

diff --git a/.changes/unreleased/Dependencies-20230626-011550.yaml b/.changes/unreleased/Dependencies-20230626-011550.yaml
new file mode 100644
index 000000000..658fbb9c5
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230626-011550.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pytest-xdist requirement from ~=3.2 to ~=3.3"
+time: 2023-06-26T01:15:50.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 790
diff --git a/dev-requirements.txt b/dev-requirements.txt
index ae7e43a12..6ecc64ae1 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -20,7 +20,7 @@ pytest~=7.4
 pytest-csv~=3.0
 pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
-pytest-xdist~=3.2
+pytest-xdist~=3.3
 pytz~=2023.3
 tox~=4.6
 types-pytz~=2023.3

From ff77927e9ab0b43e9c5b95cefac0846bdc82c849 Mon Sep 17 00:00:00 2001
From: Christophe Oudar <kayrnt@gmail.com>
Date: Wed, 28 Jun 2023 03:43:54 +0200
Subject: [PATCH 657/860] move job link logging just after the job submit
 (#697)

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Features-20230505-184427.yaml |  6 ++++++
 dbt/adapters/bigquery/connections.py              | 13 ++++++++++---
 2 files changed, 16 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Features-20230505-184427.yaml

diff --git a/.changes/unreleased/Features-20230505-184427.yaml b/.changes/unreleased/Features-20230505-184427.yaml
new file mode 100644
index 000000000..1267d02ab
--- /dev/null
+++ b/.changes/unreleased/Features-20230505-184427.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Move the BQ Job link after the job submission instead of job done
+time: 2023-05-05T18:44:27.939038+02:00
+custom:
+  Author: Kayrnt
+  Issue: "696"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index f726481e7..109e63c64 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -534,9 +534,6 @@ def execute(
         else:
             message = f"{code}"
 
-        if location is not None and job_id is not None and project_id is not None:
-            logger.debug(self._bq_job_link(location, project_id, job_id))
-
         response = BigQueryAdapterResponse(  # type: ignore[call-arg]
             _message=message,
             rows_affected=num_rows,
@@ -663,6 +660,16 @@ def _query_and_results(
         # Cannot reuse job_config if destination is set and ddl is used
         job_config = google.cloud.bigquery.QueryJobConfig(**job_params)
         query_job = client.query(query=sql, job_config=job_config, timeout=job_creation_timeout)
+
+        if (
+            query_job.location is not None
+            and query_job.job_id is not None
+            and query_job.project is not None
+        ):
+            logger.debug(
+                self._bq_job_link(query_job.location, query_job.project, query_job.job_id)
+            )
+
         iterator = query_job.result(max_results=limit, timeout=job_execution_timeout)
 
         return query_job, iterator

From 9caac588ad5ec5128426a01449ca6072c8456284 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 29 Jun 2023 13:42:06 -0500
Subject: [PATCH 658/860] remove dependent_projects_dict from utils.py (#800)

---
 .changes/unreleased/Fixes-20230628-162853.yaml | 6 ++++++
 tests/unit/utils.py                            | 1 -
 2 files changed, 6 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20230628-162853.yaml

diff --git a/.changes/unreleased/Fixes-20230628-162853.yaml b/.changes/unreleased/Fixes-20230628-162853.yaml
new file mode 100644
index 000000000..b00ca3063
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230628-162853.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Remove dependent_projects argument from PartialProject call in unit test
+time: 2023-06-28T16:28:53.408844-05:00
+custom:
+  Author: McKnight-42
+  Issue: "7955"
diff --git a/tests/unit/utils.py b/tests/unit/utils.py
index 4d6dee451..7cd2bb9ab 100644
--- a/tests/unit/utils.py
+++ b/tests/unit/utils.py
@@ -75,7 +75,6 @@ def project_from_dict(project, profile, packages=None, selectors=None, cli_vars=
         project_root=project_root,
         project_dict=project,
         packages_dict=packages,
-        dependent_projects_dict={},
         selectors_dict=selectors,
     )
     return partial.render(renderer)

From e3d34f8587f49fa9ffd7f330608b97a0a7e49c69 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 29 Jun 2023 15:06:26 -0500
Subject: [PATCH 659/860] add thread_id to seed files for hooks tests to be in
 line with change in core (#802)

---
 tests/functional/adapter/hooks/data/seed_model.sql | 3 ++-
 tests/functional/adapter/hooks/data/seed_run.sql   | 3 ++-
 2 files changed, 4 insertions(+), 2 deletions(-)

diff --git a/tests/functional/adapter/hooks/data/seed_model.sql b/tests/functional/adapter/hooks/data/seed_model.sql
index 9b5130953..ccea4830f 100644
--- a/tests/functional/adapter/hooks/data/seed_model.sql
+++ b/tests/functional/adapter/hooks/data/seed_model.sql
@@ -11,5 +11,6 @@ create table `{schema}.on_model_hook` (
     target_pass      STRING,
     target_threads   INTEGER,
     run_started_at   STRING,
-    invocation_id    STRING
+    invocation_id    STRING,
+    thread_id        STRING
 );
diff --git a/tests/functional/adapter/hooks/data/seed_run.sql b/tests/functional/adapter/hooks/data/seed_run.sql
index 4699b0132..b39ba4e2d 100644
--- a/tests/functional/adapter/hooks/data/seed_run.sql
+++ b/tests/functional/adapter/hooks/data/seed_run.sql
@@ -12,5 +12,6 @@ create table {schema}.on_run_hook (
     target_pass      STRING,
     target_threads   INTEGER,
     run_started_at   STRING,
-    invocation_id    STRING
+    invocation_id    STRING,
+    thread_id        STRING
 );

From 2182d3d4c77276d1be974bb6a6893ea9697ce188 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Wed, 5 Jul 2023 15:08:14 -0400
Subject: [PATCH 660/860] Update pip-tools requirement from ~=6.13 to ~=6.14
 (#809)

* Update pip-tools requirement from ~=6.13 to ~=6.14

Updates the requirements on [pip-tools](https://github.com/jazzband/pip-tools) to permit the latest version.
- [Release notes](https://github.com/jazzband/pip-tools/releases)
- [Changelog](https://github.com/jazzband/pip-tools/blob/main/CHANGELOG.md)
- [Commits](https://github.com/jazzband/pip-tools/compare/6.13.0...6.14.0)

---
updated-dependencies:
- dependency-name: pip-tools
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230703-000325.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230703-000325.yaml

diff --git a/.changes/unreleased/Dependencies-20230703-000325.yaml b/.changes/unreleased/Dependencies-20230703-000325.yaml
new file mode 100644
index 000000000..21a999121
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230703-000325.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pip-tools requirement from ~=6.13 to ~=6.14"
+time: 2023-07-03T00:03:25.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 809
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 6ecc64ae1..abf89fa2f 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -13,7 +13,7 @@ flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
 mypy==1.4.1  # patch updates have historically introduced breaking changes
-pip-tools~=6.13
+pip-tools~=6.14
 pre-commit~=3.3
 pre-commit-hooks~=4.4
 pytest~=7.4

From 2fb56817037ef3f4ad8723765ed2db862edfb897 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Fri, 7 Jul 2023 00:43:23 -0500
Subject: [PATCH 661/860] =?UTF-8?q?add=20clean=5Fup=20method=20for=20store?=
 =?UTF-8?q?=5Ftest=5Ffailures=20tests=20to=20stop=20hanging=20art=E2=80=A6?=
 =?UTF-8?q?=20(#803)?=
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

* add clean_up method for store_test_failures tests to stop hanging artifacts in bigquery

* rename clean_up to teardown and change scope

* re-add function scope
---
 .../test_store_test_failures.py                     | 13 +++++++++++++
 1 file changed, 13 insertions(+)

diff --git a/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py b/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
index fbc224529..d22dab3ae 100644
--- a/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
+++ b/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
@@ -1,9 +1,22 @@
+import pytest
 from dbt.tests.adapter.store_test_failures_tests.test_store_test_failures import (
     StoreTestFailuresBase,
 )
 
 
+TEST_AUDIT_SCHEMA_SUFFIX = "dbt_test__aud"
+
+
 class TestBigQueryStoreTestFailures(StoreTestFailuresBase):
+    @pytest.fixture(scope="function", autouse=True)
+    def teardown_method(self, project):
+        yield
+        relation = project.adapter.Relation.create(
+            database=project.database, schema=f"{project.test_schema}_{TEST_AUDIT_SCHEMA_SUFFIX}"
+        )
+
+        project.adapter.drop_schema(relation)
+
     def test_store_and_assert(self, project):
         self.run_tests_store_one_failure(project)
         self.run_tests_store_failures_and_assert(project)

From c36c90257b7699fa79d516b6632b84c5f50fcc2e Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Fri, 7 Jul 2023 08:08:54 -0600
Subject: [PATCH 662/860] Enable single-line `private_key` for BigQuery Service
 Account JSON Authentication (#625)

* Enable single-line `private_key` for BigQuery Service Account JSON Authentication

* Use post-init processing to handle single-line `private_key`s
---
 .changes/unreleased/Fixes-20230322-150627.yaml | 6 ++++++
 dbt/adapters/bigquery/connections.py           | 6 ++++++
 2 files changed, 12 insertions(+)
 create mode 100644 .changes/unreleased/Fixes-20230322-150627.yaml

diff --git a/.changes/unreleased/Fixes-20230322-150627.yaml b/.changes/unreleased/Fixes-20230322-150627.yaml
new file mode 100644
index 000000000..90ef354f9
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230322-150627.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Enable single-line `private_key` for BigQuery Service Account JSON Authentication
+time: 2023-03-22T15:06:27.761416-06:00
+custom:
+  Author: dbeatty10
+  Issue: "7164"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 109e63c64..06da1ff90 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -158,6 +158,12 @@ class BigQueryCredentials(Credentials):
         "timeout_seconds": "job_execution_timeout_seconds",
     }
 
+    def __post_init__(self):
+        if self.keyfile_json and "private_key" in self.keyfile_json:
+            self.keyfile_json["private_key"] = self.keyfile_json["private_key"].replace(
+                "\\n", "\n"
+            )
+
     @property
     def type(self):
         return "bigquery"

From ae2303265284612a8e957f30026fdfc24f260962 Mon Sep 17 00:00:00 2001
From: Nathaniel May <nathaniel.may@fishtownanalytics.com>
Date: Mon, 10 Jul 2023 13:20:50 -0400
Subject: [PATCH 663/860] update pr template (#813)

---
 .github/pull_request_template.md | 31 ++++++++++++++++++++-----------
 1 file changed, 20 insertions(+), 11 deletions(-)

diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md
index 33a1e4538..f3fe5ac83 100644
--- a/.github/pull_request_template.md
+++ b/.github/pull_request_template.md
@@ -1,26 +1,35 @@
-resolves #
+resolves #  
+[docs](https://github.com/dbt-labs/docs.getdbt.com/issues/new/choose) dbt-labs/docs.getdbt.com/#  
 
 <!---
   Include the number of the issue addressed by this PR above if applicable.
   PRs for code changes without an associated issue *will not be merged*.
   See CONTRIBUTING.md for more information.
 
-  Example:
-    resolves #1234
+  Include the number of the docs issue that was opened for this PR. If
+  this change has no user-facing implications, "N/A" suffices instead. New
+  docs tickets can be created by clicking the link above or by going to
+  https://github.com/dbt-labs/docs.getdbt.com/issues/new/choose.
 -->
 
-### Description
+### Problem
 
 <!---
-  Describe the Pull Request here. Add any references and info to help reviewers
-  understand your changes. Include any tradeoffs you considered.
+  Describe the problem this PR is solving. What is the application state
+  before this PR is merged?
+-->
+
+### Solution
+
+<!---
+  Describe the way this PR solves the above problem. Add as much detail as you
+  can to help reviewers understand your changes. Include any alternatives and
+  tradeoffs you considered.
 -->
 
 ### Checklist
 
-- [ ] I have read [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md) and understand what's expected of me
-- [ ] I have signed the [CLA](https://docs.getdbt.com/docs/contributor-license-agreements)
-- [ ] I have run this code in development and it appears to resolve the stated issue
+- [ ] I have read [the contributing guide](https://github.com/dbt-labs/dbt-core/blob/main/CONTRIBUTING.md) and understand what's expected of me  
+- [ ] I have run this code in development and it appears to resolve the stated issue  
 - [ ] This PR includes tests, or tests are not required/relevant for this PR
-- [ ] I have [opened an issue to add/update docs](https://github.com/dbt-labs/docs.getdbt.com/issues/new/choose), or docs changes are not required/relevant for this PR
-- [ ] I have run `changie new` to [create a changelog entry](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#Adding-CHANGELOG-Entry)
+- [ ] This PR has no interface changes (e.g. macros, cli, logs, json artifacts, config files, adapter interface, etc) or this PR has already received feedback and approval from Product or DX

From 627307b4e07af0f9a1e069361a9d7cdc748be521 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Mon, 10 Jul 2023 16:05:24 -0500
Subject: [PATCH 664/860] fix whitespace change (#814)

---
 .github/pull_request_template.md | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md
index f3fe5ac83..a3c340cc3 100644
--- a/.github/pull_request_template.md
+++ b/.github/pull_request_template.md
@@ -1,5 +1,5 @@
-resolves #  
-[docs](https://github.com/dbt-labs/docs.getdbt.com/issues/new/choose) dbt-labs/docs.getdbt.com/#  
+resolves #
+[docs](https://github.com/dbt-labs/docs.getdbt.com/issues/new/choose) dbt-labs/docs.getdbt.com/#
 
 <!---
   Include the number of the issue addressed by this PR above if applicable.
@@ -29,7 +29,7 @@ resolves #
 
 ### Checklist
 
-- [ ] I have read [the contributing guide](https://github.com/dbt-labs/dbt-core/blob/main/CONTRIBUTING.md) and understand what's expected of me  
-- [ ] I have run this code in development and it appears to resolve the stated issue  
+- [ ] I have read [the contributing guide](https://github.com/dbt-labs/dbt-core/blob/main/CONTRIBUTING.md) and understand what's expected of me
+- [ ] I have run this code in development and it appears to resolve the stated issue
 - [ ] This PR includes tests, or tests are not required/relevant for this PR
 - [ ] This PR has no interface changes (e.g. macros, cli, logs, json artifacts, config files, adapter interface, etc) or this PR has already received feedback and approval from Product or DX

From 2f80a27ead2ef79068ebceb14b48b4bdbcd73dc3 Mon Sep 17 00:00:00 2001
From: dave-connors-3 <73915542+dave-connors-3@users.noreply.github.com>
Date: Tue, 11 Jul 2023 10:21:12 -0500
Subject: [PATCH 665/860] Fix split part negative part number (#618)

* add part number to negative case

* Add a negative number (rather than subtract a positive number)

* Restore original dev-requirements.txt

---------

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20230320-153618.yaml   | 6 ++++++
 dbt/include/bigquery/macros/utils/split_part.sql | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20230320-153618.yaml

diff --git a/.changes/unreleased/Fixes-20230320-153618.yaml b/.changes/unreleased/Fixes-20230320-153618.yaml
new file mode 100644
index 000000000..e3f1fa011
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230320-153618.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: add negative part_number arg for split part macro
+time: 2023-03-20T15:36:18.858565-05:00
+custom:
+  Author: dave-connors-3
+  Issue: "615"
diff --git a/dbt/include/bigquery/macros/utils/split_part.sql b/dbt/include/bigquery/macros/utils/split_part.sql
index 2add716f8..a13f0d8ce 100644
--- a/dbt/include/bigquery/macros/utils/split_part.sql
+++ b/dbt/include/bigquery/macros/utils/split_part.sql
@@ -13,7 +13,7 @@
           length({{ string_text }})
           - length(
               replace({{ string_text }},  {{ delimiter_text }}, '')
-          ) + 1
+          ) + 1 + {{ part_number }}
         )]
   {% endif %}
 

From 291713c92263f6ce8749bb2b9bc7ea75cae9cf3b Mon Sep 17 00:00:00 2001
From: Michelle Ark <MichelleArk@users.noreply.github.com>
Date: Tue, 11 Jul 2023 09:36:42 -0700
Subject: [PATCH 666/860] Contracts: Handle struct column specified both at
 root and nested levels + arrays of structs (#806)

---
 .../unreleased/Fixes-20230630-213112.yaml     |   7 ++
 dbt/adapters/bigquery/column.py               |  77 +++++++++---
 dbt/adapters/bigquery/impl.py                 |   2 +-
 .../macros/utils/get_columns_spec_ddl.sql     |  10 ++
 tests/unit/test_column.py                     | 111 ++++++++++++++++++
 5 files changed, 189 insertions(+), 18 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230630-213112.yaml

diff --git a/.changes/unreleased/Fixes-20230630-213112.yaml b/.changes/unreleased/Fixes-20230630-213112.yaml
new file mode 100644
index 000000000..7238c0cb1
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230630-213112.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: 'Contracts: Handle struct column specified both at root and nested levels +
+  arrays of structs'
+time: 2023-06-30T21:31:12.63257-04:00
+custom:
+  Author: michelleark
+  Issue: 781 782
diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py
index cfa298b82..a5a60cfc0 100644
--- a/dbt/adapters/bigquery/column.py
+++ b/dbt/adapters/bigquery/column.py
@@ -5,6 +5,8 @@
 
 from google.cloud.bigquery import SchemaField
 
+_PARENT_DATA_TYPE_KEY = "__parent_data_type"
+
 Self = TypeVar("Self", bound="BigQueryColumn")
 
 
@@ -131,7 +133,7 @@ def column_to_bq_schema(self) -> SchemaField:
 def get_nested_column_data_types(
     columns: Dict[str, Dict[str, Any]],
     constraints: Optional[Dict[str, str]] = None,
-) -> Dict[str, Dict[str, str]]:
+) -> Dict[str, Dict[str, Optional[str]]]:
     """
     columns:
         * Dictionary where keys are of flat columns names and values are dictionary of column attributes
@@ -159,16 +161,16 @@ def get_nested_column_data_types(
     """
     constraints = constraints or {}
 
-    nested_column_data_types: Dict[str, Union[str, Dict]] = {}
+    nested_column_data_types: Dict[str, Optional[Union[str, Dict]]] = {}
     for column in columns.values():
         _update_nested_column_data_types(
             column["name"],
-            column["data_type"],
+            column.get("data_type"),
             constraints.get(column["name"]),
             nested_column_data_types,
         )
 
-    formatted_nested_column_data_types: Dict[str, Dict[str, str]] = {}
+    formatted_nested_column_data_types: Dict[str, Dict[str, Optional[str]]] = {}
     for column_name, unformatted_column_type in nested_column_data_types.items():
         formatted_nested_column_data_types[column_name] = {
             "name": column_name,
@@ -191,9 +193,9 @@ def get_nested_column_data_types(
 
 def _update_nested_column_data_types(
     column_name: str,
-    column_data_type: str,
+    column_data_type: Optional[str],
     column_rendered_constraint: Optional[str],
-    nested_column_data_types: Dict[str, Union[str, Dict]],
+    nested_column_data_types: Dict[str, Optional[Union[str, Dict]]],
 ) -> None:
     """
     Recursively update nested_column_data_types given a column_name, column_data_type, and optional column_rendered_constraint.
@@ -215,15 +217,38 @@ def _update_nested_column_data_types(
 
     if len(column_name_parts) == 1:
         # Base case: column is not nested - store its data_type concatenated with constraint if provided.
-        nested_column_data_types[root_column_name] = (
-            column_data_type
-            if column_rendered_constraint is None
-            else f"{column_data_type} {column_rendered_constraint}"
+        column_data_type_and_constraints = (
+            (
+                column_data_type
+                if column_rendered_constraint is None
+                else f"{column_data_type} {column_rendered_constraint}"
+            )
+            if column_data_type
+            else None
         )
+
+        if existing_nested_column_data_type := nested_column_data_types.get(root_column_name):
+            assert isinstance(existing_nested_column_data_type, dict)  # keeping mypy happy
+            # entry could already exist if this is a parent column -- preserve the parent data type under "_PARENT_DATA_TYPE_KEY"
+            existing_nested_column_data_type.update(
+                {_PARENT_DATA_TYPE_KEY: column_data_type_and_constraints}
+            )
+        else:
+            nested_column_data_types.update({root_column_name: column_data_type_and_constraints})
     else:
-        # Initialize nested dictionary
-        if root_column_name not in nested_column_data_types:
-            nested_column_data_types[root_column_name] = {}
+        parent_data_type = nested_column_data_types.get(root_column_name)
+        if isinstance(parent_data_type, dict):
+            # nested dictionary already initialized
+            pass
+        elif parent_data_type is None:
+            # initialize nested dictionary
+            nested_column_data_types.update({root_column_name: {}})
+        else:
+            # a parent specified its base type -- preserve its data_type and potential rendered constraints
+            # this is used to specify a top-level 'struct' or 'array' field with its own description, constraints, etc
+            nested_column_data_types.update(
+                {root_column_name: {_PARENT_DATA_TYPE_KEY: parent_data_type}}
+            )
 
         # Recursively process rest of remaining column name
         remaining_column_name = ".".join(column_name_parts[1:])
@@ -237,7 +262,9 @@ def _update_nested_column_data_types(
         )
 
 
-def _format_nested_data_type(unformatted_nested_data_type: Union[str, Dict[str, Any]]) -> str:
+def _format_nested_data_type(
+    unformatted_nested_data_type: Optional[Union[str, Dict[str, Any]]]
+) -> Optional[str]:
     """
     Recursively format a (STRUCT) data type given an arbitrarily nested data type structure.
 
@@ -249,11 +276,27 @@ def _format_nested_data_type(unformatted_nested_data_type: Union[str, Dict[str,
     >>> BigQueryAdapter._format_nested_data_type({'c': 'string not_null', 'd': {'e': 'string'}})
     'struct<c string not_null, d struct<e string>>'
     """
-    if isinstance(unformatted_nested_data_type, str):
+    if unformatted_nested_data_type is None:
+        return None
+    elif isinstance(unformatted_nested_data_type, str):
         return unformatted_nested_data_type
     else:
+        parent_data_type, *parent_constraints = unformatted_nested_data_type.pop(
+            _PARENT_DATA_TYPE_KEY, ""
+        ).split() or [None]
+
         formatted_nested_types = [
-            f"{column_name} {_format_nested_data_type(column_type)}"
+            f"{column_name} {_format_nested_data_type(column_type) or ''}".strip()
             for column_name, column_type in unformatted_nested_data_type.items()
         ]
-        return f"""struct<{", ".join(formatted_nested_types)}>"""
+
+        formatted_nested_type = f"""struct<{", ".join(formatted_nested_types)}>"""
+
+        if parent_data_type and parent_data_type.lower() == "array":
+            formatted_nested_type = f"""array<{formatted_nested_type}>"""
+
+        if parent_constraints:
+            parent_constraints = " ".join(parent_constraints)
+            formatted_nested_type = f"""{formatted_nested_type} {parent_constraints}"""
+
+        return formatted_nested_type
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 5c965ca7c..353be08d8 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -300,7 +300,7 @@ def nest_column_data_types(
         cls,
         columns: Dict[str, Dict[str, Any]],
         constraints: Optional[Dict[str, str]] = None,
-    ) -> Dict[str, Dict[str, str]]:
+    ) -> Dict[str, Dict[str, Optional[str]]]:
         return get_nested_column_data_types(columns, constraints)
 
     def get_columns_in_relation(self, relation: BigQueryRelation) -> List[BigQueryColumn]:
diff --git a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
index a1247dcd6..1a4193c71 100644
--- a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
+++ b/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
@@ -5,6 +5,16 @@
 {%- endmacro -%}
 
 {% macro bigquery__get_empty_schema_sql(columns) %}
+    {%- set col_err = [] -%}
+    {% for col in columns.values() %}
+      {%- if col['data_type'] is not defined -%}
+        {{ col_err.append(col['name']) }}
+      {%- endif -%}
+    {%- endfor -%}
+    {%- if (col_err | length) > 0 -%}
+      {{ exceptions.column_type_missing(column_names=col_err) }}
+    {%- endif -%}
+
     {%- set columns = adapter.nest_column_data_types(columns) -%}
     {{ return(dbt.default__get_empty_schema_sql(columns)) }}
 {% endmacro %}
diff --git a/tests/unit/test_column.py b/tests/unit/test_column.py
index 8c3c645bb..10f30594e 100644
--- a/tests/unit/test_column.py
+++ b/tests/unit/test_column.py
@@ -14,6 +14,12 @@
             None,
             {"a": {"name": "a", "data_type": "string"}},
         ),
+        # Flat column - missing data_type
+        (
+            {"a": {"name": "a"}},
+            None,
+            {"a": {"name": "a", "data_type": None}},
+        ),
         # Flat column - with constraints
         (
             {"a": {"name": "a", "data_type": "string"}},
@@ -32,18 +38,75 @@
             None,
             {"b": {"name": "b", "data_type": "struct<nested string>"}},
         ),
+        # Single nested column, 1 level - missing data_type
+        (
+            {"b.nested": {"name": "b.nested"}},
+            None,
+            {"b": {"name": "b", "data_type": "struct<nested>"}},
+        ),
         # Single nested column, 1 level - with constraints
         (
             {"b.nested": {"name": "b.nested", "data_type": "string"}},
             {"b.nested": "not null"},
             {"b": {"name": "b", "data_type": "struct<nested string not null>"}},
         ),
+        # Single nested column, 1 level - with constraints, missing data_type (constraints not valid without data_type)
+        (
+            {"b.nested": {"name": "b.nested"}},
+            {"b.nested": "not null"},
+            {"b": {"name": "b", "data_type": "struct<nested>"}},
+        ),
         # Single nested column, 1 level - with constraints + other keys
         (
             {"b.nested": {"name": "b.nested", "data_type": "string", "other": "unpreserved"}},
             {"b.nested": "not null"},
             {"b": {"name": "b", "data_type": "struct<nested string not null>"}},
         ),
+        # Single nested column, 1 level - with corresponding parent column
+        (
+            {
+                "b": {"name": "b", "data_type": "struct"},
+                "b.nested": {"name": "b.nested", "data_type": "string"},
+            },
+            None,
+            {"b": {"name": "b", "data_type": "struct<nested string>"}},
+        ),
+        # Single nested column, 1 level - with corresponding parent column specified last
+        (
+            {
+                "b.nested": {"name": "b.nested", "data_type": "string"},
+                "b": {"name": "b", "data_type": "struct"},
+            },
+            None,
+            {"b": {"name": "b", "data_type": "struct<nested string>"}},
+        ),
+        # Single nested column, 1 level - with corresponding parent column + parent constraint
+        (
+            {
+                "b": {"name": "b", "data_type": "struct"},
+                "b.nested": {"name": "b.nested", "data_type": "string"},
+            },
+            {"b": "not null"},
+            {"b": {"name": "b", "data_type": "struct<nested string> not null"}},
+        ),
+        # Single nested column, 1 level - with corresponding parent column as array
+        (
+            {
+                "b": {"name": "b", "data_type": "array"},
+                "b.nested": {"name": "b.nested", "data_type": "string"},
+            },
+            None,
+            {"b": {"name": "b", "data_type": "array<struct<nested string>>"}},
+        ),
+        # Single nested column, 1 level - with corresponding parent column as array + constraint
+        (
+            {
+                "b": {"name": "b", "data_type": "array"},
+                "b.nested": {"name": "b.nested", "data_type": "string"},
+            },
+            {"b": "not null"},
+            {"b": {"name": "b", "data_type": "array<struct<nested string>> not null"}},
+        ),
         # Multiple nested columns, 1 level
         (
             {
@@ -106,6 +169,28 @@
                 },
             },
         ),
+        # Nested columns, multiple levels - missing data_type
+        (
+            {
+                "b.user.name.first": {
+                    "name": "b.user.name.first",
+                    "data_type": "string",
+                },
+                "b.user.name.last": {
+                    "name": "b.user.name.last",
+                    "data_type": "string",
+                },
+                "b.user.id": {"name": "b.user.id", "data_type": "int64"},
+                "b.user.country": {"name": "b.user.country"},  # missing data_type
+            },
+            None,
+            {
+                "b": {
+                    "name": "b",
+                    "data_type": "struct<user struct<name struct<first string, last string>, id int64, country>>",
+                },
+            },
+        ),
         # Nested columns, multiple levels - with constraints!
         (
             {
@@ -128,6 +213,32 @@
                 },
             },
         ),
+        # Nested columns, multiple levels - with parent arrays and constraints!
+        (
+            {
+                "b.user.names": {
+                    "name": "b.user.names",
+                    "data_type": "array",
+                },
+                "b.user.names.first": {
+                    "name": "b.user.names.first",
+                    "data_type": "string",
+                },
+                "b.user.names.last": {
+                    "name": "b.user.names.last",
+                    "data_type": "string",
+                },
+                "b.user.id": {"name": "b.user.id", "data_type": "int64"},
+                "b.user.country": {"name": "b.user.country", "data_type": "string"},
+            },
+            {"b.user.names.first": "not null", "b.user.id": "unique"},
+            {
+                "b": {
+                    "name": "b",
+                    "data_type": "struct<user struct<names array<struct<first string not null, last string>>, id int64 unique, country string>>",
+                },
+            },
+        ),
     ],
 )
 def test_get_nested_column_data_types(columns, constraints, expected_nested_columns):

From 3c34dbb4ec294767b563ed12cf4525ba00c8e308 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Tue, 11 Jul 2023 15:35:34 -0500
Subject: [PATCH 667/860] add macros and adapter test for new dbt_clone flag
 (#784)

* add macros and adapter test for new dbt_clone flag

* add changelog entry

* add clean_up method to drop alt schema names after tests run

* change pointer for dev-requirements and update to main
---
 .../unreleased/Features-20230622-113645.yaml  |  6 ++++++
 .../macros/materializations/clone.sql         |  9 +++++++++
 .../adapter/dbt_clone/test_dbt_clone.py       | 20 +++++++++++++++++++
 3 files changed, 35 insertions(+)
 create mode 100644 .changes/unreleased/Features-20230622-113645.yaml
 create mode 100644 dbt/include/bigquery/macros/materializations/clone.sql
 create mode 100644 tests/functional/adapter/dbt_clone/test_dbt_clone.py

diff --git a/.changes/unreleased/Features-20230622-113645.yaml b/.changes/unreleased/Features-20230622-113645.yaml
new file mode 100644
index 000000000..b1ab93f96
--- /dev/null
+++ b/.changes/unreleased/Features-20230622-113645.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: add dbt-bigquery portion of dbt_clone fucntionality
+time: 2023-06-22T11:36:45.348416-05:00
+custom:
+  Author: McKnight-42, aranke
+  Issue: "7256"
diff --git a/dbt/include/bigquery/macros/materializations/clone.sql b/dbt/include/bigquery/macros/materializations/clone.sql
new file mode 100644
index 000000000..3964be2b3
--- /dev/null
+++ b/dbt/include/bigquery/macros/materializations/clone.sql
@@ -0,0 +1,9 @@
+{% macro bigquery__can_clone_table() %}
+    {{ return(True) }}
+{% endmacro %}
+
+{% macro bigquery__create_or_replace_clone(this_relation, defer_relation) %}
+    create or replace
+      table {{ this_relation }}
+      clone {{ defer_relation }}
+{% endmacro %}
diff --git a/tests/functional/adapter/dbt_clone/test_dbt_clone.py b/tests/functional/adapter/dbt_clone/test_dbt_clone.py
new file mode 100644
index 000000000..189a3f067
--- /dev/null
+++ b/tests/functional/adapter/dbt_clone/test_dbt_clone.py
@@ -0,0 +1,20 @@
+import pytest
+from dbt.tests.adapter.dbt_clone.test_dbt_clone import BaseClonePossible
+
+
+class TestBigQueryClonePossible(BaseClonePossible):
+    @pytest.fixture(autouse=True)
+    def clean_up(self, project):
+        yield
+        with project.adapter.connection_named("__test"):
+            relation = project.adapter.Relation.create(
+                database=project.database, schema=f"{project.test_schema}_seeds"
+            )
+            project.adapter.drop_schema(relation)
+
+            relation = project.adapter.Relation.create(
+                database=project.database, schema=project.test_schema
+            )
+            project.adapter.drop_schema(relation)
+
+    pass

From d0d593eafe93c32eb3e6ae32d4d865efa499c398 Mon Sep 17 00:00:00 2001
From: Thomas Lento <tlento@users.noreply.github.com>
Date: Wed, 12 Jul 2023 09:27:06 -0700
Subject: [PATCH 668/860] Add support for validate_sql method to BigQuery
 (#819)

In CLI contexts MetricFlow will issue dry run queries as part
of its warehouse validation operations, and so we are adding a
validate_sql method to all adapters.

This commit adds support for the validate_sql method to BigQuery. It
does so by creating a BigQuery-specific `dry_run` method on the
BigQueryConnectionManager. This simply passes through the input SQL
with the `dry_run` QueryJobParameter flag set True. This will result
in BigQuery computing and returning a cost estimate for the query,
or raising an exception in the event the query is not valid.

Note: constructing the response object involves some repetitive value
extraction from the QueryResult returned by BigQuery. While I would
ordinariy prefer to tidy this up first we are pressed for time, and so
we postpone that cleanup in order to keep this change as isolated
as possible.
---
 .../unreleased/Features-20230712-014350.yaml  |  6 +++
 dbt/adapters/bigquery/connections.py          | 45 ++++++++++++++++++-
 dbt/adapters/bigquery/impl.py                 | 10 +++++
 tests/functional/adapter/utils/test_utils.py  | 27 +++++++++++
 4 files changed, 86 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Features-20230712-014350.yaml

diff --git a/.changes/unreleased/Features-20230712-014350.yaml b/.changes/unreleased/Features-20230712-014350.yaml
new file mode 100644
index 000000000..9bd47f49b
--- /dev/null
+++ b/.changes/unreleased/Features-20230712-014350.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Add validate_sql to BigQuery adapter and dry_run to BigQueryConnectionManager
+time: 2023-07-12T01:43:50.36167-04:00
+custom:
+  Author: tlento
+  Issue: "805"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 06da1ff90..8662da1de 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -428,7 +428,13 @@ def get_table_from_response(cls, resp):
         column_names = [field.name for field in resp.schema]
         return agate_helper.table_from_data_flat(resp, column_names)
 
-    def raw_execute(self, sql, use_legacy_sql=False, limit: Optional[int] = None):
+    def raw_execute(
+        self,
+        sql,
+        use_legacy_sql=False,
+        limit: Optional[int] = None,
+        dry_run: bool = False,
+    ):
         conn = self.get_thread_connection()
         client = conn.handle
 
@@ -446,7 +452,11 @@ def raw_execute(self, sql, use_legacy_sql=False, limit: Optional[int] = None):
         if active_user:
             labels["dbt_invocation_id"] = active_user.invocation_id
 
-        job_params = {"use_legacy_sql": use_legacy_sql, "labels": labels}
+        job_params = {
+            "use_legacy_sql": use_legacy_sql,
+            "labels": labels,
+            "dry_run": dry_run,
+        }
 
         priority = conn.credentials.priority
         if priority == Priority.Batch:
@@ -554,6 +564,37 @@ def execute(
 
         return response, table
 
+    def dry_run(self, sql: str) -> BigQueryAdapterResponse:
+        """Run the given sql statement with the `dry_run` job parameter set.
+
+        This will allow BigQuery to validate the SQL and immediately return job cost
+        estimates, which we capture in the BigQueryAdapterResponse. Invalid SQL
+        will result in an exception.
+        """
+        sql = self._add_query_comment(sql)
+        query_job, _ = self.raw_execute(sql, dry_run=True)
+
+        # TODO: Factor this repetitive block out into a factory method on
+        # BigQueryAdapterResponse
+        message = f"Ran dry run query for statement of type {query_job.statement_type}"
+        bytes_billed = query_job.total_bytes_billed
+        processed_bytes = self.format_bytes(query_job.total_bytes_processed)
+        location = query_job.location
+        project_id = query_job.project
+        job_id = query_job.job_id
+        slot_ms = query_job.slot_millis
+
+        return BigQueryAdapterResponse(
+            _message=message,
+            code="DRY RUN",
+            bytes_billed=bytes_billed,
+            bytes_processed=processed_bytes,
+            location=location,
+            project_id=project_id,
+            job_id=job_id,
+            slot_ms=slot_ms,
+        )
+
     @staticmethod
     def _bq_job_link(location, project_id, job_id) -> str:
         return f"https://console.cloud.google.com/bigquery?project={project_id}&j=bq:{location}:{job_id}&page=queryresults"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 353be08d8..f53cd4084 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -2,6 +2,7 @@
 import threading
 from typing import Dict, List, Optional, Any, Set, Union, Type
 
+from dbt.contracts.connection import AdapterResponse
 from dbt.contracts.graph.nodes import ColumnLevelConstraint, ModelLevelConstraint, ConstraintType  # type: ignore
 from dbt.dataclass_schema import dbtClassMixin, ValidationError
 
@@ -1024,3 +1025,12 @@ def render_model_constraint(cls, constraint: ModelLevelConstraint) -> Optional[s
     def debug_query(self):
         """Override for DebugTask method"""
         self.execute("select 1 as id")
+
+    def validate_sql(self, sql: str) -> AdapterResponse:
+        """Submit the given SQL to the engine for validation, but not execution.
+
+        This submits the query with the `dry_run` flag set True.
+
+        :param str sql: The sql to validate
+        """
+        return self.connections.dry_run(sql)
diff --git a/tests/functional/adapter/utils/test_utils.py b/tests/functional/adapter/utils/test_utils.py
index 6fb2d05d2..dc42c4db0 100644
--- a/tests/functional/adapter/utils/test_utils.py
+++ b/tests/functional/adapter/utils/test_utils.py
@@ -1,4 +1,7 @@
+import random
+
 import pytest
+from google.api_core.exceptions import NotFound
 
 from dbt.tests.adapter.utils.test_array_append import BaseArrayAppend
 from dbt.tests.adapter.utils.test_array_concat import BaseArrayConcat
@@ -24,6 +27,7 @@
 from dbt.tests.adapter.utils.test_safe_cast import BaseSafeCast
 from dbt.tests.adapter.utils.test_split_part import BaseSplitPart
 from dbt.tests.adapter.utils.test_string_literal import BaseStringLiteral
+from dbt.tests.adapter.utils.test_validate_sql import BaseValidateSqlMethod
 from tests.functional.adapter.utils.fixture_array_append import (
     models__array_append_actual_sql,
     models__array_append_expected_sql,
@@ -167,3 +171,26 @@ class TestSplitPart(BaseSplitPart):
 
 class TestStringLiteral(BaseStringLiteral):
     pass
+
+
+class TestValidateSqlMethod(BaseValidateSqlMethod):
+    pass
+
+
+class TestDryRunMethod:
+    """Test connection manager dry run method operation."""
+
+    def test_dry_run_method(self, project) -> None:
+        """Test dry run method on a DDL statement.
+
+        This allows us to demonstrate that no SQL is executed.
+        """
+        with project.adapter.connection_named("_test"):
+            client = project.adapter.connections.get_thread_connection().handle
+            random_suffix = "".join(random.choices([str(i) for i in range(10)], k=10))
+            table_name = f"test_dry_run_{random_suffix}"
+            table_id = "{}.{}.{}".format(project.database, project.test_schema, table_name)
+            res = project.adapter.connections.dry_run(f"CREATE TABLE {table_id} (x INT64)")
+            assert res.code == "DRY RUN"
+            with pytest.raises(expected_exception=NotFound):
+                client.get_table(table_id)

From e202bb878cdcc6c24a40c997aa7b937024cc75f5 Mon Sep 17 00:00:00 2001
From: Anders <anders.swanson@dbtlabs.com>
Date: Thu, 13 Jul 2023 13:34:05 -0400
Subject: [PATCH 669/860] Revert "Add `batch_id` param to profile  to pass
 along with `dataproc_v1.CreateBatchRequest` (#727)" (#826)

This reverts commit a42e13d8023123102c4d1aeefdfa6cd6246c4768.
---
 .changes/unreleased/Features-20230517-092205.yaml | 6 ------
 dbt/adapters/bigquery/connections.py              | 1 -
 dbt/adapters/bigquery/python_submissions.py       | 1 -
 test.env.example                                  | 1 -
 tests/conftest.py                                 | 2 --
 5 files changed, 11 deletions(-)
 delete mode 100644 .changes/unreleased/Features-20230517-092205.yaml

diff --git a/.changes/unreleased/Features-20230517-092205.yaml b/.changes/unreleased/Features-20230517-092205.yaml
deleted file mode 100644
index 9afde587d..000000000
--- a/.changes/unreleased/Features-20230517-092205.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Add batch_id param to profile
-time: 2023-05-17T09:22:05.264368+01:00
-custom:
-  Author: nickozilla
-  Issue: "671"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 8662da1de..b466fee3b 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -133,7 +133,6 @@ class BigQueryCredentials(Credentials):
     dataproc_region: Optional[str] = None
     dataproc_cluster_name: Optional[str] = None
     gcs_bucket: Optional[str] = None
-    batch_id: Optional[str] = None
 
     dataproc_batch: Optional[DataprocBatchConfig] = field(
         metadata={
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 29e33032d..0c7ce1917 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -122,7 +122,6 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         request = dataproc_v1.CreateBatchRequest(
             parent=parent,
             batch=batch,
-            batch_id=self.credential.batch_id,
         )
         # make the request
         operation = self.job_client.create_batch(request=request)  # type: ignore
diff --git a/test.env.example b/test.env.example
index 70befbbbb..ffe9ee060 100644
--- a/test.env.example
+++ b/test.env.example
@@ -15,4 +15,3 @@ DBT_TEST_USER_3="serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserv
 DATAPROC_REGION=us-
 DATAPROC_CLUSTER_NAME=
 GCS_BUCKET=
-BATCH_ID=
diff --git a/tests/conftest.py b/tests/conftest.py
index b5a967a09..0ba0091fb 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -33,7 +33,6 @@ def oauth_target():
         "dataproc_region": os.getenv("DATAPROC_REGION"),
         "dataproc_cluster_name": os.getenv("DATAPROC_CLUSTER_NAME"),
         "gcs_bucket": os.getenv("GCS_BUCKET"),
-        "batch_id": os.getenv("BATCH_ID"),
     }
 
 
@@ -54,5 +53,4 @@ def service_account_target():
             "DATAPROC_CLUSTER_NAME"
         ),  # only needed for cluster submission method
         "gcs_bucket": os.getenv("GCS_BUCKET"),
-        "batch_id": os.getenv("BATCH_ID"),
     }

From 1d4896733de69769f8e35866bd859bd4cb67dae3 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 13 Jul 2023 11:01:28 -0700
Subject: [PATCH 670/860] add input to allow testing against a non-main core
 branch (#823)

* add input to allow testing against a non-main core branch

* add shell script
---
 .github/scripts/update_dbt_core_branch.sh | 20 ++++++++++++++++++++
 .github/workflows/integration.yml         | 11 +++++++++++
 2 files changed, 31 insertions(+)
 create mode 100755 .github/scripts/update_dbt_core_branch.sh

diff --git a/.github/scripts/update_dbt_core_branch.sh b/.github/scripts/update_dbt_core_branch.sh
new file mode 100755
index 000000000..d28a40c35
--- /dev/null
+++ b/.github/scripts/update_dbt_core_branch.sh
@@ -0,0 +1,20 @@
+#!/bin/bash -e
+set -e
+
+git_branch=$1
+target_req_file="dev-requirements.txt"
+core_req_sed_pattern="s|dbt-core.git.*#egg=dbt-core|dbt-core.git@${git_branch}#egg=dbt-core|g"
+postgres_req_sed_pattern="s|dbt-core.git.*#egg=dbt-postgres|dbt-core.git@${git_branch}#egg=dbt-postgres|g"
+tests_req_sed_pattern="s|dbt-core.git.*#egg=dbt-tests|dbt-core.git@${git_branch}#egg=dbt-tests|g"
+if [[ "$OSTYPE" == darwin* ]]; then
+ # mac ships with a different version of sed that requires a delimiter arg
+ sed -i "" "$core_req_sed_pattern" $target_req_file
+ sed -i "" "$postgres_req_sed_pattern" $target_req_file
+ sed -i "" "$tests_req_sed_pattern" $target_req_file
+else
+ sed -i "$core_req_sed_pattern" $target_req_file
+ sed -i "$postgres_req_sed_pattern" $target_req_file
+ sed -i "$tests_req_sed_pattern" $target_req_file
+fi
+core_version=$(curl "https://raw.githubusercontent.com/dbt-labs/dbt-core/${git_branch}/core/dbt/version.py" | grep "__version__ = *"|cut -d'=' -f2)
+bumpversion --allow-dirty --new-version "$core_version" major
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index a155701f2..57ea6c9ed 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -33,6 +33,11 @@ on:
   pull_request_target:
   # manual trigger
   workflow_dispatch:
+    inputs:
+      dbt-core-branch:
+        description: "branch of dbt-core to use in dev-requirements.txt"
+        required: false
+        type: string
 
 # explicitly turn off permissions for `GITHUB_TOKEN`
 permissions: read-all
@@ -156,6 +161,12 @@ jobs:
           python -m pip --version
           tox --version
 
+      - name: Update dev_requirements.txt
+        if: inputs.dbt-core-branch != ''
+        run: |
+          pip install bumpversion
+          ./.github/scripts/update_dbt_core_branch.sh ${{ inputs.dbt-core-branch }}
+
       - name: Run tox (bigquery)
         if: matrix.adapter == 'bigquery'
         env:

From d3deeb8077ea26c9bfdf9e7ae58d20b808104081 Mon Sep 17 00:00:00 2001
From: Christophe Oudar
 <90898980+github-christophe-oudar@users.noreply.github.com>
Date: Fri, 14 Jul 2023 02:03:38 +0200
Subject: [PATCH 671/860] Use tmp table in static insert overwrite (#630)

* Use tmp table in static insert overwrite

* review changes

* Update .changes/unreleased/Fixes-20230325-204352.yaml

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>

* add test for static overwrite day case

---------

Co-authored-by: Christophe Oudar <kayrnt@gmail.com>
Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
---
 .../unreleased/Fixes-20230325-204352.yaml     |  6 +++
 .../incremental_strategy/insert_overwrite.sql | 34 ++++++++++----
 .../incremental_strategy_fixtures.py          | 47 +++++++++++++++++++
 .../test_incremental_strategies.py            |  7 ++-
 4 files changed, 82 insertions(+), 12 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230325-204352.yaml

diff --git a/.changes/unreleased/Fixes-20230325-204352.yaml b/.changes/unreleased/Fixes-20230325-204352.yaml
new file mode 100644
index 000000000..8f213c703
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230325-204352.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Use tmp table in static insert overwrite to avoid computing the SQL twice
+time: 2023-03-25T20:43:52.830135+01:00
+custom:
+  Author: Kayrnt
+  Issue: 427 556
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
index 41c11c15d..fd7d40e0f 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
@@ -1,5 +1,5 @@
 {% macro bq_generate_incremental_insert_overwrite_build_sql(
-    tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, on_schema_change, copy_partitions
+    tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions
 ) %}
     {% if partition_by is none %}
       {% set missing_partition_msg -%}
@@ -9,7 +9,7 @@
     {% endif %}
 
     {% set build_sql = bq_insert_overwrite_sql(
-        tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, on_schema_change, copy_partitions
+        tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions
     ) %}
 
     {{ return(build_sql) }}
@@ -39,14 +39,14 @@
     tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions
 ) %}
   {% if partitions is not none and partitions != [] %} {# static #}
-      {{ bq_static_insert_overwrite_sql(tmp_relation, target_relation, sql, partition_by, partitions, dest_columns, copy_partitions) }}
+      {{ bq_static_insert_overwrite_sql(tmp_relation, target_relation, sql, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions) }}
   {% else %} {# dynamic #}
       {{ bq_dynamic_insert_overwrite_sql(tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists, copy_partitions) }}
   {% endif %}
 {% endmacro %}
 
 {% macro bq_static_insert_overwrite_sql(
-    tmp_relation, target_relation, sql, partition_by, partitions, dest_columns, copy_partitions
+    tmp_relation, target_relation, sql, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions
 ) %}
 
       {% set predicate -%}
@@ -57,11 +57,19 @@
 
       {%- set source_sql -%}
         (
-          {%- if partition_by.time_ingestion_partitioning -%}
-          {{ wrap_with_time_ingestion_partitioning_sql(partition_by, sql, True) }}
+          {% if partition_by.time_ingestion_partitioning and tmp_relation_exists -%}
+          select
+            {{ partition_by.insertable_time_partitioning_field() }},
+            * from {{ tmp_relation }}
+          {% elif tmp_relation_exists -%}
+            select
+            * from {{ tmp_relation }}
+          {%- elif partition_by.time_ingestion_partitioning -%}
+            {{ wrap_with_time_ingestion_partitioning_sql(partition_by, sql, True) }}
           {%- else -%}
-          {{sql}}
+            {{sql}}
           {%- endif -%}
+
         )
       {%- endset -%}
 
@@ -69,13 +77,19 @@
           {% do bq_copy_partitions(tmp_relation, target_relation, partitions, partition_by) %}
       {% else %}
 
-      {#-- Because we're putting the model SQL _directly_ into the MERGE statement,
+      {#-- In case we're putting the model SQL _directly_ into the MERGE statement,
          we need to prepend the MERGE statement with the user-configured sql_header,
          which may be needed to resolve that model SQL (e.g. referencing a variable or UDF in the header)
-         in the "dynamic" case, we save the model SQL result as a temp table first, wherein the
+         in the "temporary table exists" case, we save the model SQL result as a temp table first, wherein the
          sql_header is included by the create_table_as macro.
       #}
-      {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate], include_sql_header=true) }}
+      -- 1. run the merge statement
+      {{ get_insert_overwrite_merge_sql(target_relation, source_sql, dest_columns, [predicate], include_sql_header = not tmp_relation_exists) }};
+
+      {%- if tmp_relation_exists -%}
+      -- 2. clean up the temp table
+      drop table if exists {{ tmp_relation }};
+      {%- endif -%}
 
   {% endif %}
 {% endmacro %}
diff --git a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
index 162ede883..8dd470ffb 100644
--- a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
+++ b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
@@ -450,3 +450,50 @@
 where date_time > '2020-01-01'
 {% endif %}
 """.lstrip()
+
+overwrite_static_day_sql = """
+{% set partitions_to_replace = [
+  "'2020-01-01'",
+  "'2020-01-02'",
+] %}
+
+{{
+    config(
+        materialized="incremental",
+        incremental_strategy="insert_overwrite",
+        cluster_by="id",
+        partition_by={
+            "field": "date_time",
+            "data_type": "datetime",
+            "granularity": "day"
+        },
+        partitions=partitions_to_replace,
+        on_schema_change="sync_all_columns"
+    )
+}}
+
+
+with data as (
+
+    {% if not is_incremental() %}
+
+        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 4 as id, cast('2020-01-01' as datetime) as date_time
+
+    {% else %}
+
+        -- we want to overwrite the 4 records in the 2020-01-01 partition
+        -- with the 2 records below, but add two more in the 2020-01-02 partition
+        select 10 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 20 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 30 as id, cast('2020-01-02' as datetime) as date_time union all
+        select 40 as id, cast('2020-01-02' as datetime) as date_time
+
+    {% endif %}
+
+)
+
+select * from data
+""".lstrip()
diff --git a/tests/functional/adapter/incremental/test_incremental_strategies.py b/tests/functional/adapter/incremental/test_incremental_strategies.py
index 8a90b98ab..b3a51ad09 100644
--- a/tests/functional/adapter/incremental/test_incremental_strategies.py
+++ b/tests/functional/adapter/incremental/test_incremental_strategies.py
@@ -25,6 +25,7 @@
     overwrite_time_sql,
     overwrite_day_with_time_ingestion_sql,
     overwrite_day_with_time_partition_datetime_sql,
+    overwrite_static_day_sql,
 )
 
 
@@ -46,6 +47,7 @@ def models(self):
             "incremental_overwrite_time.sql": overwrite_time_sql,
             "incremental_overwrite_day_with_time_partition.sql": overwrite_day_with_time_ingestion_sql,
             "incremental_overwrite_day_with_time_partition_datetime.sql": overwrite_day_with_time_partition_datetime_sql,
+            "incremental_overwrite_static_day.sql": overwrite_static_day_sql,
         }
 
     @pytest.fixture(scope="class")
@@ -63,10 +65,10 @@ def seeds(self):
     def test__bigquery_assert_incremental_configurations_apply_the_right_strategy(self, project):
         run_dbt(["seed"])
         results = run_dbt()
-        assert len(results) == 10
+        assert len(results) == 11
 
         results = run_dbt()
-        assert len(results) == 10
+        assert len(results) == 11
         incremental_strategies = [
             ("incremental_merge_range", "merge_expected"),
             ("incremental_merge_time", "merge_expected"),
@@ -79,6 +81,7 @@ def test__bigquery_assert_incremental_configurations_apply_the_right_strategy(se
                 "incremental_overwrite_day_with_time_partition_datetime",
                 "incremental_overwrite_day_with_time_partition_expected",
             ),
+            ("incremental_overwrite_static_day", "incremental_overwrite_day_expected"),
         ]
         db_with_schema = f"{project.database}.{project.test_schema}"
         for incremental_strategy in incremental_strategies:

From ea258bb76169375ded8f7ff9e596a436a5ed165a Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Mon, 17 Jul 2023 17:19:31 -0500
Subject: [PATCH 672/860] add datadog env vars (#829)

---
 .github/workflows/integration.yml | 7 ++++++-
 dev-requirements.txt              | 1 +
 tox.ini                           | 4 ++++
 3 files changed, 11 insertions(+), 1 deletion(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 57ea6c9ed..99f78e33d 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -132,6 +132,11 @@ jobs:
       TOXENV: integration-${{ matrix.adapter }}
       PYTEST_ADDOPTS: "-v --color=yes -n4 --csv integration_results.csv"
       DBT_INVOCATION_ENV: github-actions
+      DD_CIVISIBILITY_AGENTLESS_ENABLED: true
+      DD_API_KEY: ${{ secrets.DATADOG_API_KEY }}
+      DD_SITE: datadoghq.com
+      DD_ENV: ci
+      DD_SERVICE: ${{ github.event.repository.name }}
 
     steps:
       - name: Check out the repository
@@ -179,7 +184,7 @@ jobs:
           DATAPROC_REGION: us-central1
           DATAPROC_CLUSTER_NAME: dbt-test-1
           GCS_BUCKET: dbt-ci
-        run: tox
+        run: tox -- --ddtrace
 
       - uses: actions/upload-artifact@v3
         if: always()
diff --git a/dev-requirements.txt b/dev-requirements.txt
index abf89fa2f..7ded80e1b 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -8,6 +8,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 black~=23.3
 bumpversion~=0.6.0
 click~=8.1
+ddtrace~=1.16
 flake8~=6.0
 flaky~=3.7
 freezegun~=1.2
diff --git a/tox.ini b/tox.ini
index e0342e04d..54718cacc 100644
--- a/tox.ini
+++ b/tox.ini
@@ -22,6 +22,8 @@ passenv =
     PYTEST_ADDOPTS
     DATAPROC_*
     GCS_BUCKET
+    DD_SERVICE
+    DD_ENV
 commands =
   bigquery: {envpython} -m pytest {posargs} -vv tests/functional -k "not TestPython" --profile service_account
 deps =
@@ -37,6 +39,8 @@ passenv =
     PYTEST_ADDOPTS
     DATAPROC_*
     GCS_BUCKET
+    DD_SERVICE
+    DD_ENV
 commands =
   {envpython} -m pytest {posargs} -vv tests/functional -k "TestPython" --profile service_account
 deps =

From 1f80a200a127a2a107be6cb92d2de130f8907ea9 Mon Sep 17 00:00:00 2001
From: FishtownBuildBot <77737458+FishtownBuildBot@users.noreply.github.com>
Date: Tue, 18 Jul 2023 12:00:19 -0500
Subject: [PATCH 673/860] Cleanup main after cutting new 1.6.latest branch
 (#832)

* Clean up changelog on main

* Bumping version to 1.7.0a1

* pre-commit

* changie merge

* re trigger tests

---------

Co-authored-by: Matthew McKnight <matthew.mcknight@dbtlabs.com>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .bumpversion.cfg                              |  2 +-
 .changes/0.0.0.md                             |  1 +
 .changes/1.6.0-a1.md                          |  1 -
 .changes/1.6.0-b1.md                          | 13 -----
 .changes/1.6.0-b2.md                          |  1 -
 .changes/1.6.0-b3.md                          | 13 -----
 .changes/1.6.0-b4.md                          | 14 ------
 .../Breaking Changes-20230530-174051.yaml     |  6 ---
 .../1.6.0/Dependencies-20230618-220517.yaml   |  6 ---
 .changes/1.6.0/Features-20230601-141255.yaml  |  6 ---
 .changes/1.6.0/Features-20230604-034603.yaml  |  6 ---
 .changes/1.6.0/Fixes-20230202-010912.yaml     |  8 ---
 .changes/1.6.0/Fixes-20230421-120450.yaml     |  6 ---
 .changes/1.6.0/Fixes-20230427-141957.yaml     |  6 ---
 .changes/1.6.0/Fixes-20230428-232904.yaml     |  6 ---
 .changes/1.6.0/Fixes-20230509-222705.yaml     |  6 ---
 .changes/1.6.0/Fixes-20230511-143217.yaml     |  6 ---
 .changes/1.6.0/Fixes-20230601-120430.yaml     |  6 ---
 .changes/1.6.0/Fixes-20230609-132727.yaml     |  7 ---
 .../Dependencies-20230606-003859.yaml         |  6 ---
 .../Dependencies-20230606-005912.yaml         |  6 ---
 .../Dependencies-20230621-005934.yaml         |  6 ---
 .../Dependencies-20230624-172518.yaml         |  6 ---
 .../Dependencies-20230626-011525.yaml         |  6 ---
 .../Dependencies-20230626-011535.yaml         |  6 ---
 .../Dependencies-20230626-011544.yaml         |  6 ---
 .../Dependencies-20230626-011550.yaml         |  6 ---
 .../Dependencies-20230703-000325.yaml         |  6 ---
 .../unreleased/Features-20230505-184427.yaml  |  6 ---
 .../unreleased/Features-20230622-113645.yaml  |  6 ---
 .../unreleased/Features-20230712-014350.yaml  |  6 ---
 .../unreleased/Fixes-20230320-153618.yaml     |  6 ---
 .../unreleased/Fixes-20230322-150627.yaml     |  6 ---
 .../unreleased/Fixes-20230325-204352.yaml     |  6 ---
 .../unreleased/Fixes-20230626-105156.yaml     |  6 ---
 .../unreleased/Fixes-20230628-162853.yaml     |  6 ---
 .../unreleased/Fixes-20230630-213112.yaml     |  7 ---
 CHANGELOG.md                                  | 50 +------------------
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 40 files changed, 5 insertions(+), 278 deletions(-)
 delete mode 100644 .changes/1.6.0-a1.md
 delete mode 100644 .changes/1.6.0-b1.md
 delete mode 100644 .changes/1.6.0-b2.md
 delete mode 100644 .changes/1.6.0-b3.md
 delete mode 100644 .changes/1.6.0-b4.md
 delete mode 100644 .changes/1.6.0/Breaking Changes-20230530-174051.yaml
 delete mode 100644 .changes/1.6.0/Dependencies-20230618-220517.yaml
 delete mode 100644 .changes/1.6.0/Features-20230601-141255.yaml
 delete mode 100644 .changes/1.6.0/Features-20230604-034603.yaml
 delete mode 100644 .changes/1.6.0/Fixes-20230202-010912.yaml
 delete mode 100644 .changes/1.6.0/Fixes-20230421-120450.yaml
 delete mode 100644 .changes/1.6.0/Fixes-20230427-141957.yaml
 delete mode 100644 .changes/1.6.0/Fixes-20230428-232904.yaml
 delete mode 100644 .changes/1.6.0/Fixes-20230509-222705.yaml
 delete mode 100644 .changes/1.6.0/Fixes-20230511-143217.yaml
 delete mode 100644 .changes/1.6.0/Fixes-20230601-120430.yaml
 delete mode 100644 .changes/1.6.0/Fixes-20230609-132727.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20230606-003859.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20230606-005912.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20230621-005934.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20230624-172518.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20230626-011525.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20230626-011535.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20230626-011544.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20230626-011550.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20230703-000325.yaml
 delete mode 100644 .changes/unreleased/Features-20230505-184427.yaml
 delete mode 100644 .changes/unreleased/Features-20230622-113645.yaml
 delete mode 100644 .changes/unreleased/Features-20230712-014350.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230320-153618.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230322-150627.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230325-204352.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230626-105156.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230628-162853.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230630-213112.yaml

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index d529ec377..6e8ac4d80 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.6.0b4
+current_version = 1.7.0a1
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/0.0.0.md b/.changes/0.0.0.md
index e754a7f1a..dd49fb1b7 100644
--- a/.changes/0.0.0.md
+++ b/.changes/0.0.0.md
@@ -1,5 +1,6 @@
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
+- [1.6](https://github.com/dbt-labs/dbt-bigquery/blob/1.6.latest/CHANGELOG.md)
 - [1.5](https://github.com/dbt-labs/dbt-bigquery/blob/1.5.latest/CHANGELOG.md)
 - [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
 - [1.3](https://github.com/dbt-labs/dbt-bigquery/blob/1.3.latest/CHANGELOG.md)
diff --git a/.changes/1.6.0-a1.md b/.changes/1.6.0-a1.md
deleted file mode 100644
index b724b7b13..000000000
--- a/.changes/1.6.0-a1.md
+++ /dev/null
@@ -1 +0,0 @@
-## dbt-bigquery 1.6.0-a1 - April 17, 2023
diff --git a/.changes/1.6.0-b1.md b/.changes/1.6.0-b1.md
deleted file mode 100644
index 8593f2dad..000000000
--- a/.changes/1.6.0-b1.md
+++ /dev/null
@@ -1,13 +0,0 @@
-## dbt-bigquery 1.6.0-b1 - May 12, 2023
-
-### Fixes
-
-- Support all types of data_type using time ingestion partitioning as previously `date` was failing ([#486](https://github.com/dbt-labs/dbt-bigquery/issues/486))
--  Fix issue of sporadic failure to apply grants during high transaction volumes ([#614](https://github.com/dbt-labs/dbt-bigquery/issues/614))
-- Fix UDF usage with time ingestion ([#684](https://github.com/dbt-labs/dbt-bigquery/issues/684))
-- Fixes adding policy tags when a struct is defined in the yml ([#687](https://github.com/dbt-labs/dbt-bigquery/issues/687))
-- Update signature for execute method ([#](https://github.com/dbt-labs/dbt-bigquery/issues/), [#](https://github.com/dbt-labs/dbt-bigquery/issues/))
-
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#486](https://github.com/dbt-labs/dbt-bigquery/issues/486), [#684](https://github.com/dbt-labs/dbt-bigquery/issues/684))
-- [@dgreen161](https://github.com/dgreen161) ([#687](https://github.com/dbt-labs/dbt-bigquery/issues/687))
diff --git a/.changes/1.6.0-b2.md b/.changes/1.6.0-b2.md
deleted file mode 100644
index 41f646609..000000000
--- a/.changes/1.6.0-b2.md
+++ /dev/null
@@ -1 +0,0 @@
-## dbt-bigquery 1.6.0-b2 - May 25, 2023
diff --git a/.changes/1.6.0-b3.md b/.changes/1.6.0-b3.md
deleted file mode 100644
index 966e9f2f2..000000000
--- a/.changes/1.6.0-b3.md
+++ /dev/null
@@ -1,13 +0,0 @@
-## dbt-bigquery 1.6.0-b3 - June 08, 2023
-
-### Breaking Changes
-
-- Drop support for python 3.7 ([#dbt-labs/dbt-core/7082](https://github.com/dbt-labs/dbt-bigquery/issues/dbt-labs/dbt-core/7082))
-
-### Features
-
-- Standardize the _connection_keys and debug_query for `dbt debug`. ([#PR754](https://github.com/dbt-labs/dbt-bigquery/issues/PR754))
-
-### Fixes
-
-- test foreign key constraint rendering ([#7512](https://github.com/dbt-labs/dbt-bigquery/issues/7512))
diff --git a/.changes/1.6.0-b4.md b/.changes/1.6.0-b4.md
deleted file mode 100644
index 47310f31d..000000000
--- a/.changes/1.6.0-b4.md
+++ /dev/null
@@ -1,14 +0,0 @@
-## dbt-bigquery 1.6.0-b4 - June 23, 2023
-
-### Features
-
-- Support model contracts + constraints on nested columns ([#673](https://github.com/dbt-labs/dbt-bigquery/issues/673))
-
-### Fixes
-
-- test model constraints with sql headers ([#7714](https://github.com/dbt-labs/dbt-bigquery/issues/7714))
-- Pass python model timeout to polling operation so model execution times out as expected. ([#577](https://github.com/dbt-labs/dbt-bigquery/issues/577))
-
-### Dependencies
-
-- Rm explicit agate pin, in favor of transitive dependency from dbt-core ([#777](https://github.com/dbt-labs/dbt-bigquery/pull/777))
diff --git a/.changes/1.6.0/Breaking Changes-20230530-174051.yaml b/.changes/1.6.0/Breaking Changes-20230530-174051.yaml
deleted file mode 100644
index c4d90db97..000000000
--- a/.changes/1.6.0/Breaking Changes-20230530-174051.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Breaking Changes
-body: Drop support for python 3.7
-time: 2023-05-30T17:40:51.510639-04:00
-custom:
-  Author: mikealfare
-  Issue: dbt-labs/dbt-core/7082
diff --git a/.changes/1.6.0/Dependencies-20230618-220517.yaml b/.changes/1.6.0/Dependencies-20230618-220517.yaml
deleted file mode 100644
index 359ae9b7c..000000000
--- a/.changes/1.6.0/Dependencies-20230618-220517.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Dependencies
-body: Rm explicit agate pin, in favor of transitive dependency from dbt-core
-time: 2023-06-18T22:05:17.020743-04:00
-custom:
-  Author: jtcohen6
-  PR: "777"
diff --git a/.changes/1.6.0/Features-20230601-141255.yaml b/.changes/1.6.0/Features-20230601-141255.yaml
deleted file mode 100644
index cfb428e3f..000000000
--- a/.changes/1.6.0/Features-20230601-141255.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Support model contracts + constraints on nested columns
-time: 2023-06-01T14:12:55.433346-04:00
-custom:
-  Author: MichelleArk
-  Issue: "673"
diff --git a/.changes/1.6.0/Features-20230604-034603.yaml b/.changes/1.6.0/Features-20230604-034603.yaml
deleted file mode 100644
index a1eebc327..000000000
--- a/.changes/1.6.0/Features-20230604-034603.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Standardize the _connection_keys and debug_query for `dbt debug`.
-time: 2023-06-04T03:46:03.065575-07:00
-custom:
-  Author: versusfacit
-  Issue: PR754
diff --git a/.changes/1.6.0/Fixes-20230202-010912.yaml b/.changes/1.6.0/Fixes-20230202-010912.yaml
deleted file mode 100644
index d85c2ed84..000000000
--- a/.changes/1.6.0/Fixes-20230202-010912.yaml
+++ /dev/null
@@ -1,8 +0,0 @@
-kind: Fixes
-body: Support all types of data_type using time ingestion partitioning as previously
-  `date` was failing
-time: 2023-02-02T01:09:12.013631+01:00
-custom:
-  Author: Kayrnt
-  Issue: "486"
-  PR: "496"
diff --git a/.changes/1.6.0/Fixes-20230421-120450.yaml b/.changes/1.6.0/Fixes-20230421-120450.yaml
deleted file mode 100644
index 64cee345f..000000000
--- a/.changes/1.6.0/Fixes-20230421-120450.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: ' Fix issue of sporadic failure to apply grants during high transaction volumes'
-time: 2023-04-21T12:04:50.391534-04:00
-custom:
-  Author: mikealfare
-  Issue: "614"
diff --git a/.changes/1.6.0/Fixes-20230427-141957.yaml b/.changes/1.6.0/Fixes-20230427-141957.yaml
deleted file mode 100644
index 7b5bf598d..000000000
--- a/.changes/1.6.0/Fixes-20230427-141957.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: 'Fix UDF usage with time ingestion'
-time: 2023-04-27T14:19:57.518037+02:00
-custom:
-  Author: Kayrnt
-  Issue: "684"
diff --git a/.changes/1.6.0/Fixes-20230428-232904.yaml b/.changes/1.6.0/Fixes-20230428-232904.yaml
deleted file mode 100644
index db9a5e20c..000000000
--- a/.changes/1.6.0/Fixes-20230428-232904.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Fixes adding policy tags when a struct is defined in the yml
-time: 2023-04-28T23:29:04.08966+01:00
-custom:
-  Author: dgreen161
-  Issue: "687"
diff --git a/.changes/1.6.0/Fixes-20230509-222705.yaml b/.changes/1.6.0/Fixes-20230509-222705.yaml
deleted file mode 100644
index f86e3d245..000000000
--- a/.changes/1.6.0/Fixes-20230509-222705.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Update signature for execute method
-time: 2023-05-09T22:27:05.976611-07:00
-custom:
-  Author: aranke
-  Issue: ' '
diff --git a/.changes/1.6.0/Fixes-20230511-143217.yaml b/.changes/1.6.0/Fixes-20230511-143217.yaml
deleted file mode 100644
index ff56ce5d8..000000000
--- a/.changes/1.6.0/Fixes-20230511-143217.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: test foreign key constraint rendering
-time: 2023-05-11T14:32:17.364819-04:00
-custom:
-  Author: michelleark
-  Issue: "7512"
diff --git a/.changes/1.6.0/Fixes-20230601-120430.yaml b/.changes/1.6.0/Fixes-20230601-120430.yaml
deleted file mode 100644
index 715ac26a6..000000000
--- a/.changes/1.6.0/Fixes-20230601-120430.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: test model constraints with sql headers
-time: 2023-06-01T12:04:30.876751-04:00
-custom:
-  Author: michelleark
-  Issue: "7714"
diff --git a/.changes/1.6.0/Fixes-20230609-132727.yaml b/.changes/1.6.0/Fixes-20230609-132727.yaml
deleted file mode 100644
index c37d2e72a..000000000
--- a/.changes/1.6.0/Fixes-20230609-132727.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: Pass python model timeout to polling operation so model execution times out
-  as expected.
-time: 2023-06-09T13:27:27.279842-07:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "577"
diff --git a/.changes/unreleased/Dependencies-20230606-003859.yaml b/.changes/unreleased/Dependencies-20230606-003859.yaml
deleted file mode 100644
index d362c9c8e..000000000
--- a/.changes/unreleased/Dependencies-20230606-003859.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update types-requests requirement from ~=2.28 to ~=2.31"
-time: 2023-06-06T00:38:59.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 737
diff --git a/.changes/unreleased/Dependencies-20230606-005912.yaml b/.changes/unreleased/Dependencies-20230606-005912.yaml
deleted file mode 100644
index 4e2ccab63..000000000
--- a/.changes/unreleased/Dependencies-20230606-005912.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update tox requirement from ~=4.4 to ~=4.6"
-time: 2023-06-06T00:59:12.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 757
diff --git a/.changes/unreleased/Dependencies-20230621-005934.yaml b/.changes/unreleased/Dependencies-20230621-005934.yaml
deleted file mode 100644
index 4730a2462..000000000
--- a/.changes/unreleased/Dependencies-20230621-005934.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump mypy from 1.2.0 to 1.4.0"
-time: 2023-06-21T00:59:34.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 780
diff --git a/.changes/unreleased/Dependencies-20230624-172518.yaml b/.changes/unreleased/Dependencies-20230624-172518.yaml
deleted file mode 100644
index 817bc17c4..000000000
--- a/.changes/unreleased/Dependencies-20230624-172518.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update types-protobuf requirement from ~=4.22 to ~=4.23"
-time: 2023-06-24T17:25:18.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 706
diff --git a/.changes/unreleased/Dependencies-20230626-011525.yaml b/.changes/unreleased/Dependencies-20230626-011525.yaml
deleted file mode 100644
index fc8261ec1..000000000
--- a/.changes/unreleased/Dependencies-20230626-011525.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pre-commit requirement from ~=3.2 to ~=3.3"
-time: 2023-06-26T01:15:25.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 787
diff --git a/.changes/unreleased/Dependencies-20230626-011535.yaml b/.changes/unreleased/Dependencies-20230626-011535.yaml
deleted file mode 100644
index 150d56534..000000000
--- a/.changes/unreleased/Dependencies-20230626-011535.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pytest requirement from ~=7.3 to ~=7.4"
-time: 2023-06-26T01:15:35.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 788
diff --git a/.changes/unreleased/Dependencies-20230626-011544.yaml b/.changes/unreleased/Dependencies-20230626-011544.yaml
deleted file mode 100644
index c5f297a41..000000000
--- a/.changes/unreleased/Dependencies-20230626-011544.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump mypy from 1.4.0 to 1.4.1"
-time: 2023-06-26T01:15:45.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 789
diff --git a/.changes/unreleased/Dependencies-20230626-011550.yaml b/.changes/unreleased/Dependencies-20230626-011550.yaml
deleted file mode 100644
index 658fbb9c5..000000000
--- a/.changes/unreleased/Dependencies-20230626-011550.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pytest-xdist requirement from ~=3.2 to ~=3.3"
-time: 2023-06-26T01:15:50.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 790
diff --git a/.changes/unreleased/Dependencies-20230703-000325.yaml b/.changes/unreleased/Dependencies-20230703-000325.yaml
deleted file mode 100644
index 21a999121..000000000
--- a/.changes/unreleased/Dependencies-20230703-000325.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pip-tools requirement from ~=6.13 to ~=6.14"
-time: 2023-07-03T00:03:25.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 809
diff --git a/.changes/unreleased/Features-20230505-184427.yaml b/.changes/unreleased/Features-20230505-184427.yaml
deleted file mode 100644
index 1267d02ab..000000000
--- a/.changes/unreleased/Features-20230505-184427.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Move the BQ Job link after the job submission instead of job done
-time: 2023-05-05T18:44:27.939038+02:00
-custom:
-  Author: Kayrnt
-  Issue: "696"
diff --git a/.changes/unreleased/Features-20230622-113645.yaml b/.changes/unreleased/Features-20230622-113645.yaml
deleted file mode 100644
index b1ab93f96..000000000
--- a/.changes/unreleased/Features-20230622-113645.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: add dbt-bigquery portion of dbt_clone fucntionality
-time: 2023-06-22T11:36:45.348416-05:00
-custom:
-  Author: McKnight-42, aranke
-  Issue: "7256"
diff --git a/.changes/unreleased/Features-20230712-014350.yaml b/.changes/unreleased/Features-20230712-014350.yaml
deleted file mode 100644
index 9bd47f49b..000000000
--- a/.changes/unreleased/Features-20230712-014350.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Add validate_sql to BigQuery adapter and dry_run to BigQueryConnectionManager
-time: 2023-07-12T01:43:50.36167-04:00
-custom:
-  Author: tlento
-  Issue: "805"
diff --git a/.changes/unreleased/Fixes-20230320-153618.yaml b/.changes/unreleased/Fixes-20230320-153618.yaml
deleted file mode 100644
index e3f1fa011..000000000
--- a/.changes/unreleased/Fixes-20230320-153618.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: add negative part_number arg for split part macro
-time: 2023-03-20T15:36:18.858565-05:00
-custom:
-  Author: dave-connors-3
-  Issue: "615"
diff --git a/.changes/unreleased/Fixes-20230322-150627.yaml b/.changes/unreleased/Fixes-20230322-150627.yaml
deleted file mode 100644
index 90ef354f9..000000000
--- a/.changes/unreleased/Fixes-20230322-150627.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Enable single-line `private_key` for BigQuery Service Account JSON Authentication
-time: 2023-03-22T15:06:27.761416-06:00
-custom:
-  Author: dbeatty10
-  Issue: "7164"
diff --git a/.changes/unreleased/Fixes-20230325-204352.yaml b/.changes/unreleased/Fixes-20230325-204352.yaml
deleted file mode 100644
index 8f213c703..000000000
--- a/.changes/unreleased/Fixes-20230325-204352.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Use tmp table in static insert overwrite to avoid computing the SQL twice
-time: 2023-03-25T20:43:52.830135+01:00
-custom:
-  Author: Kayrnt
-  Issue: 427 556
diff --git a/.changes/unreleased/Fixes-20230626-105156.yaml b/.changes/unreleased/Fixes-20230626-105156.yaml
deleted file mode 100644
index d1c6b9e25..000000000
--- a/.changes/unreleased/Fixes-20230626-105156.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: remove call to dataset update if dataset has not changed
-time: 2023-06-26T10:51:56.698483-07:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "770"
diff --git a/.changes/unreleased/Fixes-20230628-162853.yaml b/.changes/unreleased/Fixes-20230628-162853.yaml
deleted file mode 100644
index b00ca3063..000000000
--- a/.changes/unreleased/Fixes-20230628-162853.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Remove dependent_projects argument from PartialProject call in unit test
-time: 2023-06-28T16:28:53.408844-05:00
-custom:
-  Author: McKnight-42
-  Issue: "7955"
diff --git a/.changes/unreleased/Fixes-20230630-213112.yaml b/.changes/unreleased/Fixes-20230630-213112.yaml
deleted file mode 100644
index 7238c0cb1..000000000
--- a/.changes/unreleased/Fixes-20230630-213112.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: 'Contracts: Handle struct column specified both at root and nested levels +
-  arrays of structs'
-time: 2023-06-30T21:31:12.63257-04:00
-custom:
-  Author: michelleark
-  Issue: 781 782
diff --git a/CHANGELOG.md b/CHANGELOG.md
index c43286a02..ade60b8f6 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,57 +5,9 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
-## dbt-bigquery 1.6.0-b4 - June 23, 2023
-
-### Features
-
-- Support model contracts + constraints on nested columns ([#673](https://github.com/dbt-labs/dbt-bigquery/issues/673))
-
-### Fixes
-
-- test model constraints with sql headers ([#7714](https://github.com/dbt-labs/dbt-bigquery/issues/7714))
-- Pass python model timeout to polling operation so model execution times out as expected. ([#577](https://github.com/dbt-labs/dbt-bigquery/issues/577))
-
-### Dependencies
-
-- Rm explicit agate pin, in favor of transitive dependency from dbt-core ([#777](https://github.com/dbt-labs/dbt-bigquery/pull/777))
-
-
-
-## dbt-bigquery 1.6.0-b3 - June 08, 2023
-
-### Breaking Changes
-
-- Drop support for python 3.7 ([#dbt-labs/dbt-core/7082](https://github.com/dbt-labs/dbt-bigquery/issues/dbt-labs/dbt-core/7082))
-
-### Features
-
-- Standardize the _connection_keys and debug_query for `dbt debug`. ([#PR754](https://github.com/dbt-labs/dbt-bigquery/issues/PR754))
-
-### Fixes
-
-- test foreign key constraint rendering ([#7512](https://github.com/dbt-labs/dbt-bigquery/issues/7512))
-
-## dbt-bigquery 1.6.0-b2 - May 25, 2023
-
-## dbt-bigquery 1.6.0-b1 - May 12, 2023
-
-### Fixes
-
-- Support all types of data_type using time ingestion partitioning as previously `date` was failing ([#486](https://github.com/dbt-labs/dbt-bigquery/issues/486))
--  Fix issue of sporadic failure to apply grants during high transaction volumes ([#614](https://github.com/dbt-labs/dbt-bigquery/issues/614))
-- Fix UDF usage with time ingestion ([#684](https://github.com/dbt-labs/dbt-bigquery/issues/684))
-- Fixes adding policy tags when a struct is defined in the yml ([#687](https://github.com/dbt-labs/dbt-bigquery/issues/687))
-- Update signature for execute method ([#](https://github.com/dbt-labs/dbt-bigquery/issues/), [#](https://github.com/dbt-labs/dbt-bigquery/issues/))
-
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#486](https://github.com/dbt-labs/dbt-bigquery/issues/486), [#684](https://github.com/dbt-labs/dbt-bigquery/issues/684))
-- [@dgreen161](https://github.com/dgreen161) ([#687](https://github.com/dbt-labs/dbt-bigquery/issues/687))
-
-## dbt-bigquery 1.6.0-a1 - April 17, 2023
-
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
+- [1.6](https://github.com/dbt-labs/dbt-bigquery/blob/1.6.latest/CHANGELOG.md)
 - [1.5](https://github.com/dbt-labs/dbt-bigquery/blob/1.5.latest/CHANGELOG.md)
 - [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
 - [1.3](https://github.com/dbt-labs/dbt-bigquery/blob/1.3.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 091852496..874bd74c8 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.6.0b4"
+version = "1.7.0a1"
diff --git a/setup.py b/setup.py
index 05bda5c87..c18c1913d 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.6.0b4"
+package_version = "1.7.0a1"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From da9f2da0f57cbbd13d6bcdfc2fc2a15a447d2185 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Mon, 24 Jul 2023 11:29:50 -0700
Subject: [PATCH 674/860] Create a schema for each Grant Access Test run (#842)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* have test grant access dynamically create schema

* have test grant access dynamically create schema
---
 .../adapter/test_grant_access_to.py           | 78 +++++++++++++------
 1 file changed, 56 insertions(+), 22 deletions(-)

diff --git a/tests/functional/adapter/test_grant_access_to.py b/tests/functional/adapter/test_grant_access_to.py
index 109cab125..633cebe92 100644
--- a/tests/functional/adapter/test_grant_access_to.py
+++ b/tests/functional/adapter/test_grant_access_to.py
@@ -5,45 +5,79 @@
 from dbt.tests.util import run_dbt
 
 
-SELECT_1 = """
-{{ config(
-    materialized='view',
-    grant_access_to=[
-      {'project': 'dbt-test-env-alt', 'dataset': 'GrantAccessTest'},
-    ]
-) }}
-SELECT 1 as one
-"""
+def select_1(dataset: str, materialized: str):
+    config = f"""config(
+                materialized='{materialized}',
+                grant_access_to=[
+                  {{'project': 'dbt-test-env', 'dataset': '{dataset}'}},
+                ]
+            )"""
+    return (
+        "{{"
+        + config
+        + "}}"
+        + """
+           SELECT 1 as one"""
+    )
+
 
-SELECT_1_TABLE = """
-{{ config(
-    materialized='table',
-    grant_access_to=[
-      {'project': 'dbt-test-env-alt', 'dataset': 'GrantAccessTest'},
-    ]
-) }}
-SELECT 1 as one
-"""
 BAD_CONFIG_TABLE_NAME = "bad_view"
 BAD_CONFIG_TABLE = """
 {{ config(
     materialized='view',
     grant_access_to=[
-      {'project': 'dbt-test-env-alt', 'dataset': 'NonExistentDataset'},
+      {'project': 'dbt-test-env', 'dataset': 'NonExistentDataset'},
     ]
 ) }}
+
 SELECT 1 as one
 """
 
 BAD_CONFIG_CHILD_TABLE = "SELECT 1 as one FROM {{ref('" + BAD_CONFIG_TABLE_NAME + "')}}"
 
 
+def get_schema_name(base_schema_name: str) -> str:
+    return f"{base_schema_name}_grant_access"
+
+
 class TestAccessGrantSucceeds:
     @pytest.fixture(scope="class")
-    def models(self):
-        return {"select_1.sql": SELECT_1, "select_1_table.sql": SELECT_1_TABLE}
+    def setup_grant_schema(
+        self,
+        project,
+        unique_schema,
+    ):
+        with project.adapter.connection_named("__test_grants"):
+            relation = project.adapter.Relation.create(
+                database=project.database,
+                schema=get_schema_name(unique_schema),
+                identifier="grant_access",
+            )
+            project.adapter.create_schema(relation)
+            yield relation
+
+    @pytest.fixture(scope="class")
+    def teardown_grant_schema(
+        self,
+        project,
+        unique_schema,
+    ):
+        yield
+        with project.adapter.connection_named("__test_grants"):
+            relation = project.adapter.Relation.create(
+                database=project.database, schema=get_schema_name(unique_schema)
+            )
+            project.adapter.drop_schema(relation)
+
+    @pytest.fixture(scope="class")
+    def models(self, unique_schema):
+        dataset = get_schema_name(unique_schema)
+        return {
+            "select_1.sql": select_1(dataset=dataset, materialized="view"),
+            "select_1_table.sql": select_1(dataset=dataset, materialized="table"),
+        }
 
-    def test_grant_access_succeeds(self, project):
+    def test_grant_access_succeeds(self, project, setup_grant_schema, teardown_grant_schema):
         # Need to run twice to validate idempotency
         results = run_dbt(["run"])
         assert len(results) == 2

From 924c0c3c2592b5f0ccdf6aee90ef03a81f153f12 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Tue, 25 Jul 2023 10:10:30 -0400
Subject: [PATCH 675/860] updated stale workflow to use centralized version
 (#844)

---
 .../Under the Hood-20230724-164946.yaml       |  6 ++++++
 .github/workflows/stale.yml                   | 19 +++++++------------
 2 files changed, 13 insertions(+), 12 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20230724-164946.yaml

diff --git a/.changes/unreleased/Under the Hood-20230724-164946.yaml b/.changes/unreleased/Under the Hood-20230724-164946.yaml
new file mode 100644
index 000000000..2b14d4c4c
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20230724-164946.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Update stale workflow to use centralized version
+time: 2023-07-24T16:49:46.771984-04:00
+custom:
+  Author: mikealfare
+  Issue: "844"
diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml
index a1ca95861..d902340a9 100644
--- a/.github/workflows/stale.yml
+++ b/.github/workflows/stale.yml
@@ -1,17 +1,12 @@
-name: 'Close stale issues and PRs'
+name: "Close stale issues and PRs"
 on:
   schedule:
     - cron: "30 1 * * *"
+
+permissions:
+  issues: write
+  pull-requests: write
+
 jobs:
   stale:
-    runs-on: ubuntu-latest
-    steps:
-      # pinned at v4 (https://github.com/actions/stale/releases/tag/v4.0.0)
-      - uses: actions/stale@cdf15f641adb27a71842045a94023bef6945e3aa
-        with:
-          stale-issue-message: "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 remove the stale label or comment on the issue, or it will be closed in 7 days."
-          stale-pr-message: "This PR has been marked as Stale because it has been open for 180 days with no activity. If you would like the PR to remain open, please remove the stale label or comment on the PR, or it will be closed in 7 days."
-          # mark issues/PRs stale when they haven't seen activity in 180 days
-          days-before-stale: 180
-          # ignore checking issues with the following labels
-          exempt-issue-labels: "epic, discussion"
+    uses: dbt-labs/actions/.github/workflows/stale-bot-matrix.yml@main

From 2bce97721fb7f7733e0a49ec97430072720d7033 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 25 Jul 2023 15:15:32 -0700
Subject: [PATCH 676/860] Update wheel requirement from ~=0.40 to ~=0.41 (#843)

* Update wheel requirement from ~=0.40 to ~=0.41

Updates the requirements on [wheel](https://github.com/pypa/wheel) to permit the latest version.
- [Changelog](https://github.com/pypa/wheel/blob/main/docs/news.rst)
- [Commits](https://github.com/pypa/wheel/compare/0.40.0...0.41.0)

---
updated-dependencies:
- dependency-name: wheel
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

* pass pre-comit

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Co-authored-by: Matthew McKnight <matthew.mcknight@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230724-004957.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230724-004957.yaml

diff --git a/.changes/unreleased/Dependencies-20230724-004957.yaml b/.changes/unreleased/Dependencies-20230724-004957.yaml
new file mode 100644
index 000000000..ee65265c5
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230724-004957.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update wheel requirement from ~=0.40 to ~=0.41"
+time: 2023-07-24T00:49:57.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 843
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 7ded80e1b..3e2b7ade7 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -28,4 +28,4 @@ types-pytz~=2023.3
 types-protobuf~=4.23
 types-requests~=2.31
 twine~=4.0
-wheel~=0.40
+wheel~=0.41

From 6e167f489da613ca94324e2178bbce19c4f31311 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Wed, 2 Aug 2023 18:04:43 -0400
Subject: [PATCH 677/860] Update flake8 requirement from ~=6.0 to ~=6.1 (#847)

* Update flake8 requirement from ~=6.0 to ~=6.1

Updates the requirements on [flake8](https://github.com/pycqa/flake8) to permit the latest version.
- [Commits](https://github.com/pycqa/flake8/compare/6.0.0...6.1.0)

---
updated-dependencies:
- dependency-name: flake8
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

* removed extra line in changelog and workflow that creates the changelog

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230731-004314.yaml | 6 ++++++
 .github/workflows/bot-changelog.yml                   | 2 +-
 dev-requirements.txt                                  | 2 +-
 3 files changed, 8 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Dependencies-20230731-004314.yaml

diff --git a/.changes/unreleased/Dependencies-20230731-004314.yaml b/.changes/unreleased/Dependencies-20230731-004314.yaml
new file mode 100644
index 000000000..68e8e9b8d
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230731-004314.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update flake8 requirement from ~=6.0 to ~=6.1"
+time: 2023-07-31T00:43:14.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 847
diff --git a/.github/workflows/bot-changelog.yml b/.github/workflows/bot-changelog.yml
index 94498d25f..fcd344802 100644
--- a/.github/workflows/bot-changelog.yml
+++ b/.github/workflows/bot-changelog.yml
@@ -57,4 +57,4 @@ jobs:
         commit_message: "Add automated changelog yaml from template for bot PR"
         changie_kind: ${{ matrix.changie_kind }}
         label: ${{ matrix.label }}
-        custom_changelog_string: "custom:\n  Author: ${{ github.event.pull_request.user.login }}\n  PR: ${{ github.event.pull_request.number }}\n"
+        custom_changelog_string: "custom:\n  Author: ${{ github.event.pull_request.user.login }}\n  PR: ${{ github.event.pull_request.number }}"
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 3e2b7ade7..d2d1823e0 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -9,7 +9,7 @@ black~=23.3
 bumpversion~=0.6.0
 click~=8.1
 ddtrace~=1.16
-flake8~=6.0
+flake8~=6.1
 flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13

From 6a2a90ccfa2a705672ddaccc74f050c09e45cff6 Mon Sep 17 00:00:00 2001
From: Christophe Oudar <kayrnt@gmail.com>
Date: Thu, 3 Aug 2023 01:49:31 +0200
Subject: [PATCH 678/860] avoid creating twice the temp table in dynamic insert
 overwrite (#845)

* avoid creating twice the temp table in dynamic insert overwrite

* add tests for copy_partitions & time ingestion on on_schema_change

---------

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .../unreleased/Fixes-20230727-213526.yaml     |   6 +
 .../incremental_strategy/insert_overwrite.sql |  16 +-
 .../test_incremental_on_schema_change.py      | 140 +++++++++++++++---
 3 files changed, 135 insertions(+), 27 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230727-213526.yaml

diff --git a/.changes/unreleased/Fixes-20230727-213526.yaml b/.changes/unreleased/Fixes-20230727-213526.yaml
new file mode 100644
index 000000000..ced5b1350
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230727-213526.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Avoid creating twice the temp table in dynamic insert overwrite for sql on_schema_change != 'ignore' or python models
+time: 2023-07-27T21:35:26.531164+02:00
+custom:
+  Author: Kayrnt
+  Issue: "810"
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
index fd7d40e0f..4c22fd376 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
@@ -19,7 +19,9 @@
 {% macro bq_copy_partitions(tmp_relation, target_relation, partitions, partition_by) %}
 
   {% for partition in partitions %}
-    {% if partition_by.granularity == 'hour' %}
+    {% if partition_by.data_type == 'int64' %}
+      {% set partition = partition | as_text %}
+    {% elif partition_by.granularity == 'hour' %}
       {% set partition = partition.strftime("%Y%m%d%H") %}
     {% elif partition_by.granularity == 'day' %}
       {% set partition = partition.strftime("%Y%m%d") %}
@@ -97,11 +99,13 @@
 {% macro bq_dynamic_copy_partitions_insert_overwrite_sql(
   tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists, copy_partitions
   ) %}
-  {# We run temp table creation in a separated script to move to partitions copy #}
-  {%- call statement('create_tmp_relation_for_copy', language='sql') -%}
-    {{ bq_create_table_as(partition_by, True, tmp_relation, sql, 'sql')
-  }}
-  {%- endcall %}
+  {%- if tmp_relation_exists is false -%}
+  {# We run temp table creation in a separated script to move to partitions copy if it doesn't already exist #}
+    {%- call statement('create_tmp_relation_for_copy', language='sql') -%}
+      {{ bq_create_table_as(partition_by, True, tmp_relation, sql, 'sql')
+    }}
+    {%- endcall %}
+  {%- endif -%}
   {%- set partitions_sql -%}
     select distinct {{ partition_by.render_wrapped() }}
     from {{ tmp_relation }}
diff --git a/tests/functional/adapter/incremental/test_incremental_on_schema_change.py b/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
index 08ee44bfa..60f4500df 100644
--- a/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
+++ b/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
@@ -58,11 +58,12 @@ class TestIncrementalOnSchemaChange(BaseIncrementalOnSchemaChange):
 {% endif %}
 """
 
-_MODELS__INCREMENTAL_TIME_INGESTION_PARTITIONING = """
+_MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_TIME_INGESTION_PARTITIONING = """
 
 {{
     config(
         materialized="incremental",
+        on_schema_change='sync_all_columns',
         incremental_strategy='insert_overwrite',
         partition_by={
             "field": "date_hour",
@@ -78,19 +79,43 @@ class TestIncrementalOnSchemaChange(BaseIncrementalOnSchemaChange):
 
     {% if not is_incremental() %}
 
-        select 1 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 2 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 3 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 4 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour
+        select 1 as id,
+        cast('2020-01-01 01:00:00' as datetime) as date_hour,
+        1 as field_1,
+        2 as field_2 union all
+        select 2 as id,
+        cast('2020-01-01 01:00:00' as datetime) as date_hour,
+        1 as field_1,
+        2 as field_2 union all
+        select 3 as id,
+        cast('2020-01-01 01:00:00' as datetime) as date_hour,
+        1 as field_1,
+        2 as field_2 union all
+        select 4 as id,
+        cast('2020-01-01 01:00:00' as datetime) as date_hour,
+        1 as field_1,
+        2 as field_2
 
     {% else %}
 
         -- we want to overwrite the 4 records in the 2020-01-01 01:00:00 partition
         -- with the 2 records below, but add two more in the 2020-01-00 02:00:00 partition
-        select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-        select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour union all
-        select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+        select 10 as id,
+        cast('2020-01-01 01:00:00' as datetime) as date_hour,
+        3 as field_3,
+        2 as field_2 union all
+        select 20 as id,
+        cast('2020-01-01 01:00:00' as datetime) as date_hour,
+        3 as field_3,
+        2 as field_2 union all
+        select 30 as id,
+        cast('2020-01-01 02:00:00' as datetime) as date_hour,
+        3 as field_3,
+        2 as field_2 union all
+        select 40 as id,
+        cast('2020-01-01 02:00:00' as datetime) as date_hour,
+        3 as field_3,
+        2 as field_2
 
     {% endif %}
 
@@ -99,10 +124,11 @@ class TestIncrementalOnSchemaChange(BaseIncrementalOnSchemaChange):
 select * from data
 """
 
-_MODELS__INCREMENTAL_TIME_INGESTION_PARTITIONING_TARGET = """
+_MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_TIME_INGESTION_PARTITIONING_TARGET = """
 {{
     config(
         materialized="incremental",
+        on_schema_change='sync_all_columns',
         partition_by={
             "field": "date_hour",
             "data_type": "datetime",
@@ -114,13 +140,71 @@ class TestIncrementalOnSchemaChange(BaseIncrementalOnSchemaChange):
 
 {% if not is_incremental() %}
 
-    select 10 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-    select 30 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+    select 10 as id,
+    cast('2020-01-01 01:00:00' as datetime) as date_hour,
+    3 as field_3,
+    2 as field_2
+    union all
+    select 30 as id,
+    cast('2020-01-01 02:00:00' as datetime) as date_hour,
+    3 as field_3,
+    2 as field_2
 
 {% else %}
 
-    select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
-    select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+    select 20 as id,
+    cast('2020-01-01 01:00:00' as datetime) as date_hour,
+    3 as field_3,
+    2 as field_2
+    union all
+    select 40 as id,
+    cast('2020-01-01 02:00:00' as datetime) as date_hour,
+    3 as field_3,
+    2 as field_2
+
+{% endif %}
+"""
+
+_MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_DYNAMIC_INSERT_OVERWRITE_COPY_PARTITIONS = """
+{{
+    config(
+        materialized='incremental',
+        unique_key='id',
+        on_schema_change='sync_all_columns',
+        partition_by={
+            "field": "id",
+            "data_type": "int64",
+            "range": {
+                "start": 1,
+                "end": 7,
+                "interval": 1
+            },
+            "copy_partitions": true
+        },
+        incremental_strategy='insert_overwrite'
+    )
+}}
+
+WITH source_data AS (SELECT * FROM {{ ref('model_a') }} )
+
+{% set string_type = 'string' %}
+
+{% if is_incremental() %}
+
+SELECT id,
+       cast(field1 as {{string_type}}) as field1,
+       cast(field3 as {{string_type}}) as field3, -- to validate new fields
+       cast(field4 as {{string_type}}) AS field4 -- to validate new fields
+
+FROM source_data WHERE id > _dbt_max_partition
+
+{% else %}
+
+select id,
+       cast(field1 as {{string_type}}) as field1,
+       cast(field2 as {{string_type}}) as field2
+
+from source_data where id <= 3
 
 {% endif %}
 """
@@ -132,9 +216,10 @@ def models(self):
         return {
             "model_a.sql": _MODELS__A,
             "incremental_sync_all_columns_dynamic_insert_overwrite.sql": _MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_DYNAMIC_INSERT_OVERWRITE,
+            "incremental_sync_all_columns_dynamic_insert_overwrite_copy_partitions.sql": _MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_DYNAMIC_INSERT_OVERWRITE_COPY_PARTITIONS,
             "incremental_sync_all_columns_target.sql": _MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_TARGET,
-            "incremental_time_ingestion_partitioning.sql": _MODELS__INCREMENTAL_TIME_INGESTION_PARTITIONING,
-            "incremental_time_ingestion_partitioning_target.sql": _MODELS__INCREMENTAL_TIME_INGESTION_PARTITIONING_TARGET,
+            "incremental_sync_all_columns_time_ingestion_partitioning.sql": _MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_TIME_INGESTION_PARTITIONING,
+            "incremental_sync_all_columns_time_ingestion_partitioning_target.sql": _MODELS__INCREMENTAL_SYNC_ALL_COLUMNS_TIME_INGESTION_PARTITIONING_TARGET,
         }
 
     def test_run_incremental_sync_all_columns_dynamic_insert_overwrite(self, project):
@@ -143,9 +228,22 @@ def test_run_incremental_sync_all_columns_dynamic_insert_overwrite(self, project
         compare_target = "incremental_sync_all_columns_target"
         self.run_twice_and_assert(select, compare_source, compare_target, project)
 
-    # TODO: this test was added here, but it doesn't actually use 'on_schema_change'
-    def test_run_incremental_time_ingestion_partitioning(self, project):
-        select = "model_a incremental_time_ingestion_partitioning incremental_time_ingestion_partitioning_target"
-        compare_source = "incremental_time_ingestion_partitioning"
-        compare_target = "incremental_time_ingestion_partitioning_target"
+    def test_run_incremental_sync_all_columns_dynamic_insert_overwrite_copy_partitions(
+        self, project
+    ):
+        select = (
+            "model_a incremental_sync_all_columns_dynamic_insert_overwrite_copy_partitions "
+            "incremental_sync_all_columns_target"
+        )
+        compare_source = "incremental_sync_all_columns_dynamic_insert_overwrite_copy_partitions"
+        compare_target = "incremental_sync_all_columns_target"
+        self.run_twice_and_assert(select, compare_source, compare_target, project)
+
+    def test_run_incremental_sync_all_columns_time_ingestion_partitioning(self, project):
+        select = (
+            "model_a incremental_sync_all_columns_time_ingestion_partitioning "
+            "incremental_sync_all_columns_time_ingestion_partitioning_target"
+        )
+        compare_source = "incremental_sync_all_columns_time_ingestion_partitioning"
+        compare_target = "incremental_sync_all_columns_time_ingestion_partitioning_target"
         self.run_twice_and_assert(select, compare_source, compare_target, project)

From 37dfbca5e3d764c0e792b93d48fed215aa9f5a20 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 3 Aug 2023 09:13:14 -0400
Subject: [PATCH 679/860] Update pip-tools requirement from ~=6.14 to ~=7.0
 (#831)

* Update pip-tools requirement from ~=6.14 to ~=7.0

Updates the requirements on [pip-tools](https://github.com/jazzband/pip-tools) to permit the latest version.
- [Release notes](https://github.com/jazzband/pip-tools/releases)
- [Changelog](https://github.com/jazzband/pip-tools/blob/main/CHANGELOG.md)
- [Commits](https://github.com/jazzband/pip-tools/compare/6.14.0...7.0.0)

---
updated-dependencies:
- dependency-name: pip-tools
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

* removed extra line in changelog

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230717-000639.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230717-000639.yaml

diff --git a/.changes/unreleased/Dependencies-20230717-000639.yaml b/.changes/unreleased/Dependencies-20230717-000639.yaml
new file mode 100644
index 000000000..81bead79c
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230717-000639.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pip-tools requirement from ~=6.14 to ~=7.0"
+time: 2023-07-17T00:06:39.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 831
diff --git a/dev-requirements.txt b/dev-requirements.txt
index d2d1823e0..85b548200 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -14,7 +14,7 @@ flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
 mypy==1.4.1  # patch updates have historically introduced breaking changes
-pip-tools~=6.14
+pip-tools~=7.0
 pre-commit~=3.3
 pre-commit-hooks~=4.4
 pytest~=7.4

From b009a13e5b422ca63780645de22d7823455309e2 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 3 Aug 2023 11:22:26 -0400
Subject: [PATCH 680/860] Update black requirement from ~=23.3 to ~=23.7 (#815)

* Update black requirement from ~=23.3 to ~=23.7

Updates the requirements on [black](https://github.com/psf/black) to permit the latest version.
- [Release notes](https://github.com/psf/black/releases)
- [Changelog](https://github.com/psf/black/blob/main/CHANGES.md)
- [Commits](https://github.com/psf/black/compare/23.3.0...23.7.0)

---
updated-dependencies:
- dependency-name: black
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

* removed extra line in changelog

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230711-002411.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230711-002411.yaml

diff --git a/.changes/unreleased/Dependencies-20230711-002411.yaml b/.changes/unreleased/Dependencies-20230711-002411.yaml
new file mode 100644
index 000000000..f19b4f8e9
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230711-002411.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update black requirement from ~=23.3 to ~=23.7"
+time: 2023-07-11T00:24:11.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 815
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 85b548200..71164be56 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -5,7 +5,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor
-black~=23.3
+black~=23.7
 bumpversion~=0.6.0
 click~=8.1
 ddtrace~=1.16

From 070b74eff496dbb06e78f5137dfb2ef4ea3556aa Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 3 Aug 2023 11:51:20 -0400
Subject: [PATCH 681/860] Update pip-tools requirement from ~=7.0 to ~=7.2
 (#851)

* Update pip-tools requirement from ~=7.0 to ~=7.2

Updates the requirements on [pip-tools](https://github.com/jazzband/pip-tools) to permit the latest version.
- [Release notes](https://github.com/jazzband/pip-tools/releases)
- [Changelog](https://github.com/jazzband/pip-tools/blob/main/CHANGELOG.md)
- [Commits](https://github.com/jazzband/pip-tools/compare/7.0.0...7.2.0)

---
updated-dependencies:
- dependency-name: pip-tools
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20230803-131551.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230803-131551.yaml

diff --git a/.changes/unreleased/Dependencies-20230803-131551.yaml b/.changes/unreleased/Dependencies-20230803-131551.yaml
new file mode 100644
index 000000000..0c0245168
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230803-131551.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pip-tools requirement from ~=7.0 to ~=7.2"
+time: 2023-08-03T13:15:51.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 851
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 71164be56..7985f5b8e 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -14,7 +14,7 @@ flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
 mypy==1.4.1  # patch updates have historically introduced breaking changes
-pip-tools~=7.0
+pip-tools~=7.2
 pre-commit~=3.3
 pre-commit-hooks~=4.4
 pytest~=7.4

From 3ed72ee8836a8753c4e7a5e7e7dec9cd8867e036 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 3 Aug 2023 12:40:53 -0400
Subject: [PATCH 682/860] Update ddtrace requirement from ~=1.16 to ~=1.17
 (#839)

* Update ddtrace requirement from ~=1.16 to ~=1.17

Updates the requirements on [ddtrace](https://github.com/DataDog/dd-trace-py) to permit the latest version.
- [Release notes](https://github.com/DataDog/dd-trace-py/releases)
- [Changelog](https://github.com/DataDog/dd-trace-py/blob/1.x/CHANGELOG.md)
- [Commits](https://github.com/DataDog/dd-trace-py/compare/v1.16.0...v1.17.0)

---
updated-dependencies:
- dependency-name: ddtrace
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

* removed extra line in changelog

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20230721-010106.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230721-010106.yaml

diff --git a/.changes/unreleased/Dependencies-20230721-010106.yaml b/.changes/unreleased/Dependencies-20230721-010106.yaml
new file mode 100644
index 000000000..266f491c5
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230721-010106.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update ddtrace requirement from ~=1.16 to ~=1.17"
+time: 2023-07-21T01:01:06.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 839
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 7985f5b8e..e22885dc4 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -8,7 +8,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 black~=23.7
 bumpversion~=0.6.0
 click~=8.1
-ddtrace~=1.16
+ddtrace~=1.17
 flake8~=6.1
 flaky~=3.7
 freezegun~=1.2

From 4368ef497d05e6069b4b45ea100d4258536e9521 Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Thu, 3 Aug 2023 11:01:31 -0600
Subject: [PATCH 683/860] Requiring Python >=3.8 enables walrus operator (#852)

* Requiring Python >=3.8 enables walrus operator

* Update comment to be evergreen

---------

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20230803-102915.yaml | 6 ++++++
 setup.py                                       | 4 ++--
 2 files changed, 8 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230803-102915.yaml

diff --git a/.changes/unreleased/Fixes-20230803-102915.yaml b/.changes/unreleased/Fixes-20230803-102915.yaml
new file mode 100644
index 000000000..6070fb448
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230803-102915.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Requiring Python >=3.8 enables walrus operator
+time: 2023-08-03T10:29:15.371798-06:00
+custom:
+  Author: cemsbr dbeatty10
+  Issue: "849"
diff --git a/setup.py b/setup.py
index c18c1913d..8112a713e 100644
--- a/setup.py
+++ b/setup.py
@@ -1,7 +1,7 @@
 #!/usr/bin/env python
 import sys
 
-# require python 3.7 or newer
+# require a supported version of Python
 if sys.version_info < (3, 8):
     print("Error: dbt does not support this version of Python.")
     print("Please upgrade to Python 3.8 or higher.")
@@ -91,5 +91,5 @@ def _dbt_core_version(plugin_version: str) -> str:
         "Programming Language :: Python :: 3.10",
         "Programming Language :: Python :: 3.11",
     ],
-    python_requires=">=3.7",
+    python_requires=">=3.8",
 )

From b06d2309cff249f74b8aea02d0c56e8d763f943c Mon Sep 17 00:00:00 2001
From: Nathaniel May <nathaniel.may@fishtownanalytics.com>
Date: Thu, 3 Aug 2023 14:36:14 -0400
Subject: [PATCH 684/860] move slack secret (#859)

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .github/workflows/release.yml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml
index 1c0885001..88942e251 100644
--- a/.github/workflows/release.yml
+++ b/.github/workflows/release.yml
@@ -225,4 +225,4 @@ jobs:
       status: "failure"
 
     secrets:
-      SLACK_WEBHOOK_URL: ${{ secrets.SLACK_DEV_CORE_ALERTS }}
+      SLACK_WEBHOOK_URL: ${{ secrets.SLACK_DEV_ADAPTER_ALERTS }}

From 1b8cae258618106eb4e2af747d464696c14bb8ba Mon Sep 17 00:00:00 2001
From: Mykola Dementiev <dementiev2711@gmail.com>
Date: Mon, 7 Aug 2023 18:20:59 +0300
Subject: [PATCH 685/860] Fix contact validation failure on query data types
 change (#862)

* add current_timestamp to prevent empty query caching
* add changie log
---
 .changes/unreleased/Fixes-20230804-005817.yaml   |  6 ++++++
 dbt/include/bigquery/macros/adapters/columns.sql | 10 ++++++++++
 2 files changed, 16 insertions(+)
 create mode 100644 .changes/unreleased/Fixes-20230804-005817.yaml
 create mode 100644 dbt/include/bigquery/macros/adapters/columns.sql

diff --git a/.changes/unreleased/Fixes-20230804-005817.yaml b/.changes/unreleased/Fixes-20230804-005817.yaml
new file mode 100644
index 000000000..34e0601e1
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230804-005817.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fix contact validation failure on query data types change
+time: 2023-08-04T00:58:17.97704+03:00
+custom:
+  Author: dementiev27
+  Issue: "861"
diff --git a/dbt/include/bigquery/macros/adapters/columns.sql b/dbt/include/bigquery/macros/adapters/columns.sql
new file mode 100644
index 000000000..c540f1e2d
--- /dev/null
+++ b/dbt/include/bigquery/macros/adapters/columns.sql
@@ -0,0 +1,10 @@
+{% macro bigquery__get_empty_subquery_sql(select_sql, select_sql_header=none) %}
+    {%- if select_sql_header is not none -%}
+    {{ select_sql_header }}
+    {%- endif -%}
+    select * from (
+        {{ select_sql }}
+    ) as __dbt_sbq
+    where false and current_timestamp() = current_timestamp()
+    limit 0
+{% endmacro %}

From 4b3505d8bea1140f2559519bd5aa3cde779f4195 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Wed, 9 Aug 2023 00:33:17 -0400
Subject: [PATCH 686/860] Update tox requirement from ~=4.6 to ~=4.7 (#869)

* Update tox requirement from ~=4.6 to ~=4.7

Updates the requirements on [tox](https://github.com/tox-dev/tox) to permit the latest version.
- [Release notes](https://github.com/tox-dev/tox/releases)
- [Changelog](https://github.com/tox-dev/tox/blob/main/docs/changelog.rst)
- [Commits](https://github.com/tox-dev/tox/compare/4.6.0...4.7.0)

---
updated-dependencies:
- dependency-name: tox
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
---
 .changes/unreleased/Dependencies-20230809-034524.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230809-034524.yaml

diff --git a/.changes/unreleased/Dependencies-20230809-034524.yaml b/.changes/unreleased/Dependencies-20230809-034524.yaml
new file mode 100644
index 000000000..390c39540
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230809-034524.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update tox requirement from ~=4.6 to ~=4.7"
+time: 2023-08-09T03:45:24.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 869
diff --git a/dev-requirements.txt b/dev-requirements.txt
index e22885dc4..64967ac5a 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -23,7 +23,7 @@ pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
 pytest-xdist~=3.3
 pytz~=2023.3
-tox~=4.6
+tox~=4.7
 types-pytz~=2023.3
 types-protobuf~=4.23
 types-requests~=2.31

From cea6588372e91c2c514142f14efaf96ff9e6140e Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 11 Aug 2023 12:28:21 -0400
Subject: [PATCH 687/860] Bump mypy from 1.4.1 to 1.5.0 (#874)

* Bump mypy from 1.4.1 to 1.5.0

Bumps [mypy](https://github.com/python/mypy) from 1.4.1 to 1.5.0.
- [Commits](https://github.com/python/mypy/compare/v1.4.1...v1.5.0)

---
updated-dependencies:
- dependency-name: mypy
  dependency-type: direct:development
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
---
 .changes/unreleased/Dependencies-20230811-010246.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230811-010246.yaml

diff --git a/.changes/unreleased/Dependencies-20230811-010246.yaml b/.changes/unreleased/Dependencies-20230811-010246.yaml
new file mode 100644
index 000000000..bac65f14e
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230811-010246.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump mypy from 1.4.1 to 1.5.0"
+time: 2023-08-11T01:02:46.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 874
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 64967ac5a..6f17bf083 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -13,7 +13,7 @@ flake8~=6.1
 flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
-mypy==1.4.1  # patch updates have historically introduced breaking changes
+mypy==1.5.0  # patch updates have historically introduced breaking changes
 pip-tools~=7.2
 pre-commit~=3.3
 pre-commit-hooks~=4.4

From c6a3e3124df0b307b7c04db19ad8a122afc3ec56 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 11 Aug 2023 16:30:18 +0000
Subject: [PATCH 688/860] Update types-protobuf requirement from ~=4.23 to
 ~=4.24 (#873)

* Update types-protobuf requirement from ~=4.23 to ~=4.24

Updates the requirements on [types-protobuf](https://github.com/python/typeshed) to permit the latest version.
- [Commits](https://github.com/python/typeshed/commits)

---
updated-dependencies:
- dependency-name: types-protobuf
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
---
 .changes/unreleased/Dependencies-20230810-002407.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230810-002407.yaml

diff --git a/.changes/unreleased/Dependencies-20230810-002407.yaml b/.changes/unreleased/Dependencies-20230810-002407.yaml
new file mode 100644
index 000000000..d89aaca4f
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230810-002407.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update types-protobuf requirement from ~=4.23 to ~=4.24"
+time: 2023-08-10T00:24:07.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 873
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 6f17bf083..cd0d063dc 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -25,7 +25,7 @@ pytest-xdist~=3.3
 pytz~=2023.3
 tox~=4.7
 types-pytz~=2023.3
-types-protobuf~=4.23
+types-protobuf~=4.24
 types-requests~=2.31
 twine~=4.0
 wheel~=0.41

From eb1b97edaf82168ca9837447e4b5d75d874099b7 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Fri, 11 Aug 2023 12:00:58 -0500
Subject: [PATCH 689/860] add env vars for datadog test visibility (#875)

* add env vars for datadog test visibility

* force tests

* fix test
---
 tox.ini | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)

diff --git a/tox.ini b/tox.ini
index 54718cacc..1d5413dcf 100644
--- a/tox.ini
+++ b/tox.ini
@@ -22,8 +22,11 @@ passenv =
     PYTEST_ADDOPTS
     DATAPROC_*
     GCS_BUCKET
-    DD_SERVICE
+    DD_CIVISIBILITY_AGENTLESS_ENABLED
+    DD_API_KEY
+    DD_SITE
     DD_ENV
+    DD_SERVICE
 commands =
   bigquery: {envpython} -m pytest {posargs} -vv tests/functional -k "not TestPython" --profile service_account
 deps =
@@ -39,8 +42,11 @@ passenv =
     PYTEST_ADDOPTS
     DATAPROC_*
     GCS_BUCKET
-    DD_SERVICE
+    DD_CIVISIBILITY_AGENTLESS_ENABLED
+    DD_API_KEY
+    DD_SITE
     DD_ENV
+    DD_SERVICE
 commands =
   {envpython} -m pytest {posargs} -vv tests/functional -k "TestPython" --profile service_account
 deps =

From dc5613033612bb699bc43ddfbe1c4545d7cd41d2 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 11 Aug 2023 18:07:57 +0000
Subject: [PATCH 690/860] Update pip-tools requirement from ~=7.2 to ~=7.3
 (#870)

* Update pip-tools requirement from ~=7.2 to ~=7.3

Updates the requirements on [pip-tools](https://github.com/jazzband/pip-tools) to permit the latest version.
- [Release notes](https://github.com/jazzband/pip-tools/releases)
- [Changelog](https://github.com/jazzband/pip-tools/blob/main/CHANGELOG.md)
- [Commits](https://github.com/jazzband/pip-tools/compare/7.2.0...7.3.0)

---
updated-dependencies:
- dependency-name: pip-tools
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20230809-005844.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230809-005844.yaml

diff --git a/.changes/unreleased/Dependencies-20230809-005844.yaml b/.changes/unreleased/Dependencies-20230809-005844.yaml
new file mode 100644
index 000000000..bd2edc1cd
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230809-005844.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pip-tools requirement from ~=7.2 to ~=7.3"
+time: 2023-08-09T00:58:44.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 870
diff --git a/dev-requirements.txt b/dev-requirements.txt
index cd0d063dc..9782c95ff 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -14,7 +14,7 @@ flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
 mypy==1.5.0  # patch updates have historically introduced breaking changes
-pip-tools~=7.2
+pip-tools~=7.3
 pre-commit~=3.3
 pre-commit-hooks~=4.4
 pytest~=7.4

From 3510f76a24247cf13044e8e3adcb3f80be0524c9 Mon Sep 17 00:00:00 2001
From: Nicholas Zeolla <nick.zeolla@starlingbank.com>
Date: Fri, 11 Aug 2023 22:08:13 +0100
Subject: [PATCH 691/860] Swap dataproc `batch_id` declaration to model config
 (#804)

* swap batch_id declaration to model config

* address changie req, fix python submission

* Update bug that is being resolved

* implement dbeatty's suggestion

* Update .changes/unreleased/Fixes-20230630-092618.yaml

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>

* Add 2 tests

---------

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .../unreleased/Fixes-20230630-092618.yaml     |  6 ++
 dbt/adapters/bigquery/python_submissions.py   |  4 +
 tests/functional/adapter/test_python_model.py | 88 +++++++++++++++++++
 3 files changed, 98 insertions(+)
 create mode 100644 .changes/unreleased/Fixes-20230630-092618.yaml

diff --git a/.changes/unreleased/Fixes-20230630-092618.yaml b/.changes/unreleased/Fixes-20230630-092618.yaml
new file mode 100644
index 000000000..16f34bec0
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230630-092618.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Change batch_id to model override
+time: 2023-06-30T09:26:18.854492+01:00
+custom:
+  Author: nickozilla
+  Issue: "671"
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 0c7ce1917..6e5a11e52 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -115,6 +115,9 @@ def _get_job_client(self) -> dataproc_v1.BatchControllerClient:
             client_options=self.client_options, credentials=self.GoogleCredentials
         )
 
+    def _get_batch_id(self) -> str:
+        return self.parsed_model["config"].get("batch_id")
+
     def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         batch = self._configure_batch()
         parent = f"projects/{self.credential.execution_project}/locations/{self.credential.dataproc_region}"
@@ -122,6 +125,7 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
         request = dataproc_v1.CreateBatchRequest(
             parent=parent,
             batch=batch,
+            batch_id=self._get_batch_id(),
         )
         # make the request
         operation = self.job_client.create_batch(request=request)  # type: ignore
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
index 7f17429f1..241082cdb 100644
--- a/tests/functional/adapter/test_python_model.py
+++ b/tests/functional/adapter/test_python_model.py
@@ -1,5 +1,6 @@
 import os
 import pytest
+import time
 from dbt.tests.util import run_dbt, run_dbt_and_capture, write_file
 import dbt.tests.adapter.python_model.test_python_model as dbt_tests
 
@@ -64,6 +65,93 @@ def model(dbt, spark):
     return spark.createDataFrame(data, schema=['test1', 'test3'])
 """
 
+models__python_array_batch_id_python = """
+import pandas
+
+def model(dbt, spark):
+    random_array = [
+        [9001.3985362160208, -157.9871329592354],
+        [-817.8786101352823, -528.9769041860632],
+        [-886.6488625065194, 941.0504221837489],
+        [6.69525238666165, 919.5903586746183],
+        [754.3718741592056, -121.25678519054622],
+        [-352.3158889341157, 254.9985130814921],
+        [563.0633042715097, 833.2963094260072],
+    ]
+
+    df = pd.DataFrame(random_array, columns=["A", "B"])
+
+    df["C"] = df["A"] * df["B"]
+
+    final_df = df[["A", "B", "C"]]
+
+    return final_df
+"""
+
+models__python_array_batch_id_yaml = """
+models:
+  - name: python_array_batch_id
+    description: A random table with a calculated column defined in python.
+    config:
+      batch_id: '{{ run_started_at.strftime("%Y-%m-%d-%H-%M-%S") }}-python-array'
+    columns:
+      - name: A
+        description: Column A
+      - name: B
+        description: Column B
+      - name: C
+        description: Column C
+"""
+
+custom_ts_id = str("custom-" + str(time.time()).replace(".", "-"))
+
+models__bad_python_array_batch_id_yaml = f"""
+models:
+  - name: python_array_batch_id
+    description: A random table with a calculated column defined in python.
+    config:
+      batch_id: {custom_ts_id}-python-array
+    columns:
+      - name: A
+        description: Column A
+      - name: B
+        description: Column B
+      - name: C
+        description: Column C
+"""
+
+
+class TestPythonBatchIdModels:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "python_array_batch_id.py": models__python_array_batch_id_python,
+            "python_array_batch_id.yml": models__python_array_batch_id_yaml,
+        }
+
+    def test_multiple_named_python_models(self, project):
+        result, output = run_dbt_and_capture(["run"], expect_pass=True)
+        time.sleep(5)  # In case both runs are submitted simultaneously
+        result_two, output_two = run_dbt_and_capture(["run"], expect_pass=True)
+        assert len(result) == 1
+        assert len(result_two) == 1
+
+
+class TestPythonDuplicateBatchIdModels:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "python_array_batch_id.py": models__python_array_batch_id_python,
+            "python_array_batch_id.yml": models__bad_python_array_batch_id_yaml,
+        }
+
+    def test_multiple_python_models_fixed_id(self, project):
+        result, output = run_dbt_and_capture(["run"], expect_pass=True)
+        result_two, output_two = run_dbt_and_capture(["run"], expect_pass=False)
+        assert result_two[0].message.startswith("409 Already exists: Failed to create batch:")
+        assert len(result) == 1
+        assert len(result_two) == 1
+
 
 @pytest.mark.skip(reason=TEST_SKIP_MESSAGE)
 class TestChangingSchemaDataproc:

From 0dd6abec495d3b9b18105d01c4a26bc88e3ec6f0 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Wed, 16 Aug 2023 12:20:21 -0400
Subject: [PATCH 692/860] Update tox requirement from ~=4.7 to ~=4.8 (#880)

* Update tox requirement from ~=4.7 to ~=4.8

Updates the requirements on [tox](https://github.com/tox-dev/tox) to permit the latest version.
- [Release notes](https://github.com/tox-dev/tox/releases)
- [Changelog](https://github.com/tox-dev/tox/blob/main/docs/changelog.rst)
- [Commits](https://github.com/tox-dev/tox/compare/4.7.0...4.8.0)

---
updated-dependencies:
- dependency-name: tox
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
---
 .changes/unreleased/Dependencies-20230814-005415.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230814-005415.yaml

diff --git a/.changes/unreleased/Dependencies-20230814-005415.yaml b/.changes/unreleased/Dependencies-20230814-005415.yaml
new file mode 100644
index 000000000..f1722073f
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230814-005415.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update tox requirement from ~=4.7 to ~=4.8"
+time: 2023-08-14T00:54:15.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 880
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 9782c95ff..7e4f53f6d 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -23,7 +23,7 @@ pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
 pytest-xdist~=3.3
 pytz~=2023.3
-tox~=4.7
+tox~=4.8
 types-pytz~=2023.3
 types-protobuf~=4.24
 types-requests~=2.31

From a5c36ea4f5767a422c8ba310a4047241d45c994c Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Wed, 16 Aug 2023 16:22:23 +0000
Subject: [PATCH 693/860] Update ddtrace requirement from ~=1.17 to ~=1.18
 (#881)

* Update ddtrace requirement from ~=1.17 to ~=1.18

Updates the requirements on [ddtrace](https://github.com/DataDog/dd-trace-py) to permit the latest version.
- [Release notes](https://github.com/DataDog/dd-trace-py/releases)
- [Changelog](https://github.com/DataDog/dd-trace-py/blob/1.x/CHANGELOG.md)
- [Commits](https://github.com/DataDog/dd-trace-py/compare/v1.17.0...v1.18.0)

---
updated-dependencies:
- dependency-name: ddtrace
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
---
 .changes/unreleased/Dependencies-20230814-005542.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230814-005542.yaml

diff --git a/.changes/unreleased/Dependencies-20230814-005542.yaml b/.changes/unreleased/Dependencies-20230814-005542.yaml
new file mode 100644
index 000000000..7acf66fca
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230814-005542.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update ddtrace requirement from ~=1.17 to ~=1.18"
+time: 2023-08-14T00:55:42.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 881
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 7e4f53f6d..685cbe387 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -8,7 +8,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 black~=23.7
 bumpversion~=0.6.0
 click~=8.1
-ddtrace~=1.17
+ddtrace~=1.18
 flake8~=6.1
 flaky~=3.7
 freezegun~=1.2

From e9786aba3db4cdd024f1917e76e1093cab56db0a Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Thu, 17 Aug 2023 21:49:02 +0000
Subject: [PATCH 694/860] Bumping version to 1.7.0b1 and generate changelog

---
 .bumpversion.cfg                              |  2 +-
 .changes/1.7.0-b1.md                          | 36 ++++++++++++++++++
 .../Dependencies-20230711-002411.yaml         |  0
 .../Dependencies-20230717-000639.yaml         |  0
 .../Dependencies-20230721-010106.yaml         |  0
 .../Dependencies-20230724-004957.yaml         |  0
 .../Dependencies-20230731-004314.yaml         |  0
 .../Dependencies-20230803-131551.yaml         |  0
 .../Dependencies-20230809-005844.yaml         |  0
 .../Dependencies-20230809-034524.yaml         |  0
 .../Dependencies-20230810-002407.yaml         |  0
 .../Dependencies-20230811-010246.yaml         |  0
 .../Dependencies-20230814-005415.yaml         |  0
 .../Dependencies-20230814-005542.yaml         |  0
 .../Fixes-20230630-092618.yaml                |  0
 .../Fixes-20230727-213526.yaml                |  0
 .../Fixes-20230803-102915.yaml                |  0
 .../Fixes-20230804-005817.yaml                |  0
 .../Under the Hood-20230724-164946.yaml       |  0
 CHANGELOG.md                                  | 38 +++++++++++++++++++
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 22 files changed, 77 insertions(+), 3 deletions(-)
 create mode 100644 .changes/1.7.0-b1.md
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230711-002411.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230717-000639.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230721-010106.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230724-004957.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230731-004314.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230803-131551.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230809-005844.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230809-034524.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230810-002407.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230811-010246.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230814-005415.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230814-005542.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Fixes-20230630-092618.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Fixes-20230727-213526.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Fixes-20230803-102915.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Fixes-20230804-005817.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Under the Hood-20230724-164946.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 6e8ac4d80..ff82260ee 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.7.0a1
+current_version = 1.7.0b1
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.7.0-b1.md b/.changes/1.7.0-b1.md
new file mode 100644
index 000000000..13dd4982d
--- /dev/null
+++ b/.changes/1.7.0-b1.md
@@ -0,0 +1,36 @@
+## dbt-bigquery 1.7.0-b1 - August 17, 2023
+
+### Features
+
+- Change batch_id to model override ([#671](https://github.com/dbt-labs/dbt-bigquery/issues/671))
+
+### Fixes
+
+- Avoid creating twice the temp table in dynamic insert overwrite for sql on_schema_change != 'ignore' or python models ([#810](https://github.com/dbt-labs/dbt-bigquery/issues/810))
+- Requiring Python >=3.8 enables walrus operator ([#849](https://github.com/dbt-labs/dbt-bigquery/issues/849))
+- Fix contact validation failure on query data types change ([#861](https://github.com/dbt-labs/dbt-bigquery/issues/861))
+
+### Under the Hood
+
+- Update stale workflow to use centralized version ([#844](https://github.com/dbt-labs/dbt-bigquery/issues/844))
+
+### Dependencies
+
+- Update black requirement from ~=23.3 to ~=23.7 ([#815](https://github.com/dbt-labs/dbt-bigquery/pull/815))
+- Update pip-tools requirement from ~=6.14 to ~=7.0 ([#831](https://github.com/dbt-labs/dbt-bigquery/pull/831))
+- Update ddtrace requirement from ~=1.16 to ~=1.17 ([#839](https://github.com/dbt-labs/dbt-bigquery/pull/839))
+- Update wheel requirement from ~=0.40 to ~=0.41 ([#843](https://github.com/dbt-labs/dbt-bigquery/pull/843))
+- Update flake8 requirement from ~=6.0 to ~=6.1 ([#847](https://github.com/dbt-labs/dbt-bigquery/pull/847))
+- Update pip-tools requirement from ~=7.0 to ~=7.2 ([#851](https://github.com/dbt-labs/dbt-bigquery/pull/851))
+- Update pip-tools requirement from ~=7.2 to ~=7.3 ([#870](https://github.com/dbt-labs/dbt-bigquery/pull/870))
+- Update tox requirement from ~=4.6 to ~=4.7 ([#869](https://github.com/dbt-labs/dbt-bigquery/pull/869))
+- Update types-protobuf requirement from ~=4.23 to ~=4.24 ([#873](https://github.com/dbt-labs/dbt-bigquery/pull/873))
+- Bump mypy from 1.4.1 to 1.5.0 ([#874](https://github.com/dbt-labs/dbt-bigquery/pull/874))
+- Update tox requirement from ~=4.7 to ~=4.8 ([#880](https://github.com/dbt-labs/dbt-bigquery/pull/880))
+- Update ddtrace requirement from ~=1.17 to ~=1.18 ([#881](https://github.com/dbt-labs/dbt-bigquery/pull/881))
+
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#810](https://github.com/dbt-labs/dbt-bigquery/issues/810))
+- [@cemsbr](https://github.com/cemsbr) ([#849](https://github.com/dbt-labs/dbt-bigquery/issues/849))
+- [@dementiev27](https://github.com/dementiev27) ([#861](https://github.com/dbt-labs/dbt-bigquery/issues/861))
+- [@nickozilla](https://github.com/nickozilla) ([#671](https://github.com/dbt-labs/dbt-bigquery/issues/671))
diff --git a/.changes/unreleased/Dependencies-20230711-002411.yaml b/.changes/1.7.0/Dependencies-20230711-002411.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230711-002411.yaml
rename to .changes/1.7.0/Dependencies-20230711-002411.yaml
diff --git a/.changes/unreleased/Dependencies-20230717-000639.yaml b/.changes/1.7.0/Dependencies-20230717-000639.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230717-000639.yaml
rename to .changes/1.7.0/Dependencies-20230717-000639.yaml
diff --git a/.changes/unreleased/Dependencies-20230721-010106.yaml b/.changes/1.7.0/Dependencies-20230721-010106.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230721-010106.yaml
rename to .changes/1.7.0/Dependencies-20230721-010106.yaml
diff --git a/.changes/unreleased/Dependencies-20230724-004957.yaml b/.changes/1.7.0/Dependencies-20230724-004957.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230724-004957.yaml
rename to .changes/1.7.0/Dependencies-20230724-004957.yaml
diff --git a/.changes/unreleased/Dependencies-20230731-004314.yaml b/.changes/1.7.0/Dependencies-20230731-004314.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230731-004314.yaml
rename to .changes/1.7.0/Dependencies-20230731-004314.yaml
diff --git a/.changes/unreleased/Dependencies-20230803-131551.yaml b/.changes/1.7.0/Dependencies-20230803-131551.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230803-131551.yaml
rename to .changes/1.7.0/Dependencies-20230803-131551.yaml
diff --git a/.changes/unreleased/Dependencies-20230809-005844.yaml b/.changes/1.7.0/Dependencies-20230809-005844.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230809-005844.yaml
rename to .changes/1.7.0/Dependencies-20230809-005844.yaml
diff --git a/.changes/unreleased/Dependencies-20230809-034524.yaml b/.changes/1.7.0/Dependencies-20230809-034524.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230809-034524.yaml
rename to .changes/1.7.0/Dependencies-20230809-034524.yaml
diff --git a/.changes/unreleased/Dependencies-20230810-002407.yaml b/.changes/1.7.0/Dependencies-20230810-002407.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230810-002407.yaml
rename to .changes/1.7.0/Dependencies-20230810-002407.yaml
diff --git a/.changes/unreleased/Dependencies-20230811-010246.yaml b/.changes/1.7.0/Dependencies-20230811-010246.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230811-010246.yaml
rename to .changes/1.7.0/Dependencies-20230811-010246.yaml
diff --git a/.changes/unreleased/Dependencies-20230814-005415.yaml b/.changes/1.7.0/Dependencies-20230814-005415.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230814-005415.yaml
rename to .changes/1.7.0/Dependencies-20230814-005415.yaml
diff --git a/.changes/unreleased/Dependencies-20230814-005542.yaml b/.changes/1.7.0/Dependencies-20230814-005542.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230814-005542.yaml
rename to .changes/1.7.0/Dependencies-20230814-005542.yaml
diff --git a/.changes/unreleased/Fixes-20230630-092618.yaml b/.changes/1.7.0/Fixes-20230630-092618.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230630-092618.yaml
rename to .changes/1.7.0/Fixes-20230630-092618.yaml
diff --git a/.changes/unreleased/Fixes-20230727-213526.yaml b/.changes/1.7.0/Fixes-20230727-213526.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230727-213526.yaml
rename to .changes/1.7.0/Fixes-20230727-213526.yaml
diff --git a/.changes/unreleased/Fixes-20230803-102915.yaml b/.changes/1.7.0/Fixes-20230803-102915.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230803-102915.yaml
rename to .changes/1.7.0/Fixes-20230803-102915.yaml
diff --git a/.changes/unreleased/Fixes-20230804-005817.yaml b/.changes/1.7.0/Fixes-20230804-005817.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230804-005817.yaml
rename to .changes/1.7.0/Fixes-20230804-005817.yaml
diff --git a/.changes/unreleased/Under the Hood-20230724-164946.yaml b/.changes/1.7.0/Under the Hood-20230724-164946.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20230724-164946.yaml
rename to .changes/1.7.0/Under the Hood-20230724-164946.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index ade60b8f6..09fd2d4a1 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,6 +5,44 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.7.0-b1 - August 17, 2023
+
+### Features
+
+- Change batch_id to model override ([#671](https://github.com/dbt-labs/dbt-bigquery/issues/671))
+
+### Fixes
+
+- Avoid creating twice the temp table in dynamic insert overwrite for sql on_schema_change != 'ignore' or python models ([#810](https://github.com/dbt-labs/dbt-bigquery/issues/810))
+- Requiring Python >=3.8 enables walrus operator ([#849](https://github.com/dbt-labs/dbt-bigquery/issues/849))
+- Fix contact validation failure on query data types change ([#861](https://github.com/dbt-labs/dbt-bigquery/issues/861))
+
+### Under the Hood
+
+- Update stale workflow to use centralized version ([#844](https://github.com/dbt-labs/dbt-bigquery/issues/844))
+
+### Dependencies
+
+- Update black requirement from ~=23.3 to ~=23.7 ([#815](https://github.com/dbt-labs/dbt-bigquery/pull/815))
+- Update pip-tools requirement from ~=6.14 to ~=7.0 ([#831](https://github.com/dbt-labs/dbt-bigquery/pull/831))
+- Update ddtrace requirement from ~=1.16 to ~=1.17 ([#839](https://github.com/dbt-labs/dbt-bigquery/pull/839))
+- Update wheel requirement from ~=0.40 to ~=0.41 ([#843](https://github.com/dbt-labs/dbt-bigquery/pull/843))
+- Update flake8 requirement from ~=6.0 to ~=6.1 ([#847](https://github.com/dbt-labs/dbt-bigquery/pull/847))
+- Update pip-tools requirement from ~=7.0 to ~=7.2 ([#851](https://github.com/dbt-labs/dbt-bigquery/pull/851))
+- Update pip-tools requirement from ~=7.2 to ~=7.3 ([#870](https://github.com/dbt-labs/dbt-bigquery/pull/870))
+- Update tox requirement from ~=4.6 to ~=4.7 ([#869](https://github.com/dbt-labs/dbt-bigquery/pull/869))
+- Update types-protobuf requirement from ~=4.23 to ~=4.24 ([#873](https://github.com/dbt-labs/dbt-bigquery/pull/873))
+- Bump mypy from 1.4.1 to 1.5.0 ([#874](https://github.com/dbt-labs/dbt-bigquery/pull/874))
+- Update tox requirement from ~=4.7 to ~=4.8 ([#880](https://github.com/dbt-labs/dbt-bigquery/pull/880))
+- Update ddtrace requirement from ~=1.17 to ~=1.18 ([#881](https://github.com/dbt-labs/dbt-bigquery/pull/881))
+
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#810](https://github.com/dbt-labs/dbt-bigquery/issues/810))
+- [@cemsbr](https://github.com/cemsbr) ([#849](https://github.com/dbt-labs/dbt-bigquery/issues/849))
+- [@dementiev27](https://github.com/dementiev27) ([#861](https://github.com/dbt-labs/dbt-bigquery/issues/861))
+- [@nickozilla](https://github.com/nickozilla) ([#671](https://github.com/dbt-labs/dbt-bigquery/issues/671))
+
+
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 - [1.6](https://github.com/dbt-labs/dbt-bigquery/blob/1.6.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 874bd74c8..48607b01f 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.7.0a1"
+version = "1.7.0b1"
diff --git a/setup.py b/setup.py
index 8112a713e..947554f6d 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.7.0a1"
+package_version = "1.7.0b1"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From f2525cc1cf10d30381d6280ef707cadbdc561082 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Tue, 22 Aug 2023 10:39:43 -0500
Subject: [PATCH 695/860] change type of expected value (#890)

---
 .changes/unreleased/Fixes-20230821-180758.yaml | 6 ++++++
 tests/functional/adapter/expected_stats.py     | 6 +++---
 2 files changed, 9 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230821-180758.yaml

diff --git a/.changes/unreleased/Fixes-20230821-180758.yaml b/.changes/unreleased/Fixes-20230821-180758.yaml
new file mode 100644
index 000000000..01d042a2b
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230821-180758.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: changes expected value types to AnyInteger to take into account changes in core
+time: 2023-08-21T18:07:58.746741-05:00
+custom:
+  Author: McKnight-42
+  Issue: "889"
diff --git a/tests/functional/adapter/expected_stats.py b/tests/functional/adapter/expected_stats.py
index f4a1e022d..560d8de73 100644
--- a/tests/functional/adapter/expected_stats.py
+++ b/tests/functional/adapter/expected_stats.py
@@ -1,4 +1,4 @@
-from dbt.tests.util import AnyString, AnyFloat
+from dbt.tests.util import AnyString, AnyInteger
 
 
 def bigquery_stats(is_table, partition=None, cluster=None):
@@ -10,14 +10,14 @@ def bigquery_stats(is_table, partition=None, cluster=None):
                 "num_bytes": {
                     "id": "num_bytes",
                     "label": AnyString(),
-                    "value": AnyFloat(),
+                    "value": AnyInteger(),
                     "description": AnyString(),
                     "include": True,
                 },
                 "num_rows": {
                     "id": "num_rows",
                     "label": AnyString(),
-                    "value": AnyFloat(),
+                    "value": AnyInteger(),
                     "description": AnyString(),
                     "include": True,
                 },

From 2c7220a9afa59f7a7cf5e456a0b609954f1c38e2 Mon Sep 17 00:00:00 2001
From: Christophe Oudar
 <90898980+github-christophe-oudar@users.noreply.github.com>
Date: Tue, 22 Aug 2023 20:27:11 +0200
Subject: [PATCH 696/860] case insensitive check on partition matching (#888)

* case insensitive check on partition matching

* Review change

---------

Co-authored-by: Christophe Oudar <kayrnt@gmail.com>
---
 .../unreleased/Fixes-20230818-214616.yaml     |  6 ++++
 dbt/adapters/bigquery/__init__.py             |  2 +-
 dbt/adapters/bigquery/impl.py                 |  5 +--
 tests/unit/test_bigquery_adapter.py           | 31 ++++++++++++++++++-
 4 files changed, 40 insertions(+), 4 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230818-214616.yaml

diff --git a/.changes/unreleased/Fixes-20230818-214616.yaml b/.changes/unreleased/Fixes-20230818-214616.yaml
new file mode 100644
index 000000000..a66059e21
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230818-214616.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: case insensitive check on partition matching
+time: 2023-08-18T21:46:16.828488+02:00
+custom:
+  Author: Kayrnt
+  Issue: "886"
diff --git a/dbt/adapters/bigquery/__init__.py b/dbt/adapters/bigquery/__init__.py
index b66ef278a..adbd67590 100644
--- a/dbt/adapters/bigquery/__init__.py
+++ b/dbt/adapters/bigquery/__init__.py
@@ -2,7 +2,7 @@
 from dbt.adapters.bigquery.connections import BigQueryCredentials
 from dbt.adapters.bigquery.relation import BigQueryRelation  # noqa
 from dbt.adapters.bigquery.column import BigQueryColumn  # noqa
-from dbt.adapters.bigquery.impl import BigQueryAdapter, GrantTarget  # noqa
+from dbt.adapters.bigquery.impl import BigQueryAdapter, GrantTarget, PartitionConfig  # noqa
 
 from dbt.adapters.base import AdapterPlugin
 from dbt.include import bigquery
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index f53cd4084..3f8351861 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -620,12 +620,13 @@ def _partitions_match(table, conf_partition: Optional[PartitionConfig]) -> bool:
             table_field = (
                 table.time_partitioning.field.lower() if table.time_partitioning.field else None
             )
+
             table_granularity = table.partitioning_type
             conf_table_field = conf_partition.field
             return (
-                table_field == conf_table_field
+                table_field == conf_table_field.lower()
                 or (conf_partition.time_ingestion_partitioning and table_field is not None)
-            ) and table_granularity == conf_partition.granularity
+            ) and table_granularity.lower() == conf_partition.granularity.lower()
         elif conf_partition and table.range_partitioning is not None:
             dest_part = table.range_partitioning
             conf_part = conf_partition.range or {}
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 56b8e07d7..bb98db86f 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -12,10 +12,12 @@
 
 import dbt.dataclass_schema
 
+from dbt.adapters.bigquery import PartitionConfig
 from dbt.adapters.bigquery import BigQueryCredentials
 from dbt.adapters.bigquery import BigQueryAdapter
 from dbt.adapters.bigquery import BigQueryRelation
 from dbt.adapters.bigquery import Plugin as BigQueryPlugin
+from google.cloud.bigquery.table import Table
 from dbt.adapters.bigquery.connections import BigQueryConnectionManager
 from dbt.adapters.bigquery.connections import _sanitize_label, _VALIDATE_LABEL_LENGTH_LIMIT
 from dbt.adapters.base.query_headers import MacroQueryStringSetter
@@ -376,7 +378,10 @@ def test_location_user_agent(self, mock_bq, mock_auth_default):
         mock_client.assert_not_called()
         connection.handle
         mock_client.assert_called_once_with(
-            "dbt-unit-000000", creds, location="Luna Station", client_info=HasUserAgent()
+            "dbt-unit-000000",
+            creds,
+            location="Luna Station",
+            client_info=HasUserAgent(),
         )
 
 
@@ -1023,6 +1028,30 @@ def test_convert_time_type(self):
         for col_idx, expect in enumerate(expected):
             assert BigQueryAdapter.convert_time_type(agate_table, col_idx) == expect
 
+    # The casing in this case can't be enforced on the API side,
+    # so we have to validate that we have a case-insensitive comparison
+    def test_partitions_match(self):
+        table = Table.from_api_repr(
+            {
+                "tableReference": {
+                    "projectId": "test-project",
+                    "datasetId": "test_dataset",
+                    "tableId": "test_table",
+                },
+                "timePartitioning": {"type": "DAY", "field": "ts"},
+            }
+        )
+        partition_config = PartitionConfig.parse(
+            {
+                "field": "TS",
+                "data_type": "date",
+                "granularity": "day",
+                "time_ingestion_partitioning": False,
+                "copy_partitions": False,
+            }
+        )
+        assert BigQueryAdapter._partitions_match(table, partition_config) is True
+
 
 class TestBigQueryGrantAccessTo(BaseTestBigQueryAdapter):
     entity = BigQueryRelation.from_dict(

From d7fb235a42780841c82ea5b01a647b3416070afe Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 23 Aug 2023 15:11:00 -0500
Subject: [PATCH 697/860] revert change made in pr 890 to go along with core
 revert (#894)

---
 .changes/unreleased/Fixes-20230821-180758.yaml | 6 ------
 tests/functional/adapter/expected_stats.py     | 6 +++---
 2 files changed, 3 insertions(+), 9 deletions(-)
 delete mode 100644 .changes/unreleased/Fixes-20230821-180758.yaml

diff --git a/.changes/unreleased/Fixes-20230821-180758.yaml b/.changes/unreleased/Fixes-20230821-180758.yaml
deleted file mode 100644
index 01d042a2b..000000000
--- a/.changes/unreleased/Fixes-20230821-180758.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: changes expected value types to AnyInteger to take into account changes in core
-time: 2023-08-21T18:07:58.746741-05:00
-custom:
-  Author: McKnight-42
-  Issue: "889"
diff --git a/tests/functional/adapter/expected_stats.py b/tests/functional/adapter/expected_stats.py
index 560d8de73..f4a1e022d 100644
--- a/tests/functional/adapter/expected_stats.py
+++ b/tests/functional/adapter/expected_stats.py
@@ -1,4 +1,4 @@
-from dbt.tests.util import AnyString, AnyInteger
+from dbt.tests.util import AnyString, AnyFloat
 
 
 def bigquery_stats(is_table, partition=None, cluster=None):
@@ -10,14 +10,14 @@ def bigquery_stats(is_table, partition=None, cluster=None):
                 "num_bytes": {
                     "id": "num_bytes",
                     "label": AnyString(),
-                    "value": AnyInteger(),
+                    "value": AnyFloat(),
                     "description": AnyString(),
                     "include": True,
                 },
                 "num_rows": {
                     "id": "num_rows",
                     "label": AnyString(),
-                    "value": AnyInteger(),
+                    "value": AnyFloat(),
                     "description": AnyString(),
                     "include": True,
                 },

From 6e7c0fce74dd45b8f5f24fdbbdbe4f64ca1de765 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Mon, 28 Aug 2023 15:53:31 -0500
Subject: [PATCH 698/860] remove some connection fields that may be sensitive
 user information that we don't want showing up in dbt debug commands (#901)

---
 dbt/adapters/bigquery/connections.py | 3 ---
 1 file changed, 3 deletions(-)

diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index b466fee3b..15c983480 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -186,12 +186,9 @@ def _connection_keys(self):
             "job_creation_timeout_seconds",
             "job_execution_timeout_seconds",
             "keyfile",
-            "keyfile_json",
             "timeout_seconds",
-            "token",
             "refresh_token",
             "client_id",
-            "client_secret",
             "token_uri",
             "dataproc_region",
             "dataproc_cluster_name",

From 7628ee052e848d0a342335bb6f4c0a81da27224a Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Thu, 31 Aug 2023 11:10:55 -0400
Subject: [PATCH 699/860] =?UTF-8?q?break=20out=20MV-relevant=20macros=20in?=
 =?UTF-8?q?=20adapters.sql=20into=20separate=20files=20in=20m=E2=80=A6=20(?=
 =?UTF-8?q?#904)?=
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

* break out MV-relevant macros in adapters.sql into separate files in macros/relations
---
 .../Under the Hood-20230829-122918.yaml       |  6 ++
 dbt/include/bigquery/macros/adapters.sql      | 60 -------------------
 .../bigquery/macros/relations/cluster.sql     | 13 ++++
 .../bigquery/macros/relations/drop.sql        |  5 ++
 .../bigquery/macros/relations/options.sql     |  8 +++
 .../bigquery/macros/relations/partition.sql   | 15 +++++
 .../bigquery/macros/relations/rename.sql      |  3 +
 .../macros/relations/table/options.sql        |  4 ++
 .../macros/relations/view/options.sql         |  4 ++
 9 files changed, 58 insertions(+), 60 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20230829-122918.yaml
 create mode 100644 dbt/include/bigquery/macros/relations/cluster.sql
 create mode 100644 dbt/include/bigquery/macros/relations/drop.sql
 create mode 100644 dbt/include/bigquery/macros/relations/options.sql
 create mode 100644 dbt/include/bigquery/macros/relations/partition.sql
 create mode 100644 dbt/include/bigquery/macros/relations/rename.sql
 create mode 100644 dbt/include/bigquery/macros/relations/table/options.sql
 create mode 100644 dbt/include/bigquery/macros/relations/view/options.sql

diff --git a/.changes/unreleased/Under the Hood-20230829-122918.yaml b/.changes/unreleased/Under the Hood-20230829-122918.yaml
new file mode 100644
index 000000000..6dace0db4
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20230829-122918.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Re-organize adapters.sql into more granular files inside of macros/relations
+time: 2023-08-29T12:29:18.356174-04:00
+custom:
+  Author: mikealfare
+  Issue: "904"
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt/include/bigquery/macros/adapters.sql
index 23a3f3bf6..f166e5d05 100644
--- a/dbt/include/bigquery/macros/adapters.sql
+++ b/dbt/include/bigquery/macros/adapters.sql
@@ -1,48 +1,3 @@
-
-{% macro partition_by(partition_config) -%}
-    {%- if partition_config is none -%}
-      {% do return('') %}
-    {%- elif partition_config.time_ingestion_partitioning -%}
-        partition by {{ partition_config.render_wrapped() }}
-    {%- elif partition_config.data_type | lower in ('date','timestamp','datetime') -%}
-        partition by {{ partition_config.render() }}
-    {%- elif partition_config.data_type | lower in ('int64') -%}
-        {%- set range = partition_config.range -%}
-        partition by range_bucket(
-            {{ partition_config.field }},
-            generate_array({{ range.start}}, {{ range.end }}, {{ range.interval }})
-        )
-    {%- endif -%}
-{%- endmacro -%}
-
-{% macro cluster_by(raw_cluster_by) %}
-  {%- if raw_cluster_by is not none -%}
-  cluster by {% if raw_cluster_by is string -%}
-    {% set raw_cluster_by = [raw_cluster_by] %}
-  {%- endif -%}
-  {%- for cluster in raw_cluster_by -%}
-    {{ cluster }}
-    {%- if not loop.last -%}, {% endif -%}
-  {%- endfor -%}
-
-  {% endif %}
-
-{%- endmacro -%}
-
-{% macro bigquery_options(opts) %}
-  {% set options -%}
-    OPTIONS({% for opt_key, opt_val in opts.items() %}
-      {{ opt_key }}={{ opt_val }}{{ "," if not loop.last }}
-    {% endfor %})
-  {%- endset %}
-  {%- do return(options) -%}
-{%- endmacro -%}
-
-{% macro bigquery_table_options(config, node, temporary) %}
-  {% set opts = adapter.get_table_options(config, node, temporary) %}
-  {%- do return(bigquery_options(opts)) -%}
-{%- endmacro -%}
-
 {% macro bigquery__create_table_as(temporary, relation, compiled_code, language='sql') -%}
   {%- if language == 'sql' -%}
     {%- set raw_partition_by = config.get('partition_by', none) -%}
@@ -102,11 +57,6 @@
 
 {%- endmacro -%}
 
-{% macro bigquery_view_options(config, node) %}
-  {% set opts = adapter.get_view_options(config, node) %}
-  {%- do return(bigquery_options(opts)) -%}
-{%- endmacro -%}
-
 {% macro bigquery__create_view_as(relation, sql) -%}
   {%- set sql_header = config.get('sql_header', none) -%}
 
@@ -126,12 +76,6 @@
   {{ adapter.drop_schema(relation) }}
 {% endmacro %}
 
-{% macro bigquery__drop_relation(relation) -%}
-  {% call statement('drop_relation') -%}
-    drop {{ relation.type }} if exists {{ relation }}
-  {%- endcall %}
-{% endmacro %}
-
 {% macro bigquery__get_columns_in_relation(relation) -%}
   {{ return(adapter.get_columns_in_relation(relation)) }}
 {% endmacro %}
@@ -162,10 +106,6 @@
   {% do adapter.update_columns(relation, column_dict) %}
 {% endmacro %}
 
-{% macro bigquery__rename_relation(from_relation, to_relation) -%}
-  {% do adapter.rename_relation(from_relation, to_relation) %}
-{% endmacro %}
-
 {% macro bigquery__alter_relation_add_columns(relation, add_columns) %}
 
   {% set sql -%}
diff --git a/dbt/include/bigquery/macros/relations/cluster.sql b/dbt/include/bigquery/macros/relations/cluster.sql
new file mode 100644
index 000000000..3d2e640ad
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/cluster.sql
@@ -0,0 +1,13 @@
+{% macro cluster_by(raw_cluster_by) %}
+  {%- if raw_cluster_by is not none -%}
+  cluster by {% if raw_cluster_by is string -%}
+    {% set raw_cluster_by = [raw_cluster_by] %}
+  {%- endif -%}
+  {%- for cluster in raw_cluster_by -%}
+    {{ cluster }}
+    {%- if not loop.last -%}, {% endif -%}
+  {%- endfor -%}
+
+  {% endif %}
+
+{%- endmacro -%}
diff --git a/dbt/include/bigquery/macros/relations/drop.sql b/dbt/include/bigquery/macros/relations/drop.sql
new file mode 100644
index 000000000..89e5e260c
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/drop.sql
@@ -0,0 +1,5 @@
+{% macro bigquery__drop_relation(relation) -%}
+  {% call statement('drop_relation') -%}
+    drop {{ relation.type }} if exists {{ relation }}
+  {%- endcall %}
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/options.sql b/dbt/include/bigquery/macros/relations/options.sql
new file mode 100644
index 000000000..11f5d8541
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/options.sql
@@ -0,0 +1,8 @@
+{% macro bigquery_options(opts) %}
+  {% set options -%}
+    OPTIONS({% for opt_key, opt_val in opts.items() %}
+      {{ opt_key }}={{ opt_val }}{{ "," if not loop.last }}
+    {% endfor %})
+  {%- endset %}
+  {%- do return(options) -%}
+{%- endmacro -%}
diff --git a/dbt/include/bigquery/macros/relations/partition.sql b/dbt/include/bigquery/macros/relations/partition.sql
new file mode 100644
index 000000000..238baeca0
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/partition.sql
@@ -0,0 +1,15 @@
+{% macro partition_by(partition_config) -%}
+    {%- if partition_config is none -%}
+      {% do return('') %}
+    {%- elif partition_config.time_ingestion_partitioning -%}
+        partition by {{ partition_config.render_wrapped() }}
+    {%- elif partition_config.data_type | lower in ('date','timestamp','datetime') -%}
+        partition by {{ partition_config.render() }}
+    {%- elif partition_config.data_type | lower in ('int64') -%}
+        {%- set range = partition_config.range -%}
+        partition by range_bucket(
+            {{ partition_config.field }},
+            generate_array({{ range.start}}, {{ range.end }}, {{ range.interval }})
+        )
+    {%- endif -%}
+{%- endmacro -%}
diff --git a/dbt/include/bigquery/macros/relations/rename.sql b/dbt/include/bigquery/macros/relations/rename.sql
new file mode 100644
index 000000000..c8abfea6f
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/rename.sql
@@ -0,0 +1,3 @@
+{% macro bigquery__rename_relation(from_relation, to_relation) -%}
+  {% do adapter.rename_relation(from_relation, to_relation) %}
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/table/options.sql b/dbt/include/bigquery/macros/relations/table/options.sql
new file mode 100644
index 000000000..9f9b6b6d1
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/table/options.sql
@@ -0,0 +1,4 @@
+{% macro bigquery_table_options(config, node, temporary) %}
+  {% set opts = adapter.get_table_options(config, node, temporary) %}
+  {%- do return(bigquery_options(opts)) -%}
+{%- endmacro -%}
diff --git a/dbt/include/bigquery/macros/relations/view/options.sql b/dbt/include/bigquery/macros/relations/view/options.sql
new file mode 100644
index 000000000..bed2176b9
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/view/options.sql
@@ -0,0 +1,4 @@
+{% macro bigquery_view_options(config, node) %}
+  {% set opts = adapter.get_view_options(config, node) %}
+  {%- do return(bigquery_options(opts)) -%}
+{%- endmacro -%}

From 0e34734d67be27b93a924c69f1711104eadb0044 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 31 Aug 2023 16:11:51 -0700
Subject: [PATCH 700/860] Fix Dataclass validation logic (#905)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* Set default BigQueryConnectionMethod to make dataclass validation happy

* add method validation

* add changie

* add schema validation check

* remove grant access change
---
 .../unreleased/Under the Hood-20230830-160707.yaml    |  6 ++++++
 dbt/adapters/bigquery/connections.py                  | 11 +++++++++--
 2 files changed, 15 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20230830-160707.yaml

diff --git a/.changes/unreleased/Under the Hood-20230830-160707.yaml b/.changes/unreleased/Under the Hood-20230830-160707.yaml
new file mode 100644
index 000000000..257e3cdbc
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20230830-160707.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Update BigQueryCredentials to support migration off hologram
+time: 2023-08-30T16:07:07.768609-07:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "906"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 15c983480..0d3c52ce6 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -104,10 +104,12 @@ def __init__(self, batch_config):
 
 @dataclass
 class BigQueryCredentials(Credentials):
-    method: BigQueryConnectionMethod
+    method: BigQueryConnectionMethod = None  # type: ignore
+
     # BigQuery allows an empty database / project, where it defers to the
     # environment for the project
-    database: Optional[str]  # type: ignore
+    database: Optional[str] = None  # type: ignore
+    schema: Optional[str] = None  # type: ignore
     execution_project: Optional[str] = None
     location: Optional[str] = None
     priority: Optional[Priority] = None
@@ -162,6 +164,11 @@ def __post_init__(self):
             self.keyfile_json["private_key"] = self.keyfile_json["private_key"].replace(
                 "\\n", "\n"
             )
+        if not self.method:
+            raise DbtRuntimeError("Must specify authentication method")
+
+        if not self.schema:
+            raise DbtRuntimeError("Must specify schema")
 
     @property
     def type(self):

From e1a12c3c09607595ec13cdcd61b9f60f15ce7fb9 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 1 Sep 2023 22:50:47 +0000
Subject: [PATCH 701/860] Update tox requirement from ~=4.8 to ~=4.9 (#884)

* Update tox requirement from ~=4.8 to ~=4.9

Updates the requirements on [tox](https://github.com/tox-dev/tox) to permit the latest version.
- [Release notes](https://github.com/tox-dev/tox/releases)
- [Changelog](https://github.com/tox-dev/tox/blob/main/docs/changelog.rst)
- [Commits](https://github.com/tox-dev/tox/compare/4.8.0...4.9.0)

---
updated-dependencies:
- dependency-name: tox
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20230817-003629.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230817-003629.yaml

diff --git a/.changes/unreleased/Dependencies-20230817-003629.yaml b/.changes/unreleased/Dependencies-20230817-003629.yaml
new file mode 100644
index 000000000..5bd27095f
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230817-003629.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update tox requirement from ~=4.8 to ~=4.9"
+time: 2023-08-17T00:36:29.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 884
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 685cbe387..15b3296af 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -23,7 +23,7 @@ pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
 pytest-xdist~=3.3
 pytz~=2023.3
-tox~=4.8
+tox~=4.9
 types-pytz~=2023.3
 types-protobuf~=4.24
 types-requests~=2.31

From 935c63ea27015718d92b548557149206a18246b4 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 1 Sep 2023 22:55:11 +0000
Subject: [PATCH 702/860] Bump mypy from 1.5.0 to 1.5.1 (#883)

* Bump mypy from 1.5.0 to 1.5.1

Bumps [mypy](https://github.com/python/mypy) from 1.5.0 to 1.5.1.
- [Commits](https://github.com/python/mypy/compare/v1.5.0...v1.5.1)

---
updated-dependencies:
- dependency-name: mypy
  dependency-type: direct:development
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20230817-003623.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230817-003623.yaml

diff --git a/.changes/unreleased/Dependencies-20230817-003623.yaml b/.changes/unreleased/Dependencies-20230817-003623.yaml
new file mode 100644
index 000000000..75723f321
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230817-003623.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump mypy from 1.5.0 to 1.5.1"
+time: 2023-08-17T00:36:23.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 883
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 15b3296af..6ba2749cc 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -13,7 +13,7 @@ flake8~=6.1
 flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
-mypy==1.5.0  # patch updates have historically introduced breaking changes
+mypy==1.5.1  # patch updates have historically introduced breaking changes
 pip-tools~=7.3
 pre-commit~=3.3
 pre-commit-hooks~=4.4

From 0a9ab72b006ad340092f5a1ab3ea184a8e5fc6da Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Mon, 11 Sep 2023 11:41:38 -0500
Subject: [PATCH 703/860] push up fix to core change in agate_helper (#916)

---
 .changes/unreleased/Fixes-20230907-161347.yaml | 6 ++++++
 tests/functional/adapter/expected_stats.py     | 6 +++---
 2 files changed, 9 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230907-161347.yaml

diff --git a/.changes/unreleased/Fixes-20230907-161347.yaml b/.changes/unreleased/Fixes-20230907-161347.yaml
new file mode 100644
index 000000000..b0309afc2
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230907-161347.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: changes expected value types to AnyInteger to take into account changes in core
+time: 2023-09-07T16:13:47.005796-05:00
+custom:
+  Author: McKnight-42
+  Issue: "915"
diff --git a/tests/functional/adapter/expected_stats.py b/tests/functional/adapter/expected_stats.py
index f4a1e022d..560d8de73 100644
--- a/tests/functional/adapter/expected_stats.py
+++ b/tests/functional/adapter/expected_stats.py
@@ -1,4 +1,4 @@
-from dbt.tests.util import AnyString, AnyFloat
+from dbt.tests.util import AnyString, AnyInteger
 
 
 def bigquery_stats(is_table, partition=None, cluster=None):
@@ -10,14 +10,14 @@ def bigquery_stats(is_table, partition=None, cluster=None):
                 "num_bytes": {
                     "id": "num_bytes",
                     "label": AnyString(),
-                    "value": AnyFloat(),
+                    "value": AnyInteger(),
                     "description": AnyString(),
                     "include": True,
                 },
                 "num_rows": {
                     "id": "num_rows",
                     "label": AnyString(),
-                    "value": AnyFloat(),
+                    "value": AnyInteger(),
                     "description": AnyString(),
                     "include": True,
                 },

From 2eb407d27fddb839946d727c9d13a398f9f3ec1f Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Mon, 11 Sep 2023 11:11:59 -0700
Subject: [PATCH 704/860] Timeout BQ queries  (#902)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* experiment with query job cancel on timeout

* modify unit tests

* remove test grants change

* starting functional test

* update functional test and experiment with polling logic

* experiment with async wait_for

* modifying connections.py for asyncio logic

* swap back to new_event_loop

* close loop, now seeing asyncio timeoutError

* improve order and update functional test

* update unit test

* add changie

* add max_result back to result call in async path

* rescope the dbt_profile_target to being a class fixture

* raise DbtRuntimeError instead database

* remove exception type check in job timeout

---------

Co-authored-by: Matthew McKnight <matthew.mcknight@dbtlabs.com>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .../unreleased/Fixes-20230829-162111.yaml     |  6 ++
 dbt/adapters/bigquery/connections.py          | 23 ++++++-
 tests/conftest.py                             |  2 +-
 tests/functional/test_job_timeout.py          | 62 +++++++++++++++++++
 tests/unit/test_bigquery_adapter.py           | 25 +++++++-
 5 files changed, 114 insertions(+), 4 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230829-162111.yaml
 create mode 100644 tests/functional/test_job_timeout.py

diff --git a/.changes/unreleased/Fixes-20230829-162111.yaml b/.changes/unreleased/Fixes-20230829-162111.yaml
new file mode 100644
index 000000000..5d34acd3e
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230829-162111.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Time out queries if user supplies `job_execution_timeout`
+time: 2023-08-29T16:21:11.69291-07:00
+custom:
+  Author: colin-rogers-dbt McKnight-42
+  Issue: "231"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 0d3c52ce6..c136042c3 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -1,7 +1,10 @@
+import asyncio
+import functools
 import json
 import re
 from contextlib import contextmanager
 from dataclasses import dataclass, field
+
 from mashumaro.helper import pass_through
 
 from functools import lru_cache
@@ -710,7 +713,6 @@ def _query_and_results(
         # Cannot reuse job_config if destination is set and ddl is used
         job_config = google.cloud.bigquery.QueryJobConfig(**job_params)
         query_job = client.query(query=sql, job_config=job_config, timeout=job_creation_timeout)
-
         if (
             query_job.location is not None
             and query_job.job_id is not None
@@ -720,8 +722,25 @@ def _query_and_results(
                 self._bq_job_link(query_job.location, query_job.project, query_job.job_id)
             )
 
-        iterator = query_job.result(max_results=limit, timeout=job_execution_timeout)
+        # only use async logic if user specifies a timeout
+        if job_execution_timeout:
+            loop = asyncio.new_event_loop()
+            future_iterator = asyncio.wait_for(
+                loop.run_in_executor(None, functools.partial(query_job.result, max_results=limit)),
+                timeout=job_execution_timeout,
+            )
 
+            try:
+                iterator = loop.run_until_complete(future_iterator)
+            except asyncio.TimeoutError:
+                query_job.cancel()
+                raise DbtRuntimeError(
+                    f"Query exceeded configured timeout of {job_execution_timeout}s"
+                )
+            finally:
+                loop.close()
+        else:
+            iterator = query_job.result(max_results=limit)
         return query_job, iterator
 
     def _retry_and_handle(self, msg, conn, fn):
diff --git a/tests/conftest.py b/tests/conftest.py
index 0ba0091fb..78f3d82e1 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -12,7 +12,7 @@ def pytest_addoption(parser):
     parser.addoption("--profile", action="store", default="oauth", type=str)
 
 
-@pytest.fixture(scope="session")
+@pytest.fixture(scope="class")
 def dbt_profile_target(request):
     profile_type = request.config.getoption("--profile")
     if profile_type == "oauth":
diff --git a/tests/functional/test_job_timeout.py b/tests/functional/test_job_timeout.py
new file mode 100644
index 000000000..be559e816
--- /dev/null
+++ b/tests/functional/test_job_timeout.py
@@ -0,0 +1,62 @@
+import pytest
+
+from dbt.tests.util import run_dbt
+
+_REASONABLE_TIMEOUT = 300
+_SHORT_TIMEOUT = 1
+
+_LONG_RUNNING_MODEL_SQL = """
+    {{ config(materialized='table') }}
+    with array_1 as (
+    select generated_ids from UNNEST(GENERATE_ARRAY(1, 200000)) AS generated_ids
+    ),
+    array_2 as (
+    select generated_ids from UNNEST(GENERATE_ARRAY(2, 200000)) AS generated_ids
+    )
+
+    SELECT array_1.generated_ids
+    FROM array_1
+    LEFT JOIN array_1 as jnd on 1=1
+    LEFT JOIN array_2 as jnd2 on 1=1
+    LEFT JOIN array_1 as jnd3 on jnd3.generated_ids >= jnd2.generated_ids
+"""
+
+_SHORT_RUNNING_QUERY = """
+    SELECT 1 as id
+    """
+
+
+class TestSuccessfulJobRun:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "model.sql": _SHORT_RUNNING_QUERY,
+        }
+
+    @pytest.fixture(scope="class")
+    def profiles_config_update(self, dbt_profile_target):
+        outputs = {"default": dbt_profile_target}
+        outputs["default"]["job_execution_timeout_seconds"] = _REASONABLE_TIMEOUT
+        return {"test": {"outputs": outputs, "target": "default"}}
+
+    def test_bigquery_job_run_succeeds_within_timeout(self, project):
+        result = run_dbt()
+        assert len(result) == 1
+
+
+class TestJobTimeout:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "model.sql": _LONG_RUNNING_MODEL_SQL,
+        }
+
+    @pytest.fixture(scope="class")
+    def profiles_config_update(self, dbt_profile_target):
+        outputs = {"default": dbt_profile_target}
+        outputs["default"]["job_execution_timeout_seconds"] = _SHORT_TIMEOUT
+        return {"test": {"outputs": outputs, "target": "default"}}
+
+    def test_job_timeout(self, project):
+        result = run_dbt(["run"], expect_pass=False)  # project setup will fail
+        assert f"Query exceeded configured timeout of {_SHORT_TIMEOUT}s" in result[0].message
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index bb98db86f..10cb3f530 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -1,3 +1,5 @@
+import time
+
 import agate
 import decimal
 import json
@@ -634,18 +636,39 @@ def test_drop_dataset(self):
 
     @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
     def test_query_and_results(self, mock_bq):
+        self.mock_client.query = Mock(return_value=Mock(state="DONE"))
         self.connections._query_and_results(
             self.mock_client,
             "sql",
             {"job_param_1": "blah"},
             job_creation_timeout=15,
-            job_execution_timeout=100,
+            job_execution_timeout=3,
+        )
+
+        mock_bq.QueryJobConfig.assert_called_once()
+        self.mock_client.query.assert_called_once_with(
+            query="sql", job_config=mock_bq.QueryJobConfig(), timeout=15
         )
 
+    @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
+    def test_query_and_results_timeout(self, mock_bq):
+        self.mock_client.query = Mock(
+            return_value=Mock(result=lambda *args, **kwargs: time.sleep(4))
+        )
+        with pytest.raises(dbt.exceptions.DbtRuntimeError) as exc:
+            self.connections._query_and_results(
+                self.mock_client,
+                "sql",
+                {"job_param_1": "blah"},
+                job_creation_timeout=15,
+                job_execution_timeout=1,
+            )
+
         mock_bq.QueryJobConfig.assert_called_once()
         self.mock_client.query.assert_called_once_with(
             query="sql", job_config=mock_bq.QueryJobConfig(), timeout=15
         )
+        assert "Query exceeded configured timeout of 1s" in str(exc.value)
 
     def test_copy_bq_table_appends(self):
         self._copy_table(write_disposition=dbt.adapters.bigquery.impl.WRITE_APPEND)

From 7bc2921cdb631e926cac24bad096e695b5fd8981 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 12 Sep 2023 00:21:07 +0000
Subject: [PATCH 705/860] Update black requirement from ~=23.7 to ~=23.9 (#920)

* Update black requirement from ~=23.7 to ~=23.9

Updates the requirements on [black](https://github.com/psf/black) to permit the latest version.
- [Release notes](https://github.com/psf/black/releases)
- [Changelog](https://github.com/psf/black/blob/main/CHANGES.md)
- [Commits](https://github.com/psf/black/compare/23.7.0...23.9.1)

---
updated-dependencies:
- dependency-name: black
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20230911-010957.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230911-010957.yaml

diff --git a/.changes/unreleased/Dependencies-20230911-010957.yaml b/.changes/unreleased/Dependencies-20230911-010957.yaml
new file mode 100644
index 000000000..e07dd91b1
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230911-010957.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update black requirement from ~=23.7 to ~=23.9"
+time: 2023-09-11T01:09:57.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 920
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 6ba2749cc..9859a5299 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -5,7 +5,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor
-black~=23.7
+black~=23.9
 bumpversion~=0.6.0
 click~=8.1
 ddtrace~=1.18

From af3e7ef99a8458c883c6514c5a32a0e0474969c0 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 12 Sep 2023 00:26:35 +0000
Subject: [PATCH 706/860] Update pre-commit requirement from ~=3.3 to ~=3.4
 (#909)

* Update pre-commit requirement from ~=3.3 to ~=3.4

Updates the requirements on [pre-commit](https://github.com/pre-commit/pre-commit) to permit the latest version.
- [Release notes](https://github.com/pre-commit/pre-commit/releases)
- [Changelog](https://github.com/pre-commit/pre-commit/blob/main/CHANGELOG.md)
- [Commits](https://github.com/pre-commit/pre-commit/compare/v3.3.0...v3.4.0)

---
updated-dependencies:
- dependency-name: pre-commit
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20230904-011145.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230904-011145.yaml

diff --git a/.changes/unreleased/Dependencies-20230904-011145.yaml b/.changes/unreleased/Dependencies-20230904-011145.yaml
new file mode 100644
index 000000000..e94fa0ea7
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230904-011145.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pre-commit requirement from ~=3.3 to ~=3.4"
+time: 2023-09-04T01:11:45.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 909
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 9859a5299..829d27bf6 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -15,7 +15,7 @@ freezegun~=1.2
 ipdb~=0.13.13
 mypy==1.5.1  # patch updates have historically introduced breaking changes
 pip-tools~=7.3
-pre-commit~=3.3
+pre-commit~=3.4
 pre-commit-hooks~=4.4
 pytest~=7.4
 pytest-csv~=3.0

From cf60e3ea38ebf216e940944ec16627680adbf5c5 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 12 Sep 2023 02:05:24 +0000
Subject: [PATCH 707/860] Update tox requirement from ~=4.9 to ~=4.11 (#907)

* Update tox requirement from ~=4.9 to ~=4.11

Updates the requirements on [tox](https://github.com/tox-dev/tox) to permit the latest version.
- [Release notes](https://github.com/tox-dev/tox/releases)
- [Changelog](https://github.com/tox-dev/tox/blob/main/docs/changelog.rst)
- [Commits](https://github.com/tox-dev/tox/compare/4.9.0...4.11.1)

---
updated-dependencies:
- dependency-name: tox
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20230901-225226.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230901-225226.yaml

diff --git a/.changes/unreleased/Dependencies-20230901-225226.yaml b/.changes/unreleased/Dependencies-20230901-225226.yaml
new file mode 100644
index 000000000..794c94ef7
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230901-225226.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update tox requirement from ~=4.9 to ~=4.11"
+time: 2023-09-01T22:52:26.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 907
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 829d27bf6..54ca40f26 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -23,7 +23,7 @@ pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
 pytest-xdist~=3.3
 pytz~=2023.3
-tox~=4.9
+tox~=4.11
 types-pytz~=2023.3
 types-protobuf~=4.24
 types-requests~=2.31

From 45e5ce5331781918a5bd126ff96ff564213a5c75 Mon Sep 17 00:00:00 2001
From: Christophe Oudar <kayrnt@gmail.com>
Date: Wed, 13 Sep 2023 01:20:22 +0200
Subject: [PATCH 708/860] quote column names in data types (#919)

* Quote column names in struct data types to fix time ingestion partitioning table creation

---------

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20230911-005506.yaml       |  7 +++++++
 dbt/adapters/bigquery/column.py                      |  2 +-
 .../incremental/test_incremental_on_schema_change.py | 12 ++++++++----
 3 files changed, 16 insertions(+), 5 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230911-005506.yaml

diff --git a/.changes/unreleased/Fixes-20230911-005506.yaml b/.changes/unreleased/Fixes-20230911-005506.yaml
new file mode 100644
index 000000000..16f15f846
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230911-005506.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: Quote column names in struct data types to fix time ingestion partitioning table
+  creation
+time: 2023-09-11T00:55:06.904238+02:00
+custom:
+  Author: Kayrnt
+  Issue: "913"
diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py
index a5a60cfc0..1820a6ba7 100644
--- a/dbt/adapters/bigquery/column.py
+++ b/dbt/adapters/bigquery/column.py
@@ -82,7 +82,7 @@ def literal(self, value):
     def data_type(self) -> str:
         if self.dtype.upper() == "RECORD":
             subcols = [
-                "{} {}".format(col.name, col.data_type) for col in self.fields  # type: ignore[attr-defined]
+                "{} {}".format(col.quoted, col.data_type) for col in self.fields  # type: ignore[attr-defined]
             ]
             field_type = "STRUCT<{}>".format(", ".join(subcols))
 
diff --git a/tests/functional/adapter/incremental/test_incremental_on_schema_change.py b/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
index 60f4500df..65e855241 100644
--- a/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
+++ b/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
@@ -82,19 +82,23 @@ class TestIncrementalOnSchemaChange(BaseIncrementalOnSchemaChange):
         select 1 as id,
         cast('2020-01-01 01:00:00' as datetime) as date_hour,
         1 as field_1,
-        2 as field_2 union all
+        2 as field_2,
+        STRUCT(1 as `group`, 2 as `WHERE`, 3 as group_2, 4 as WHERE_TO) as field_struct union all
         select 2 as id,
         cast('2020-01-01 01:00:00' as datetime) as date_hour,
         1 as field_1,
-        2 as field_2 union all
+        2 as field_2,
+        STRUCT(1 as `group`, 2 as `WHERE`, 3 as group_2, 4 as WHERE_TO) union all
         select 3 as id,
         cast('2020-01-01 01:00:00' as datetime) as date_hour,
         1 as field_1,
-        2 as field_2 union all
+        2 as field_2,
+        STRUCT(2 as `group`, 2 as `WHERE`, 3 as group_2, 4 as WHERE_TO) union all
         select 4 as id,
         cast('2020-01-01 01:00:00' as datetime) as date_hour,
         1 as field_1,
-        2 as field_2
+        2 as field_2,
+        STRUCT(2 as `group`, 2 as `WHERE`, 3 as group_2, 4 as WHERE_TO)
 
     {% else %}
 

From ffe2175440e263e71f122a0b707b51ba2dfeeb54 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 13 Sep 2023 13:32:29 -0400
Subject: [PATCH 709/860] ADAP-400: Support materialized views with create/drop
 functionality (#639)

* update `dev-requirements.txt` to point to the feature branch on `dbt-core`

* organized materializations directory

* organized materializations directory

* added materialized view stubs

* added materialized view test case

* accounted for ordered lists in test case

* changie

* updated record type to List[tuple]

* updated file structure to new macros structure

* reverting to old structure for non-MV relations

* reverting to old structure for non-MV relations

* reverting to old structure for non-MV relations

* updating to move towards adap-400

* remove db api files as shift was considered out of scope

* remove strategy files as shift was considered out of scope

* point back to main branch as adap-2 has been merged

* start to build out base tests and modify input format for some macros

* add very rough draft of bigquery version of materialization

* continued work on bigquery mat view materialization

* eod update

* break out MV-relevant macros in adapters.sql into separate files in macros/relations

* changie

* update `dev-requirements.txt` to point to the feature branch on `dbt-core`

* organized materializations directory

* organized materializations directory

* added materialized view stubs

* added materialized view test case

* accounted for ordered lists in test case

* updated record type to List[tuple]

* changie

* updated file structure to new macros structure

* reverting to old structure for non-MV relations

* reverting to old structure for non-MV relations

* reverting to old structure for non-MV relations

* updating to move towards adap-400

* remove db api files as shift was considered out of scope

* remove strategy files as shift was considered out of scope

* point back to main branch as adap-2 has been merged

* start to build out base tests and modify input format for some macros

* add very rough draft of bigquery version of materialization

* continued work on bigquery mat view materialization

* eod update

* fix whitespacing from rebase

* point back to dbt-core/main since MVs are merged in dbt-core

* fix rebasing misses

* revert unrelated test file migration

* move materialized view macros into the new macro file structure

* refactor after pair, still failing for dropping uncreated backup

* pull create or replace view into BQ, added drop statements for replace functionality, fixed signatures on some macros, added new type to adapter class

* revert dev-requirements.txt change

* implement rename for table and view

* updated renameable and replaceable relations

* updated renameable and replaceable relations to use default factory

* updated renameable and replaceable relations to use frozenset

* revert dev-requirements.txt to point back to main

---------

Co-authored-by: Matthew McKnight <matthew.mcknight@dbtlabs.com>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .../unreleased/Features-20230330-165721.yaml  |  6 ++
 dbt/adapters/bigquery/impl.py                 |  4 +
 dbt/adapters/bigquery/relation.py             |  4 +
 .../materializations/materialized_view.sql    |  8 ++
 .../bigquery/macros/materializations/view.sql |  2 +-
 .../bigquery/macros/relations/drop.sql        |  4 +-
 .../relations/materialized_view/_replace.sql  |  7 ++
 .../relations/materialized_view/alter.sql     | 15 ++++
 .../relations/materialized_view/create.sql    |  3 +
 .../relations/materialized_view/drop.sql      |  3 +
 .../relations/materialized_view/refresh.sql   |  3 +
 .../bigquery/macros/relations/table/drop.sql  |  3 +
 .../macros/relations/table/rename.sql         |  3 +
 .../bigquery/macros/relations/view/drop.sql   |  3 +
 .../bigquery/macros/relations/view/rename.sql |  3 +
 .../macros/relations/view/replace.sql         | 54 +++++++++++++
 .../test_materialized_view.py                 | 79 +++++++++++++++++++
 .../adapter/materialized_view_tests/utils.py  | 13 +++
 18 files changed, 213 insertions(+), 4 deletions(-)
 create mode 100644 .changes/unreleased/Features-20230330-165721.yaml
 create mode 100644 dbt/include/bigquery/macros/materializations/materialized_view.sql
 create mode 100644 dbt/include/bigquery/macros/relations/materialized_view/_replace.sql
 create mode 100644 dbt/include/bigquery/macros/relations/materialized_view/alter.sql
 create mode 100644 dbt/include/bigquery/macros/relations/materialized_view/create.sql
 create mode 100644 dbt/include/bigquery/macros/relations/materialized_view/drop.sql
 create mode 100644 dbt/include/bigquery/macros/relations/materialized_view/refresh.sql
 create mode 100644 dbt/include/bigquery/macros/relations/table/drop.sql
 create mode 100644 dbt/include/bigquery/macros/relations/table/rename.sql
 create mode 100644 dbt/include/bigquery/macros/relations/view/drop.sql
 create mode 100644 dbt/include/bigquery/macros/relations/view/rename.sql
 create mode 100644 dbt/include/bigquery/macros/relations/view/replace.sql
 create mode 100644 tests/functional/adapter/materialized_view_tests/test_materialized_view.py
 create mode 100644 tests/functional/adapter/materialized_view_tests/utils.py

diff --git a/.changes/unreleased/Features-20230330-165721.yaml b/.changes/unreleased/Features-20230330-165721.yaml
new file mode 100644
index 000000000..02ce58447
--- /dev/null
+++ b/.changes/unreleased/Features-20230330-165721.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Add support for materialized views
+time: 2023-03-30T16:57:21.858512-04:00
+custom:
+  Author: mikealfare McKnight-42
+  Issue: dbt-labs/dbt-core#6911
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 3f8351861..bb04c78b8 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -190,12 +190,16 @@ class BigqueryConfig(AdapterConfig):
     require_partition_filter: Optional[bool] = None
     partition_expiration_days: Optional[int] = None
     merge_update_columns: Optional[str] = None
+    enable_refresh: Optional[bool] = None
+    refresh_interval_minutes: Optional[int] = None
+    max_staleness: Optional[str] = None
 
 
 class BigQueryAdapter(BaseAdapter):
     RELATION_TYPES = {
         "TABLE": RelationType.Table,
         "VIEW": RelationType.View,
+        "MATERIALIZED_VIEW": RelationType.MaterializedView,
         "EXTERNAL": RelationType.External,
     }
 
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index d49677168..37ba33632 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -4,6 +4,7 @@
 from itertools import chain, islice
 
 from dbt.adapters.base.relation import BaseRelation, ComponentName, InformationSchema
+from dbt.contracts.relation import RelationType
 from dbt.exceptions import CompilationError
 from dbt.utils import filter_null_values
 from typing import TypeVar
@@ -16,6 +17,9 @@
 class BigQueryRelation(BaseRelation):
     quote_character: str = "`"
     location: Optional[str] = None
+    # why do we need to use default_factory here but we can assign it directly in dbt-postgres?
+    renameable_relations = frozenset({RelationType.Table})
+    replaceable_relations = frozenset({RelationType.Table, RelationType.View})
 
     def matches(
         self,
diff --git a/dbt/include/bigquery/macros/materializations/materialized_view.sql b/dbt/include/bigquery/macros/materializations/materialized_view.sql
new file mode 100644
index 000000000..8b4168aec
--- /dev/null
+++ b/dbt/include/bigquery/macros/materializations/materialized_view.sql
@@ -0,0 +1,8 @@
+{% macro materialized_view_setup(backup_relation, intermediate_relation, pre_hooks) %}
+    {{ run_hooks(pre_hooks, inside_transaction=False) }}
+{% endmacro %}
+
+
+{% macro materialized_view_teardown(backup_relation, intermediate_relation, post_hooks) %}
+    {{ run_hooks(post_hooks, inside_transaction=False) }}
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/materializations/view.sql b/dbt/include/bigquery/macros/materializations/view.sql
index e68a51421..fd05129f9 100644
--- a/dbt/include/bigquery/macros/materializations/view.sql
+++ b/dbt/include/bigquery/macros/materializations/view.sql
@@ -12,7 +12,7 @@
     -- grab current tables grants config for comparision later on
     {% set grant_config = config.get('grants') %}
 
-    {% set to_return = create_or_replace_view() %}
+    {% set to_return = bigquery__create_or_replace_view() %}
 
     {% set target_relation = this.incorporate(type='view') %}
 
diff --git a/dbt/include/bigquery/macros/relations/drop.sql b/dbt/include/bigquery/macros/relations/drop.sql
index 89e5e260c..7a50704a0 100644
--- a/dbt/include/bigquery/macros/relations/drop.sql
+++ b/dbt/include/bigquery/macros/relations/drop.sql
@@ -1,5 +1,3 @@
 {% macro bigquery__drop_relation(relation) -%}
-  {% call statement('drop_relation') -%}
-    drop {{ relation.type }} if exists {{ relation }}
-  {%- endcall %}
+    {% do adapter.drop_relation(relation) %}
 {% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/_replace.sql b/dbt/include/bigquery/macros/relations/materialized_view/_replace.sql
new file mode 100644
index 000000000..c9df0aef6
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/materialized_view/_replace.sql
@@ -0,0 +1,7 @@
+{% macro bigquery__get_replace_materialized_view_as_sql(
+    relation,
+    sql
+) %}
+    {{ get_drop_sql(existing_relation) }}
+    {{ get_create_materialized_view_as_sql(relation, sql) }}
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/alter.sql b/dbt/include/bigquery/macros/relations/materialized_view/alter.sql
new file mode 100644
index 000000000..27819c948
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/materialized_view/alter.sql
@@ -0,0 +1,15 @@
+{% macro bigquery__get_alter_materialized_view_as_sql(
+    relation,
+    configuration_changes,
+    sql,
+    existing_relation
+) %}
+    bigquery__get_replace_materialized_view_as_sql(
+        relation,
+        sql
+    )
+{% endmacro %}
+
+{% macro bigquery__get_materialized_view_configuration_changes(existing_relation, new_config) %}
+    {{- return(None) -}}
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/create.sql b/dbt/include/bigquery/macros/relations/materialized_view/create.sql
new file mode 100644
index 000000000..198a3a04a
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/materialized_view/create.sql
@@ -0,0 +1,3 @@
+{% macro bigquery__get_create_materialized_view_as_sql(relation, sql) %}
+    create materialized view if not exists {{ relation }} as {{ sql }}
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/drop.sql b/dbt/include/bigquery/macros/relations/materialized_view/drop.sql
new file mode 100644
index 000000000..76e87f7fe
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/materialized_view/drop.sql
@@ -0,0 +1,3 @@
+{% macro bigquery__drop_materialized_view(relation) %}
+    drop materialized view if exists {{ relation }}
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql b/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql
new file mode 100644
index 000000000..54ad76e1e
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql
@@ -0,0 +1,3 @@
+{% macro bigquery__refresh_materialized_view(relation) %}
+    CALL BQ.REFRESH_MATERIALIZED_VIEW('{{ relation.database }}.{{ relation.schema }}.{{ relation.identifier }}');
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/table/drop.sql b/dbt/include/bigquery/macros/relations/table/drop.sql
new file mode 100644
index 000000000..cffb41e06
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/table/drop.sql
@@ -0,0 +1,3 @@
+{% macro bigquery__drop_table(relation) %}
+    drop table if exists {{ relation }}
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/table/rename.sql b/dbt/include/bigquery/macros/relations/table/rename.sql
new file mode 100644
index 000000000..eff0d5d7f
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/table/rename.sql
@@ -0,0 +1,3 @@
+{%- macro bigquery__get_rename_table_sql(relation, new_name) -%}
+    alter table {{ relation }} rename to {{ new_name }}
+{%- endmacro -%}
diff --git a/dbt/include/bigquery/macros/relations/view/drop.sql b/dbt/include/bigquery/macros/relations/view/drop.sql
new file mode 100644
index 000000000..6f269af1d
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/view/drop.sql
@@ -0,0 +1,3 @@
+{% macro bigquery__drop_view(relation) %}
+    drop view if exists {{ relation }}
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/view/rename.sql b/dbt/include/bigquery/macros/relations/view/rename.sql
new file mode 100644
index 000000000..f21a2868c
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/view/rename.sql
@@ -0,0 +1,3 @@
+{%- macro bigquery__get_rename_view_sql(relation, new_name) -%}
+    alter view {{ relation }} rename to {{ new_name }}
+{%- endmacro -%}
diff --git a/dbt/include/bigquery/macros/relations/view/replace.sql b/dbt/include/bigquery/macros/relations/view/replace.sql
new file mode 100644
index 000000000..9799eaf87
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/view/replace.sql
@@ -0,0 +1,54 @@
+/* {#
+       Core materialization implementation. BigQuery and Snowflake are similar
+       because both can use `create or replace view` where the resulting view's columns
+       are not necessarily the same as those of the existing view. On Redshift, this would
+       result in: ERROR:  cannot change number of columns in view
+
+       This implementation is superior to the create_temp, swap_with_existing, drop_old
+       paradigm because transactions don't run DDL queries atomically on Snowflake. By using
+       `create or replace view`, the materialization becomes atomic in nature.
+#} */
+
+{% macro bigquery__create_or_replace_view() %}
+  {%- set identifier = model['alias'] -%}
+
+  {%- set old_relation = adapter.get_relation(database=database, schema=schema, identifier=identifier) -%}
+  {%- set exists_as_view = (old_relation is not none and old_relation.is_view) -%}
+
+  {%- set target_relation = api.Relation.create(
+      identifier=identifier, schema=schema, database=database,
+      type='view') -%}
+  {% set grant_config = config.get('grants') %}
+
+  {{ run_hooks(pre_hooks) }}
+
+  -- If there's a table with the same name and we weren't told to full refresh,
+  -- that's an error. If we were told to full refresh, drop it. This behavior differs
+  -- for Snowflake and BigQuery, so multiple dispatch is used.
+  {%- if old_relation is not none and not old_relation.is_view -%}
+    {{ handle_existing_table(should_full_refresh(), old_relation) }}
+  {%- endif -%}
+
+  -- build model
+  {% call statement('main') -%}
+    {{ get_create_view_as_sql(target_relation, sql) }}
+  {%- endcall %}
+
+  {% set should_revoke = should_revoke(exists_as_view, full_refresh_mode=True) %}
+  {% do apply_grants(target_relation, grant_config, should_revoke=should_revoke) %}
+
+  {{ run_hooks(post_hooks) }}
+
+  {{ return({'relations': [target_relation]}) }}
+
+{% endmacro %}
+
+
+{% macro handle_existing_table(full_refresh, old_relation) %}
+    {{ adapter.dispatch('handle_existing_table', 'dbt')(full_refresh, old_relation) }}
+{% endmacro %}
+
+{% macro default__handle_existing_table(full_refresh, old_relation) %}
+    {{ log("Dropping relation " ~ old_relation ~ " because it is of type " ~ old_relation.type) }}
+    {{ adapter.drop_relation(old_relation) }}
+{% endmacro %}
diff --git a/tests/functional/adapter/materialized_view_tests/test_materialized_view.py b/tests/functional/adapter/materialized_view_tests/test_materialized_view.py
new file mode 100644
index 000000000..03d588cd4
--- /dev/null
+++ b/tests/functional/adapter/materialized_view_tests/test_materialized_view.py
@@ -0,0 +1,79 @@
+from typing import Optional, Tuple
+
+import pytest
+
+from dbt.adapters.base.relation import BaseRelation
+from dbt.tests.util import get_connection, run_dbt
+from dbt.tests.adapter.materialized_view.basic import MaterializedViewBasic
+
+from dbt.tests.adapter.materialized_view.files import MY_TABLE, MY_VIEW
+
+
+MY_MATERIALIZED_VIEW = """
+{{ config(
+    materialized='materialized_view'
+) }}
+select * from {{ ref('my_seed') }}
+"""
+
+
+class TestBigqueryMaterializedViewsBasic(MaterializedViewBasic):
+    @pytest.fixture(scope="class", autouse=True)
+    def models(self):
+        yield {
+            "my_table.sql": MY_TABLE,
+            "my_view.sql": MY_VIEW,
+            "my_materialized_view.sql": MY_MATERIALIZED_VIEW,
+        }
+
+    @staticmethod
+    def insert_record(project, table: BaseRelation, record: Tuple[int, int]):
+        my_id, value = record
+        project.run_sql(f"insert into {table} (id, value) values ({my_id}, {value})")
+
+    @staticmethod
+    def refresh_materialized_view(project, materialized_view: BaseRelation):
+        sql = f"""
+        call bq.refresh_materialized_view(
+            '{materialized_view.database}.{materialized_view.schema}.{materialized_view.identifier}'
+        )
+        """
+        project.run_sql(sql)
+
+    @staticmethod
+    def query_row_count(project, relation: BaseRelation) -> int:
+        sql = f"select count(*) from {relation}"
+        return project.run_sql(sql, fetch="one")[0]
+
+    # look into syntax
+    @staticmethod
+    def query_relation_type(project, relation: BaseRelation) -> Optional[str]:
+        with get_connection(project.adapter) as conn:
+            table = conn.handle.get_table(
+                project.adapter.connections.get_bq_table(
+                    relation.database, relation.schema, relation.identifier
+                )
+            )
+        return table.table_type.lower()
+
+    def test_view_replaces_materialized_view(self, project, my_materialized_view):
+        """
+        We don't support replacing a view with another object in dbt-bigquery unless we use --full-refresh
+        """
+        run_dbt(["run", "--models", my_materialized_view.identifier])
+        assert self.query_relation_type(project, my_materialized_view) == "materialized_view"
+
+        self.swap_materialized_view_to_view(project, my_materialized_view)
+
+        run_dbt(
+            ["run", "--models", my_materialized_view.identifier, "--full-refresh"]
+        )  # add --full-refresh
+        assert self.query_relation_type(project, my_materialized_view) == "view"
+
+    @pytest.mark.skip(
+        "It appears BQ updates the materialized view almost immediately, which fails this test."
+    )
+    def test_materialized_view_only_updates_after_refresh(
+        self, project, my_materialized_view, my_seed
+    ):
+        pass
diff --git a/tests/functional/adapter/materialized_view_tests/utils.py b/tests/functional/adapter/materialized_view_tests/utils.py
new file mode 100644
index 000000000..25e40d337
--- /dev/null
+++ b/tests/functional/adapter/materialized_view_tests/utils.py
@@ -0,0 +1,13 @@
+from dbt.adapters.bigquery.relation import BigQueryRelation
+
+
+def query_autorefresh(project, relation: BigQueryRelation) -> bool:
+    sql = f"""
+        select
+            case mv.autorefresh when 't' then True when 'f' then False end as autorefresh
+        from stv_mv_info mv
+        where trim(mv.name) ilike '{ relation.identifier }'
+        and trim(mv.schema) ilike '{ relation.schema }'
+        and trim(mv.db_name) ilike '{ relation.database }'
+    """
+    return project.run_sql(sql, fetch="one")[0]

From 315ea7e061c2611ae2f630c7355a6b0f149c7608 Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Fri, 15 Sep 2023 16:30:08 +0000
Subject: [PATCH 710/860] Bumping version to 1.7.0b2 and generate changelog

---
 .bumpversion.cfg                              |  2 +-
 .changes/1.7.0-b2.md                          | 28 +++++++++++++++++
 .../Dependencies-20230817-003623.yaml         |  0
 .../Dependencies-20230817-003629.yaml         |  0
 .../Dependencies-20230901-225226.yaml         |  0
 .../Dependencies-20230904-011145.yaml         |  0
 .../Dependencies-20230911-010957.yaml         |  0
 .../Features-20230330-165721.yaml             |  0
 .../Fixes-20230818-214616.yaml                |  0
 .../Fixes-20230829-162111.yaml                |  0
 .../Fixes-20230907-161347.yaml                |  0
 .../Fixes-20230911-005506.yaml                |  0
 .../Under the Hood-20230829-122918.yaml       |  0
 .../Under the Hood-20230830-160707.yaml       |  0
 CHANGELOG.md                                  | 31 ++++++++++++++++++-
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 17 files changed, 61 insertions(+), 4 deletions(-)
 create mode 100644 .changes/1.7.0-b2.md
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230817-003623.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230817-003629.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230901-225226.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230904-011145.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Dependencies-20230911-010957.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Features-20230330-165721.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Fixes-20230818-214616.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Fixes-20230829-162111.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Fixes-20230907-161347.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Fixes-20230911-005506.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Under the Hood-20230829-122918.yaml (100%)
 rename .changes/{unreleased => 1.7.0}/Under the Hood-20230830-160707.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index ff82260ee..8693b3160 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.7.0b1
+current_version = 1.7.0b2
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.7.0-b2.md b/.changes/1.7.0-b2.md
new file mode 100644
index 000000000..0dd5d798b
--- /dev/null
+++ b/.changes/1.7.0-b2.md
@@ -0,0 +1,28 @@
+## dbt-bigquery 1.7.0-b2 - September 15, 2023
+
+### Features
+
+- Add support for materialized views ([#dbt-labs/dbt-core#6911](https://github.com/dbt-labs/dbt-bigquery/issues/dbt-labs/dbt-core#6911))
+
+### Fixes
+
+- case insensitive check on partition matching ([#886](https://github.com/dbt-labs/dbt-bigquery/issues/886))
+- Time out queries if user supplies `job_execution_timeout` ([#231](https://github.com/dbt-labs/dbt-bigquery/issues/231))
+- changes expected value types to AnyInteger to take into account changes in core ([#915](https://github.com/dbt-labs/dbt-bigquery/issues/915))
+- Quote column names in struct data types to fix time ingestion partitioning table creation ([#913](https://github.com/dbt-labs/dbt-bigquery/issues/913))
+
+### Under the Hood
+
+- Re-organize adapters.sql into more granular files inside of macros/relations ([#904](https://github.com/dbt-labs/dbt-bigquery/issues/904))
+- Update BigQueryCredentials to support migration off hologram ([#906](https://github.com/dbt-labs/dbt-bigquery/issues/906))
+
+### Dependencies
+
+- Bump mypy from 1.5.0 to 1.5.1 ([#883](https://github.com/dbt-labs/dbt-bigquery/pull/883))
+- Update tox requirement from ~=4.8 to ~=4.9 ([#884](https://github.com/dbt-labs/dbt-bigquery/pull/884))
+- Update tox requirement from ~=4.9 to ~=4.11 ([#907](https://github.com/dbt-labs/dbt-bigquery/pull/907))
+- Update pre-commit requirement from ~=3.3 to ~=3.4 ([#909](https://github.com/dbt-labs/dbt-bigquery/pull/909))
+- Update black requirement from ~=23.7 to ~=23.9 ([#920](https://github.com/dbt-labs/dbt-bigquery/pull/920))
+
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#886](https://github.com/dbt-labs/dbt-bigquery/issues/886), [#913](https://github.com/dbt-labs/dbt-bigquery/issues/913))
diff --git a/.changes/unreleased/Dependencies-20230817-003623.yaml b/.changes/1.7.0/Dependencies-20230817-003623.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230817-003623.yaml
rename to .changes/1.7.0/Dependencies-20230817-003623.yaml
diff --git a/.changes/unreleased/Dependencies-20230817-003629.yaml b/.changes/1.7.0/Dependencies-20230817-003629.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230817-003629.yaml
rename to .changes/1.7.0/Dependencies-20230817-003629.yaml
diff --git a/.changes/unreleased/Dependencies-20230901-225226.yaml b/.changes/1.7.0/Dependencies-20230901-225226.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230901-225226.yaml
rename to .changes/1.7.0/Dependencies-20230901-225226.yaml
diff --git a/.changes/unreleased/Dependencies-20230904-011145.yaml b/.changes/1.7.0/Dependencies-20230904-011145.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230904-011145.yaml
rename to .changes/1.7.0/Dependencies-20230904-011145.yaml
diff --git a/.changes/unreleased/Dependencies-20230911-010957.yaml b/.changes/1.7.0/Dependencies-20230911-010957.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20230911-010957.yaml
rename to .changes/1.7.0/Dependencies-20230911-010957.yaml
diff --git a/.changes/unreleased/Features-20230330-165721.yaml b/.changes/1.7.0/Features-20230330-165721.yaml
similarity index 100%
rename from .changes/unreleased/Features-20230330-165721.yaml
rename to .changes/1.7.0/Features-20230330-165721.yaml
diff --git a/.changes/unreleased/Fixes-20230818-214616.yaml b/.changes/1.7.0/Fixes-20230818-214616.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230818-214616.yaml
rename to .changes/1.7.0/Fixes-20230818-214616.yaml
diff --git a/.changes/unreleased/Fixes-20230829-162111.yaml b/.changes/1.7.0/Fixes-20230829-162111.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230829-162111.yaml
rename to .changes/1.7.0/Fixes-20230829-162111.yaml
diff --git a/.changes/unreleased/Fixes-20230907-161347.yaml b/.changes/1.7.0/Fixes-20230907-161347.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230907-161347.yaml
rename to .changes/1.7.0/Fixes-20230907-161347.yaml
diff --git a/.changes/unreleased/Fixes-20230911-005506.yaml b/.changes/1.7.0/Fixes-20230911-005506.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20230911-005506.yaml
rename to .changes/1.7.0/Fixes-20230911-005506.yaml
diff --git a/.changes/unreleased/Under the Hood-20230829-122918.yaml b/.changes/1.7.0/Under the Hood-20230829-122918.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20230829-122918.yaml
rename to .changes/1.7.0/Under the Hood-20230829-122918.yaml
diff --git a/.changes/unreleased/Under the Hood-20230830-160707.yaml b/.changes/1.7.0/Under the Hood-20230830-160707.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20230830-160707.yaml
rename to .changes/1.7.0/Under the Hood-20230830-160707.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 09fd2d4a1..d7450ff07 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,6 +5,36 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.7.0-b2 - September 15, 2023
+
+### Features
+
+- Add support for materialized views ([#dbt-labs/dbt-core#6911](https://github.com/dbt-labs/dbt-bigquery/issues/dbt-labs/dbt-core#6911))
+
+### Fixes
+
+- case insensitive check on partition matching ([#886](https://github.com/dbt-labs/dbt-bigquery/issues/886))
+- Time out queries if user supplies `job_execution_timeout` ([#231](https://github.com/dbt-labs/dbt-bigquery/issues/231))
+- changes expected value types to AnyInteger to take into account changes in core ([#915](https://github.com/dbt-labs/dbt-bigquery/issues/915))
+- Quote column names in struct data types to fix time ingestion partitioning table creation ([#913](https://github.com/dbt-labs/dbt-bigquery/issues/913))
+
+### Under the Hood
+
+- Re-organize adapters.sql into more granular files inside of macros/relations ([#904](https://github.com/dbt-labs/dbt-bigquery/issues/904))
+- Update BigQueryCredentials to support migration off hologram ([#906](https://github.com/dbt-labs/dbt-bigquery/issues/906))
+
+### Dependencies
+
+- Bump mypy from 1.5.0 to 1.5.1 ([#883](https://github.com/dbt-labs/dbt-bigquery/pull/883))
+- Update tox requirement from ~=4.8 to ~=4.9 ([#884](https://github.com/dbt-labs/dbt-bigquery/pull/884))
+- Update tox requirement from ~=4.9 to ~=4.11 ([#907](https://github.com/dbt-labs/dbt-bigquery/pull/907))
+- Update pre-commit requirement from ~=3.3 to ~=3.4 ([#909](https://github.com/dbt-labs/dbt-bigquery/pull/909))
+- Update black requirement from ~=23.7 to ~=23.9 ([#920](https://github.com/dbt-labs/dbt-bigquery/pull/920))
+
+### Contributors
+- [@Kayrnt](https://github.com/Kayrnt) ([#886](https://github.com/dbt-labs/dbt-bigquery/issues/886), [#913](https://github.com/dbt-labs/dbt-bigquery/issues/913))
+
+
 ## dbt-bigquery 1.7.0-b1 - August 17, 2023
 
 ### Features
@@ -42,7 +72,6 @@
 - [@dementiev27](https://github.com/dementiev27) ([#861](https://github.com/dbt-labs/dbt-bigquery/issues/861))
 - [@nickozilla](https://github.com/nickozilla) ([#671](https://github.com/dbt-labs/dbt-bigquery/issues/671))
 
-
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 - [1.6](https://github.com/dbt-labs/dbt-bigquery/blob/1.6.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 48607b01f..3f5d3c0b7 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.7.0b1"
+version = "1.7.0b2"
diff --git a/setup.py b/setup.py
index 947554f6d..494cd286e 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.7.0b1"
+package_version = "1.7.0b2"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From 9f6c449619cf245f1788fcf7e5cf13f3e3a855a6 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Fri, 22 Sep 2023 11:24:14 -0700
Subject: [PATCH 711/860] Address type annotation issues and clean up impl
 (#933)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* update impl ConnectionManager typing and move list_datasets into BigQueryConnectionManager

* refactor unit tests and add one to cover list_datasets

* accidental commit rever

* add changie

* Rework constructor for mypy. Remove unused functions.

* Add changelog entry.

* merge paw/type-fix

---------

Co-authored-by: Peter Allen Webb <peter.webb@dbtlabs.com>
---
 .../Under the Hood-20230921-155645.yaml       |   6 +
 .../Under the Hood-20230922-125327.yaml       |   6 +
 dbt/adapters/bigquery/connections.py          |  14 ++
 dbt/adapters/bigquery/impl.py                 |  51 +----
 tests/unit/test_bigquery_adapter.py           | 182 +---------------
 .../unit/test_bigquery_connection_manager.py  | 198 ++++++++++++++++++
 6 files changed, 230 insertions(+), 227 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20230921-155645.yaml
 create mode 100644 .changes/unreleased/Under the Hood-20230922-125327.yaml
 create mode 100644 tests/unit/test_bigquery_connection_manager.py

diff --git a/.changes/unreleased/Under the Hood-20230921-155645.yaml b/.changes/unreleased/Under the Hood-20230921-155645.yaml
new file mode 100644
index 000000000..12cd663f8
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20230921-155645.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Address type annotation issues and remove protected method ref from impl
+time: 2023-09-21T15:56:45.329798-07:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "933"
diff --git a/.changes/unreleased/Under the Hood-20230922-125327.yaml b/.changes/unreleased/Under the Hood-20230922-125327.yaml
new file mode 100644
index 000000000..9ce871321
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20230922-125327.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Fixed a mypy failure by reworking BigQueryAdapter constructor.
+time: 2023-09-22T12:53:27.339599-04:00
+custom:
+  Author: peterallenwebb
+  Issue: "934"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index c136042c3..7799ecb8a 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -700,6 +700,20 @@ def fn():
 
         self._retry_and_handle(msg="create dataset", conn=conn, fn=fn)
 
+    def list_dataset(self, database: str):
+        # the database string we get here is potentially quoted. Strip that off
+        # for the API call.
+        database = database.strip("`")
+        conn = self.get_thread_connection()
+        client = conn.handle
+
+        def query_schemas():
+            # this is similar to how we have to deal with listing tables
+            all_datasets = client.list_datasets(project=database, max_results=10000)
+            return [ds.dataset_id for ds in all_datasets]
+
+        return self._retry_and_handle(msg="list dataset", conn=conn, fn=query_schemas)
+
     def _query_and_results(
         self,
         client,
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index bb04c78b8..8fc1b69bb 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -217,6 +217,10 @@ class BigQueryAdapter(BaseAdapter):
         ConstraintType.foreign_key: ConstraintSupport.ENFORCED,
     }
 
+    def __init__(self, config) -> None:
+        super().__init__(config)
+        self.connections: BigQueryConnectionManager = self.connections
+
     ###
     # Implementations of abstract methods
     ###
@@ -267,18 +271,7 @@ def rename_relation(
 
     @available
     def list_schemas(self, database: str) -> List[str]:
-        # the database string we get here is potentially quoted. Strip that off
-        # for the API call.
-        database = database.strip("`")
-        conn = self.connections.get_thread_connection()
-        client = conn.handle
-
-        def query_schemas():
-            # this is similar to how we have to deal with listing tables
-            all_datasets = client.list_datasets(project=database, max_results=10000)
-            return [ds.dataset_id for ds in all_datasets]
-
-        return self.connections._retry_and_handle(msg="list dataset", conn=conn, fn=query_schemas)
+        return self.connections.list_dataset(database)
 
     @available.parse(lambda *a, **k: False)
     def check_schema_exists(self, database: str, schema: str) -> bool:
@@ -481,40 +474,6 @@ def _agate_to_schema(
             bq_schema.append(SchemaField(col_name, type_))  # type: ignore[arg-type]
         return bq_schema
 
-    def _materialize_as_view(self, model: Dict[str, Any]) -> str:
-        model_database = model.get("database")
-        model_schema = model.get("schema")
-        model_alias = model.get("alias")
-        model_code = model.get("compiled_code")
-
-        logger.debug("Model SQL ({}):\n{}".format(model_alias, model_code))
-        self.connections.create_view(
-            database=model_database, schema=model_schema, table_name=model_alias, sql=model_code
-        )
-        return "CREATE VIEW"
-
-    def _materialize_as_table(
-        self,
-        model: Dict[str, Any],
-        model_sql: str,
-        decorator: Optional[str] = None,
-    ) -> str:
-        model_database = model.get("database")
-        model_schema = model.get("schema")
-        model_alias = model.get("alias")
-
-        if decorator is None:
-            table_name = model_alias
-        else:
-            table_name = "{}${}".format(model_alias, decorator)
-
-        logger.debug("Model SQL ({}):\n{}".format(table_name, model_sql))
-        self.connections.create_table(
-            database=model_database, schema=model_schema, table_name=table_name, sql=model_sql
-        )
-
-        return "CREATE TABLE"
-
     @available.parse(lambda *a, **k: "")
     def copy_table(self, source, destination, materialization):
         if materialization == "incremental":
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 10cb3f530..4db2ce83d 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -1,26 +1,19 @@
-import time
-
 import agate
 import decimal
-import json
 import string
 import random
 import re
 import pytest
 import unittest
-from contextlib import contextmanager
-from requests.exceptions import ConnectionError
-from unittest.mock import patch, MagicMock, Mock, create_autospec, ANY
+from unittest.mock import patch, MagicMock, create_autospec
 
 import dbt.dataclass_schema
 
 from dbt.adapters.bigquery import PartitionConfig
-from dbt.adapters.bigquery import BigQueryCredentials
 from dbt.adapters.bigquery import BigQueryAdapter
 from dbt.adapters.bigquery import BigQueryRelation
 from dbt.adapters.bigquery import Plugin as BigQueryPlugin
 from google.cloud.bigquery.table import Table
-from dbt.adapters.bigquery.connections import BigQueryConnectionManager
 from dbt.adapters.bigquery.connections import _sanitize_label, _VALIDATE_LABEL_LENGTH_LIMIT
 from dbt.adapters.base.query_headers import MacroQueryStringSetter
 from dbt.clients import agate_helper
@@ -543,179 +536,6 @@ def test_replace(self):
         assert other_schema.quote_policy.database is False
 
 
-class TestBigQueryConnectionManager(unittest.TestCase):
-    def setUp(self):
-        credentials = Mock(BigQueryCredentials)
-        profile = Mock(query_comment=None, credentials=credentials)
-        self.connections = BigQueryConnectionManager(profile=profile)
-
-        self.mock_client = Mock(dbt.adapters.bigquery.impl.google.cloud.bigquery.Client)
-        self.mock_connection = MagicMock()
-
-        self.mock_connection.handle = self.mock_client
-
-        self.connections.get_thread_connection = lambda: self.mock_connection
-        self.connections.get_job_retry_deadline_seconds = lambda x: None
-        self.connections.get_job_retries = lambda x: 1
-
-    @patch("dbt.adapters.bigquery.connections._is_retryable", return_value=True)
-    def test_retry_and_handle(self, is_retryable):
-        self.connections.DEFAULT_MAXIMUM_DELAY = 2.0
-
-        @contextmanager
-        def dummy_handler(msg):
-            yield
-
-        self.connections.exception_handler = dummy_handler
-
-        class DummyException(Exception):
-            """Count how many times this exception is raised"""
-
-            count = 0
-
-            def __init__(self):
-                DummyException.count += 1
-
-        def raiseDummyException():
-            raise DummyException()
-
-        with self.assertRaises(DummyException):
-            self.connections._retry_and_handle(
-                "some sql", Mock(credentials=Mock(retries=8)), raiseDummyException
-            )
-            self.assertEqual(DummyException.count, 9)
-
-    @patch("dbt.adapters.bigquery.connections._is_retryable", return_value=True)
-    def test_retry_connection_reset(self, is_retryable):
-        self.connections.open = MagicMock()
-        self.connections.close = MagicMock()
-        self.connections.DEFAULT_MAXIMUM_DELAY = 2.0
-
-        @contextmanager
-        def dummy_handler(msg):
-            yield
-
-        self.connections.exception_handler = dummy_handler
-
-        def raiseConnectionResetError():
-            raise ConnectionResetError("Connection broke")
-
-        mock_conn = Mock(credentials=Mock(retries=1))
-        with self.assertRaises(ConnectionResetError):
-            self.connections._retry_and_handle("some sql", mock_conn, raiseConnectionResetError)
-        self.connections.close.assert_called_once_with(mock_conn)
-        self.connections.open.assert_called_once_with(mock_conn)
-
-    def test_is_retryable(self):
-        _is_retryable = dbt.adapters.bigquery.connections._is_retryable
-        exceptions = dbt.adapters.bigquery.impl.google.cloud.exceptions
-        internal_server_error = exceptions.InternalServerError("code broke")
-        bad_request_error = exceptions.BadRequest("code broke")
-        connection_error = ConnectionError("code broke")
-        client_error = exceptions.ClientError("bad code")
-        rate_limit_error = exceptions.Forbidden(
-            "code broke", errors=[{"reason": "rateLimitExceeded"}]
-        )
-
-        self.assertTrue(_is_retryable(internal_server_error))
-        self.assertTrue(_is_retryable(bad_request_error))
-        self.assertTrue(_is_retryable(connection_error))
-        self.assertFalse(_is_retryable(client_error))
-        self.assertTrue(_is_retryable(rate_limit_error))
-
-    def test_drop_dataset(self):
-        mock_table = Mock()
-        mock_table.reference = "table1"
-        self.mock_client.list_tables.return_value = [mock_table]
-
-        self.connections.drop_dataset("project", "dataset")
-
-        self.mock_client.list_tables.assert_not_called()
-        self.mock_client.delete_table.assert_not_called()
-        self.mock_client.delete_dataset.assert_called_once()
-
-    @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
-    def test_query_and_results(self, mock_bq):
-        self.mock_client.query = Mock(return_value=Mock(state="DONE"))
-        self.connections._query_and_results(
-            self.mock_client,
-            "sql",
-            {"job_param_1": "blah"},
-            job_creation_timeout=15,
-            job_execution_timeout=3,
-        )
-
-        mock_bq.QueryJobConfig.assert_called_once()
-        self.mock_client.query.assert_called_once_with(
-            query="sql", job_config=mock_bq.QueryJobConfig(), timeout=15
-        )
-
-    @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
-    def test_query_and_results_timeout(self, mock_bq):
-        self.mock_client.query = Mock(
-            return_value=Mock(result=lambda *args, **kwargs: time.sleep(4))
-        )
-        with pytest.raises(dbt.exceptions.DbtRuntimeError) as exc:
-            self.connections._query_and_results(
-                self.mock_client,
-                "sql",
-                {"job_param_1": "blah"},
-                job_creation_timeout=15,
-                job_execution_timeout=1,
-            )
-
-        mock_bq.QueryJobConfig.assert_called_once()
-        self.mock_client.query.assert_called_once_with(
-            query="sql", job_config=mock_bq.QueryJobConfig(), timeout=15
-        )
-        assert "Query exceeded configured timeout of 1s" in str(exc.value)
-
-    def test_copy_bq_table_appends(self):
-        self._copy_table(write_disposition=dbt.adapters.bigquery.impl.WRITE_APPEND)
-        args, kwargs = self.mock_client.copy_table.call_args
-        self.mock_client.copy_table.assert_called_once_with(
-            [self._table_ref("project", "dataset", "table1")],
-            self._table_ref("project", "dataset", "table2"),
-            job_config=ANY,
-        )
-        args, kwargs = self.mock_client.copy_table.call_args
-        self.assertEqual(
-            kwargs["job_config"].write_disposition, dbt.adapters.bigquery.impl.WRITE_APPEND
-        )
-
-    def test_copy_bq_table_truncates(self):
-        self._copy_table(write_disposition=dbt.adapters.bigquery.impl.WRITE_TRUNCATE)
-        args, kwargs = self.mock_client.copy_table.call_args
-        self.mock_client.copy_table.assert_called_once_with(
-            [self._table_ref("project", "dataset", "table1")],
-            self._table_ref("project", "dataset", "table2"),
-            job_config=ANY,
-        )
-        args, kwargs = self.mock_client.copy_table.call_args
-        self.assertEqual(
-            kwargs["job_config"].write_disposition, dbt.adapters.bigquery.impl.WRITE_TRUNCATE
-        )
-
-    def test_job_labels_valid_json(self):
-        expected = {"key": "value"}
-        labels = self.connections._labels_from_query_comment(json.dumps(expected))
-        self.assertEqual(labels, expected)
-
-    def test_job_labels_invalid_json(self):
-        labels = self.connections._labels_from_query_comment("not json")
-        self.assertEqual(labels, {"query_comment": "not_json"})
-
-    def _table_ref(self, proj, ds, table):
-        return self.connections.table_ref(proj, ds, table)
-
-    def _copy_table(self, write_disposition):
-        source = BigQueryRelation.create(database="project", schema="dataset", identifier="table1")
-        destination = BigQueryRelation.create(
-            database="project", schema="dataset", identifier="table2"
-        )
-        self.connections.copy_bq_table(source, destination, write_disposition)
-
-
 class TestBigQueryAdapter(BaseTestBigQueryAdapter):
     def test_copy_table_materialization_table(self):
         adapter = self.get_adapter("oauth")
diff --git a/tests/unit/test_bigquery_connection_manager.py b/tests/unit/test_bigquery_connection_manager.py
new file mode 100644
index 000000000..d6c3f64fc
--- /dev/null
+++ b/tests/unit/test_bigquery_connection_manager.py
@@ -0,0 +1,198 @@
+import time
+import json
+import pytest
+import unittest
+from contextlib import contextmanager
+from requests.exceptions import ConnectionError
+from unittest.mock import patch, MagicMock, Mock, ANY
+
+import dbt.dataclass_schema
+
+from dbt.adapters.bigquery import BigQueryCredentials
+from dbt.adapters.bigquery import BigQueryRelation
+from dbt.adapters.bigquery.connections import BigQueryConnectionManager
+import dbt.exceptions
+from dbt.logger import GLOBAL_LOGGER as logger  # noqa
+
+
+class TestBigQueryConnectionManager(unittest.TestCase):
+    def setUp(self):
+        credentials = Mock(BigQueryCredentials)
+        profile = Mock(query_comment=None, credentials=credentials)
+        self.connections = BigQueryConnectionManager(profile=profile)
+
+        self.mock_client = Mock(dbt.adapters.bigquery.impl.google.cloud.bigquery.Client)
+        self.mock_connection = MagicMock()
+
+        self.mock_connection.handle = self.mock_client
+
+        self.connections.get_thread_connection = lambda: self.mock_connection
+        self.connections.get_job_retry_deadline_seconds = lambda x: None
+        self.connections.get_job_retries = lambda x: 1
+
+    @patch("dbt.adapters.bigquery.connections._is_retryable", return_value=True)
+    def test_retry_and_handle(self, is_retryable):
+        self.connections.DEFAULT_MAXIMUM_DELAY = 2.0
+
+        @contextmanager
+        def dummy_handler(msg):
+            yield
+
+        self.connections.exception_handler = dummy_handler
+
+        class DummyException(Exception):
+            """Count how many times this exception is raised"""
+
+            count = 0
+
+            def __init__(self):
+                DummyException.count += 1
+
+        def raiseDummyException():
+            raise DummyException()
+
+        with self.assertRaises(DummyException):
+            self.connections._retry_and_handle(
+                "some sql", Mock(credentials=Mock(retries=8)), raiseDummyException
+            )
+            self.assertEqual(DummyException.count, 9)
+
+    @patch("dbt.adapters.bigquery.connections._is_retryable", return_value=True)
+    def test_retry_connection_reset(self, is_retryable):
+        self.connections.open = MagicMock()
+        self.connections.close = MagicMock()
+        self.connections.DEFAULT_MAXIMUM_DELAY = 2.0
+
+        @contextmanager
+        def dummy_handler(msg):
+            yield
+
+        self.connections.exception_handler = dummy_handler
+
+        def raiseConnectionResetError():
+            raise ConnectionResetError("Connection broke")
+
+        mock_conn = Mock(credentials=Mock(retries=1))
+        with self.assertRaises(ConnectionResetError):
+            self.connections._retry_and_handle("some sql", mock_conn, raiseConnectionResetError)
+        self.connections.close.assert_called_once_with(mock_conn)
+        self.connections.open.assert_called_once_with(mock_conn)
+
+    def test_is_retryable(self):
+        _is_retryable = dbt.adapters.bigquery.connections._is_retryable
+        exceptions = dbt.adapters.bigquery.impl.google.cloud.exceptions
+        internal_server_error = exceptions.InternalServerError("code broke")
+        bad_request_error = exceptions.BadRequest("code broke")
+        connection_error = ConnectionError("code broke")
+        client_error = exceptions.ClientError("bad code")
+        rate_limit_error = exceptions.Forbidden(
+            "code broke", errors=[{"reason": "rateLimitExceeded"}]
+        )
+
+        self.assertTrue(_is_retryable(internal_server_error))
+        self.assertTrue(_is_retryable(bad_request_error))
+        self.assertTrue(_is_retryable(connection_error))
+        self.assertFalse(_is_retryable(client_error))
+        self.assertTrue(_is_retryable(rate_limit_error))
+
+    def test_drop_dataset(self):
+        mock_table = Mock()
+        mock_table.reference = "table1"
+        self.mock_client.list_tables.return_value = [mock_table]
+
+        self.connections.drop_dataset("project", "dataset")
+
+        self.mock_client.list_tables.assert_not_called()
+        self.mock_client.delete_table.assert_not_called()
+        self.mock_client.delete_dataset.assert_called_once()
+
+    @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
+    def test_query_and_results(self, mock_bq):
+        self.mock_client.query = Mock(return_value=Mock(state="DONE"))
+        self.connections._query_and_results(
+            self.mock_client,
+            "sql",
+            {"job_param_1": "blah"},
+            job_creation_timeout=15,
+            job_execution_timeout=3,
+        )
+
+        mock_bq.QueryJobConfig.assert_called_once()
+        self.mock_client.query.assert_called_once_with(
+            query="sql", job_config=mock_bq.QueryJobConfig(), timeout=15
+        )
+
+    @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
+    def test_query_and_results_timeout(self, mock_bq):
+        self.mock_client.query = Mock(
+            return_value=Mock(result=lambda *args, **kwargs: time.sleep(4))
+        )
+        with pytest.raises(dbt.exceptions.DbtRuntimeError) as exc:
+            self.connections._query_and_results(
+                self.mock_client,
+                "sql",
+                {"job_param_1": "blah"},
+                job_creation_timeout=15,
+                job_execution_timeout=1,
+            )
+
+        mock_bq.QueryJobConfig.assert_called_once()
+        self.mock_client.query.assert_called_once_with(
+            query="sql", job_config=mock_bq.QueryJobConfig(), timeout=15
+        )
+        assert "Query exceeded configured timeout of 1s" in str(exc.value)
+
+    def test_copy_bq_table_appends(self):
+        self._copy_table(write_disposition=dbt.adapters.bigquery.impl.WRITE_APPEND)
+        args, kwargs = self.mock_client.copy_table.call_args
+        self.mock_client.copy_table.assert_called_once_with(
+            [self._table_ref("project", "dataset", "table1")],
+            self._table_ref("project", "dataset", "table2"),
+            job_config=ANY,
+        )
+        args, kwargs = self.mock_client.copy_table.call_args
+        self.assertEqual(
+            kwargs["job_config"].write_disposition, dbt.adapters.bigquery.impl.WRITE_APPEND
+        )
+
+    def test_copy_bq_table_truncates(self):
+        self._copy_table(write_disposition=dbt.adapters.bigquery.impl.WRITE_TRUNCATE)
+        args, kwargs = self.mock_client.copy_table.call_args
+        self.mock_client.copy_table.assert_called_once_with(
+            [self._table_ref("project", "dataset", "table1")],
+            self._table_ref("project", "dataset", "table2"),
+            job_config=ANY,
+        )
+        args, kwargs = self.mock_client.copy_table.call_args
+        self.assertEqual(
+            kwargs["job_config"].write_disposition, dbt.adapters.bigquery.impl.WRITE_TRUNCATE
+        )
+
+    def test_job_labels_valid_json(self):
+        expected = {"key": "value"}
+        labels = self.connections._labels_from_query_comment(json.dumps(expected))
+        self.assertEqual(labels, expected)
+
+    def test_job_labels_invalid_json(self):
+        labels = self.connections._labels_from_query_comment("not json")
+        self.assertEqual(labels, {"query_comment": "not_json"})
+
+    def test_list_dataset_correctly_calls_lists_datasets(self):
+        mock_dataset = Mock(dataset_id="d1")
+        mock_list_dataset = Mock(return_value=[mock_dataset])
+        self.mock_client.list_datasets = mock_list_dataset
+        result = self.connections.list_dataset("project")
+        self.mock_client.list_datasets.assert_called_once_with(
+            project="project", max_results=10000
+        )
+        assert result == ["d1"]
+
+    def _table_ref(self, proj, ds, table):
+        return self.connections.table_ref(proj, ds, table)
+
+    def _copy_table(self, write_disposition):
+        source = BigQueryRelation.create(database="project", schema="dataset", identifier="table1")
+        destination = BigQueryRelation.create(
+            database="project", schema="dataset", identifier="table2"
+        )
+        self.connections.copy_bq_table(source, destination, write_disposition)

From 4c02c07b95f8c1c582320558384a349e1d4fd51a Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Fri, 22 Sep 2023 14:03:18 -0700
Subject: [PATCH 712/860] Update SQLQuery to include node_info (#936)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* update SQLQuery to include node_info

* add changie

* revert setup
---
 .changes/unreleased/Under the Hood-20230922-114217.yaml | 6 ++++++
 dbt/adapters/bigquery/connections.py                    | 4 +++-
 2 files changed, 9 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Under the Hood-20230922-114217.yaml

diff --git a/.changes/unreleased/Under the Hood-20230922-114217.yaml b/.changes/unreleased/Under the Hood-20230922-114217.yaml
new file mode 100644
index 000000000..78fee33c4
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20230922-114217.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: update SQLQuery to include node_info
+time: 2023-09-22T11:42:17.770033-07:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "936"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 7799ecb8a..a5c7b9355 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -5,6 +5,7 @@
 from contextlib import contextmanager
 from dataclasses import dataclass, field
 
+from dbt.events.contextvars import get_node_info
 from mashumaro.helper import pass_through
 
 from functools import lru_cache
@@ -444,7 +445,8 @@ def raw_execute(
         conn = self.get_thread_connection()
         client = conn.handle
 
-        fire_event(SQLQuery(conn_name=conn.name, sql=sql))
+        fire_event(SQLQuery(conn_name=conn.name, sql=sql, node_info=get_node_info()))
+
         if (
             hasattr(self.profile, "query_comment")
             and self.profile.query_comment

From 63ae274c8723025790b7b256da4655fbfe7c2ed6 Mon Sep 17 00:00:00 2001
From: Emily Rockman <emily.rockman@dbtlabs.com>
Date: Tue, 26 Sep 2023 11:49:18 -0500
Subject: [PATCH 713/860] automate repo cleanup (#944)

---
 .github/workflows/repository-cleanup.yml | 30 ++++++++++++++++++++++++
 1 file changed, 30 insertions(+)
 create mode 100644 .github/workflows/repository-cleanup.yml

diff --git a/.github/workflows/repository-cleanup.yml b/.github/workflows/repository-cleanup.yml
new file mode 100644
index 000000000..c1d780281
--- /dev/null
+++ b/.github/workflows/repository-cleanup.yml
@@ -0,0 +1,30 @@
+# **what?**
+# Cleanup branches left over from automation and testing.  Also cleanup
+# draft releases from release testing.
+
+# **why?**
+# The automations are leaving behind branches and releases that clutter
+# the repository.  Sometimes we need them to debug processes so we don't
+# want them immediately deleted.  Running on Saturday to avoid running
+# at the same time as an actual release to prevent breaking a release
+# mid-release.
+
+# **when?**
+# Mainly on a schedule of 12:00 Saturday.
+# Manual trigger can also run on demand
+
+name: Repository Cleanup
+
+on:
+  schedule:
+    - cron: '0 12 * * SAT' # At 12:00 on Saturday - details in `why` above
+
+  workflow_dispatch: # for manual triggering
+
+permissions:
+  contents: write
+
+jobs:
+  cleanup-repo:
+    uses: dbt-labs/actions/.github/workflows/repository-cleanup.yml@main
+    secrets: inherit

From e5a89afdf7820905e44e850556e2901b7b872303 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Tue, 26 Sep 2023 15:48:57 -0700
Subject: [PATCH 714/860] poll `.GetBatch()` instead of using
 `operation.result()` (#929)

* re:PR https://github.com/dbt-labs/dbt-bigquery/pull/840/files

* adding back comment # check if job failed

* adding changelog

* precommit code format

* sleep(2) first in the while loop before the request to eliminate the last 2 seconds sleep if the response is in one of the 3 options

* removing empty spaces

* update batch request to handle `GetBatchRequest`

* conditionally run python model tests and factor out batch functions to own module

* Move events to common

* fix import

* fix mistaken import change

* update unit test

* clean up and typing

---------

Co-authored-by: Zi Wang <wazi@google.com>
Co-authored-by: wazi55 <134335723+wazi55@users.noreply.github.com>
Co-authored-by: Anders <anders.swanson@dbtlabs.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .../unreleased/Fixes-20230721-101041.yaml     |  6 ++
 .github/workflows/integration.yml             | 21 ++++++
 dbt/adapters/bigquery/dataproc/__init__.py    |  0
 dbt/adapters/bigquery/dataproc/batch.py       | 67 +++++++++++++++++++
 dbt/adapters/bigquery/python_submissions.py   | 61 +++++++----------
 tests/functional/adapter/test_python_model.py |  2 +-
 tests/unit/test_configure_dataproc_batch.py   |  4 +-
 7 files changed, 123 insertions(+), 38 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20230721-101041.yaml
 create mode 100644 dbt/adapters/bigquery/dataproc/__init__.py
 create mode 100644 dbt/adapters/bigquery/dataproc/batch.py

diff --git a/.changes/unreleased/Fixes-20230721-101041.yaml b/.changes/unreleased/Fixes-20230721-101041.yaml
new file mode 100644
index 000000000..6db81cf50
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230721-101041.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Serverless Spark to Poll with .GetBatch() instead of using operation.result()
+time: 2023-07-21T10:10:41.64843-07:00
+custom:
+  Author: wazi55
+  Issue: "734"
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 99f78e33d..bb0211b35 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -64,6 +64,7 @@ jobs:
 
     outputs:
       matrix: ${{ steps.generate-matrix.outputs.result }}
+      run-python-tests: ${{ steps.filter.outputs.bigquery-python }}
 
     steps:
       - name: Check out the repository (non-PR)
@@ -96,6 +97,11 @@ jobs:
               - 'dbt/**'
               - 'tests/**'
               - 'dev-requirements.txt'
+            bigquery-python:
+              - 'dbt/adapters/bigquery/dataproc/**'
+              - 'dbt/adapters/bigquery/python_submissions.py'
+              - 'dbt/include/bigquery/python_model/**'
+
       - name: Generate integration test matrix
         id: generate-matrix
         uses: actions/github-script@v6
@@ -186,6 +192,21 @@ jobs:
           GCS_BUCKET: dbt-ci
         run: tox -- --ddtrace
 
+      # python models tests are slow so we only want to run them if we're changing them
+      - name: Run tox (python models)
+        if: needs.test-metadata.outputs.run-python-tests == 'true'
+        env:
+          BIGQUERY_TEST_SERVICE_ACCOUNT_JSON: ${{ secrets.BIGQUERY_TEST_SERVICE_ACCOUNT_JSON }}
+          BIGQUERY_TEST_ALT_DATABASE: ${{ secrets.BIGQUERY_TEST_ALT_DATABASE }}
+          BIGQUERY_TEST_NO_ACCESS_DATABASE: ${{ secrets.BIGQUERY_TEST_NO_ACCESS_DATABASE }}
+          DBT_TEST_USER_1: group:buildbot@dbtlabs.com
+          DBT_TEST_USER_2: group:engineering-core-team@dbtlabs.com
+          DBT_TEST_USER_3: serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserviceaccount.com
+          DATAPROC_REGION: us-central1
+          DATAPROC_CLUSTER_NAME: dbt-test-1
+          GCS_BUCKET: dbt-ci
+        run: tox -e python-tests -- --ddtrace
+
       - uses: actions/upload-artifact@v3
         if: always()
         with:
diff --git a/dbt/adapters/bigquery/dataproc/__init__.py b/dbt/adapters/bigquery/dataproc/__init__.py
new file mode 100644
index 000000000..e69de29bb
diff --git a/dbt/adapters/bigquery/dataproc/batch.py b/dbt/adapters/bigquery/dataproc/batch.py
new file mode 100644
index 000000000..0dc54aa78
--- /dev/null
+++ b/dbt/adapters/bigquery/dataproc/batch.py
@@ -0,0 +1,67 @@
+from typing import Union, Dict
+
+import time
+from datetime import datetime
+from google.cloud.dataproc_v1 import (
+    CreateBatchRequest,
+    BatchControllerClient,
+    Batch,
+    GetBatchRequest,
+)
+from google.protobuf.json_format import ParseDict
+
+from dbt.adapters.bigquery.connections import DataprocBatchConfig
+
+_BATCH_RUNNING_STATES = [Batch.State.PENDING, Batch.State.RUNNING]
+DEFAULT_JAR_FILE_URI = "gs://spark-lib/bigquery/spark-bigquery-with-dependencies_2.12-0.21.1.jar"
+
+
+def create_batch_request(
+    batch: Batch, batch_id: str, project: str, region: str
+) -> CreateBatchRequest:
+    return CreateBatchRequest(
+        parent=f"projects/{project}/locations/{region}",  # type: ignore
+        batch_id=batch_id,  # type: ignore
+        batch=batch,  # type: ignore
+    )
+
+
+def poll_batch_job(
+    parent: str, batch_id: str, job_client: BatchControllerClient, timeout: int
+) -> Batch:
+    batch_name = "".join([parent, "/batches/", batch_id])
+    state = Batch.State.PENDING
+    response = None
+    run_time = 0
+    while state in _BATCH_RUNNING_STATES and run_time < timeout:
+        time.sleep(1)
+        response = job_client.get_batch(  # type: ignore
+            request=GetBatchRequest(name=batch_name),  # type: ignore
+        )
+        run_time = datetime.now().timestamp() - response.create_time.timestamp()  # type: ignore
+        state = response.state
+    if not response:
+        raise ValueError("No response from Dataproc")
+    if state != Batch.State.SUCCEEDED:
+        if run_time >= timeout:
+            raise ValueError(
+                f"Operation did not complete within the designated timeout of {timeout} seconds."
+            )
+        else:
+            raise ValueError(response.state_message)
+    return response
+
+
+def update_batch_from_config(config_dict: Union[Dict, DataprocBatchConfig], target: Batch):
+    try:
+        # updates in place
+        ParseDict(config_dict, target._pb)
+    except Exception as e:
+        docurl = (
+            "https://cloud.google.com/dataproc-serverless/docs/reference/rpc/google.cloud.dataproc.v1"
+            "#google.cloud.dataproc.v1.Batch"
+        )
+        raise ValueError(
+            f"Unable to parse dataproc_batch as valid batch specification. See {docurl}. {str(e)}"
+        ) from e
+    return target
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 6e5a11e52..8fd354eb5 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -4,11 +4,17 @@
 from google.api_core.future.polling import POLLING_PREDICATE
 
 from dbt.adapters.bigquery import BigQueryConnectionManager, BigQueryCredentials
-from dbt.adapters.bigquery.connections import DataprocBatchConfig
 from google.api_core import retry
 from google.api_core.client_options import ClientOptions
 from google.cloud import storage, dataproc_v1  # type: ignore
-from google.protobuf.json_format import ParseDict
+from google.cloud.dataproc_v1.types.batches import Batch
+
+from dbt.adapters.bigquery.dataproc.batch import (
+    create_batch_request,
+    poll_batch_job,
+    DEFAULT_JAR_FILE_URI,
+    update_batch_from_config,
+)
 
 OPERATION_RETRY_TIME = 10
 
@@ -102,8 +108,8 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
                 "job": job,
             }
         )
-        response = operation.result(polling=self.result_polling_policy)
         # check if job failed
+        response = operation.result(polling=self.result_polling_policy)
         if response.status.state == 6:
             raise ValueError(response.status.details)
         return response
@@ -118,21 +124,22 @@ def _get_job_client(self) -> dataproc_v1.BatchControllerClient:
     def _get_batch_id(self) -> str:
         return self.parsed_model["config"].get("batch_id")
 
-    def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
-        batch = self._configure_batch()
-        parent = f"projects/{self.credential.execution_project}/locations/{self.credential.dataproc_region}"
-
-        request = dataproc_v1.CreateBatchRequest(
-            parent=parent,
-            batch=batch,
-            batch_id=self._get_batch_id(),
-        )
+    def _submit_dataproc_job(self) -> Batch:
+        batch_id = self._get_batch_id()
+        request = create_batch_request(
+            batch=self._configure_batch(),
+            batch_id=batch_id,
+            region=self.credential.dataproc_region,  # type: ignore
+            project=self.credential.execution_project,  # type: ignore
+        )  # type: ignore
         # make the request
-        operation = self.job_client.create_batch(request=request)  # type: ignore
-        # this takes quite a while, waiting on GCP response to resolve
-        # (not a google-api-core issue, more likely a dataproc serverless issue)
-        response = operation.result(polling=self.result_polling_policy)
-        return response
+        self.job_client.create_batch(request=request)  # type: ignore
+        return poll_batch_job(
+            parent=request.parent,
+            batch_id=batch_id,
+            job_client=self.job_client,  # type: ignore
+            timeout=self.timeout,
+        )
         # there might be useful results here that we can parse and return
         # Dataproc job output is saved to the Cloud Storage bucket
         # allocated to the job. Use regex to obtain the bucket and blob info.
@@ -163,27 +170,11 @@ def _configure_batch(self):
         batch.pyspark_batch.main_python_file_uri = self.gcs_location
         jar_file_uri = self.parsed_model["config"].get(
             "jar_file_uri",
-            "gs://spark-lib/bigquery/spark-bigquery-with-dependencies_2.12-0.21.1.jar",
+            DEFAULT_JAR_FILE_URI,
         )
         batch.pyspark_batch.jar_file_uris = [jar_file_uri]
 
         # Apply configuration from dataproc_batch key, possibly overriding defaults.
         if self.credential.dataproc_batch:
-            self._update_batch_from_config(self.credential.dataproc_batch, batch)
+            batch = update_batch_from_config(self.credential.dataproc_batch, batch)
         return batch
-
-    @classmethod
-    def _update_batch_from_config(
-        cls, config_dict: Union[Dict, DataprocBatchConfig], target: dataproc_v1.Batch
-    ):
-        try:
-            # updates in place
-            ParseDict(config_dict, target._pb)
-        except Exception as e:
-            docurl = (
-                "https://cloud.google.com/dataproc-serverless/docs/reference/rpc/google.cloud.dataproc.v1"
-                "#google.cloud.dataproc.v1.Batch"
-            )
-            raise ValueError(
-                f"Unable to parse dataproc_batch as valid batch specification. See {docurl}. {str(e)}"
-            ) from e
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
index 241082cdb..b389fe8aa 100644
--- a/tests/functional/adapter/test_python_model.py
+++ b/tests/functional/adapter/test_python_model.py
@@ -66,7 +66,7 @@ def model(dbt, spark):
 """
 
 models__python_array_batch_id_python = """
-import pandas
+import pandas as pd
 
 def model(dbt, spark):
     random_array = [
diff --git a/tests/unit/test_configure_dataproc_batch.py b/tests/unit/test_configure_dataproc_batch.py
index 58ff52bab..94cb28efb 100644
--- a/tests/unit/test_configure_dataproc_batch.py
+++ b/tests/unit/test_configure_dataproc_batch.py
@@ -1,6 +1,6 @@
 from unittest.mock import patch
 
-from dbt.adapters.bigquery.python_submissions import ServerlessDataProcHelper
+from dbt.adapters.bigquery.dataproc.batch import update_batch_from_config
 from google.cloud import dataproc_v1
 
 from .test_bigquery_adapter import BaseTestBigQueryAdapter
@@ -39,7 +39,7 @@ def test_update_dataproc_serverless_batch(self, mock_get_bigquery_defaults):
 
         batch = dataproc_v1.Batch()
 
-        ServerlessDataProcHelper._update_batch_from_config(raw_batch_config, batch)
+        batch = update_batch_from_config(raw_batch_config, batch)
 
         def to_str_values(d):
             """google's protobuf types expose maps as dict[str, str]"""

From 0fe8afc105ed196bcd201c0c487f03e2f93aba8b Mon Sep 17 00:00:00 2001
From: Michelle Ark <MichelleArk@users.noreply.github.com>
Date: Thu, 28 Sep 2023 19:32:25 +0100
Subject: [PATCH 715/860] Add dbt show tests (#927)

* add dbt show tests

* changelog entry

* repoint to core main

* reuse core fixture for dbt show sql header test

---------

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Co-authored-by: Matthew McKnight <matthew.mcknight@dbtlabs.com>
---
 .changes/unreleased/Under the Hood-20230925-143628.yaml | 6 ++++++
 dev-requirements.txt                                    | 4 ++--
 tests/functional/adapter/dbt_show/test_dbt_show.py      | 9 +++++++++
 3 files changed, 17 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20230925-143628.yaml
 create mode 100644 tests/functional/adapter/dbt_show/test_dbt_show.py

diff --git a/.changes/unreleased/Under the Hood-20230925-143628.yaml b/.changes/unreleased/Under the Hood-20230925-143628.yaml
new file mode 100644
index 000000000..fb925ae40
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20230925-143628.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Add tests for inlined limit + sql-header in dbt show query
+time: 2023-09-25T14:36:28.335466+01:00
+custom:
+  Author: michelleark
+  Issue: "940"
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 54ca40f26..fa5c37b49 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -1,7 +1,7 @@
 # install latest changes in dbt-core
 # TODO: how to automate switching from develop to version branches?
-git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
-git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
+git+https://github.com/dbt-labs/dbt-core.git@improve-show-fixture#egg=dbt-core&subdirectory=core
+git+https://github.com/dbt-labs/dbt-core.git@improve-show-fixture#egg=dbt-tests-adapter&subdirectory=tests/adapter
 
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor
diff --git a/tests/functional/adapter/dbt_show/test_dbt_show.py b/tests/functional/adapter/dbt_show/test_dbt_show.py
new file mode 100644
index 000000000..c60a26aec
--- /dev/null
+++ b/tests/functional/adapter/dbt_show/test_dbt_show.py
@@ -0,0 +1,9 @@
+from dbt.tests.adapter.dbt_show.test_dbt_show import BaseShowSqlHeader, BaseShowLimit
+
+
+class TestBigQueryShowLimit(BaseShowLimit):
+    pass
+
+
+class TestBigQueryShowSqlHeader(BaseShowSqlHeader):
+    pass

From d683dec3c3d725057526199bbcdaedbcc892248b Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 28 Sep 2023 11:47:52 -0700
Subject: [PATCH 716/860] fix mistaken dev-requirements change (#945)

* add dbt show tests

* changelog entry

* repoint to core main

* reuse core fixture for dbt show sql header test

* fix dev-requirements.txt

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* fix dev-requirements.txt

---------

Co-authored-by: Michelle Ark <michelle.ark@dbtlabs.com>
Co-authored-by: Michelle Ark <MichelleArk@users.noreply.github.com>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Co-authored-by: Matthew McKnight <matthew.mcknight@dbtlabs.com>
---
 dev-requirements.txt | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index fa5c37b49..54ca40f26 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -1,7 +1,7 @@
 # install latest changes in dbt-core
 # TODO: how to automate switching from develop to version branches?
-git+https://github.com/dbt-labs/dbt-core.git@improve-show-fixture#egg=dbt-core&subdirectory=core
-git+https://github.com/dbt-labs/dbt-core.git@improve-show-fixture#egg=dbt-tests-adapter&subdirectory=tests/adapter
+git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
+git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
 
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor

From 3dc96c02f292c19ccd2bc306803d4b2f2812ec15 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 2 Oct 2023 11:38:06 -0500
Subject: [PATCH 717/860] Update ddtrace requirement from ~=1.18 to ~=1.19
 (#928)

* Update ddtrace requirement from ~=1.18 to ~=1.19

Updates the requirements on [ddtrace](https://github.com/DataDog/dd-trace-py) to permit the latest version.
- [Release notes](https://github.com/DataDog/dd-trace-py/releases)
- [Changelog](https://github.com/DataDog/dd-trace-py/blob/2.x/CHANGELOG.md)
- [Commits](https://github.com/DataDog/dd-trace-py/compare/v1.18.0...v1.19.0)

---
updated-dependencies:
- dependency-name: ddtrace
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20230919-003620.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20230919-003620.yaml

diff --git a/.changes/unreleased/Dependencies-20230919-003620.yaml b/.changes/unreleased/Dependencies-20230919-003620.yaml
new file mode 100644
index 000000000..31720679b
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20230919-003620.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update ddtrace requirement from ~=1.18 to ~=1.19"
+time: 2023-09-19T00:36:20.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 928
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 54ca40f26..496aa5538 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -8,7 +8,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 black~=23.9
 bumpversion~=0.6.0
 click~=8.1
-ddtrace~=1.18
+ddtrace~=1.19
 flake8~=6.1
 flaky~=3.7
 freezegun~=1.2

From d5b4114ca1ef28d3c2bd3c61812e3a2c901a2366 Mon Sep 17 00:00:00 2001
From: kodaho <23268819+kodaho@users.noreply.github.com>
Date: Fri, 6 Oct 2023 20:26:53 +0200
Subject: [PATCH 718/860] [fix] Use rendered query comment for job labels
 (#955)

* [fix] Use rendered query comment for job labels

* Add test from dbt-labs/dbt-query#872

* Valid JSON in test

* Add changie entry
---
 .../unreleased/Fixes-20231005-235950.yaml     |  6 +++
 dbt/adapters/bigquery/connections.py          |  5 +-
 .../query_comment_test/test_job_label.py      | 52 +++++++++++++++++++
 3 files changed, 61 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20231005-235950.yaml
 create mode 100644 tests/functional/adapter/query_comment_test/test_job_label.py

diff --git a/.changes/unreleased/Fixes-20231005-235950.yaml b/.changes/unreleased/Fixes-20231005-235950.yaml
new file mode 100644
index 000000000..bf0bf6fa6
--- /dev/null
+++ b/.changes/unreleased/Fixes-20231005-235950.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fix issue where job labels are not rendered when using macro for query comment
+time: 2023-10-05T23:59:50.077842+02:00
+custom:
+  Author: kodaho mikealfare
+  Issue: "863"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index a5c7b9355..1e96ed5ef 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -451,9 +451,10 @@ def raw_execute(
             hasattr(self.profile, "query_comment")
             and self.profile.query_comment
             and self.profile.query_comment.job_label
+            and self.query_header
+            and (query_comment := self.query_header.comment.query_comment)
         ):
-            query_comment = self.profile.query_comment
-            labels = self._labels_from_query_comment(query_comment.comment)
+            labels = self._labels_from_query_comment(query_comment)
         else:
             labels = {}
 
diff --git a/tests/functional/adapter/query_comment_test/test_job_label.py b/tests/functional/adapter/query_comment_test/test_job_label.py
new file mode 100644
index 000000000..af984a8c4
--- /dev/null
+++ b/tests/functional/adapter/query_comment_test/test_job_label.py
@@ -0,0 +1,52 @@
+import pytest
+
+from google.cloud.bigquery.client import Client
+
+from dbt.tests.util import run_dbt
+
+
+_MACRO__BQ_LABELS = """
+{% macro bq_labels() %}{
+    "system": "{{ env_var('LABEL_SYSTEM', 'my_system') }}",
+    "env_type": "{{ env_var('LABEL_ENV', 'dev') }}"
+}{% endmacro %}
+"""
+_MODEL__MY_TABLE = """
+{{ config(materialized= "table") }}
+select 1 as id
+"""
+
+
+class TestQueryCommentJobLabel:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {"my_table.sql": _MODEL__MY_TABLE}
+
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {"bq_labels.sql": _MACRO__BQ_LABELS}
+
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            "query-comment": {
+                "comment": "{{ bq_labels() }}",
+                "job-label": True,
+                "append": True,
+            }
+        }
+
+    def test_query_comments_displays_as_job_labels(self, project):
+        """
+        Addresses this regression in dbt-bigquery 1.6:
+        https://github.com/dbt-labs/dbt-bigquery/issues/863
+        """
+        results = run_dbt(["run"])
+        job_id = results.results[0].adapter_response.get("job_id")
+        with project.adapter.connection_named("_test"):
+            client: Client = project.adapter.connections.get_thread_connection().handle
+            job = client.get_job(job_id=job_id)
+
+        # this is what should happen
+        assert job.labels.get("system") == "my_system"
+        assert job.labels.get("env_type") == "dev"

From 3e8d389e65a67bad0ae467761c4a4519507e367d Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Fri, 6 Oct 2023 16:08:24 -0500
Subject: [PATCH 719/860] a slight rework of 955 to keep a seperation of
 concerns (#957)

* a slight rework of 955 to keep a seperation of concerns

* move query_header back into inital if, save assignement for truthy, else return empty dict
---
 dbt/adapters/bigquery/connections.py | 23 +++++++++++++----------
 1 file changed, 13 insertions(+), 10 deletions(-)

diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 1e96ed5ef..ff544f0d0 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -435,6 +435,18 @@ def get_table_from_response(cls, resp):
         column_names = [field.name for field in resp.schema]
         return agate_helper.table_from_data_flat(resp, column_names)
 
+    def get_labels_from_query_comment(cls):
+        if (
+            hasattr(cls.profile, "query_comment")
+            and cls.profile.query_comment
+            and cls.profile.query_comment.job_label
+            and cls.query_header
+        ):
+            query_comment = cls.query_header.comment.query_comment
+            return cls._labels_from_query_comment(query_comment)
+
+        return {}
+
     def raw_execute(
         self,
         sql,
@@ -447,16 +459,7 @@ def raw_execute(
 
         fire_event(SQLQuery(conn_name=conn.name, sql=sql, node_info=get_node_info()))
 
-        if (
-            hasattr(self.profile, "query_comment")
-            and self.profile.query_comment
-            and self.profile.query_comment.job_label
-            and self.query_header
-            and (query_comment := self.query_header.comment.query_comment)
-        ):
-            labels = self._labels_from_query_comment(query_comment)
-        else:
-            labels = {}
+        labels = self.get_labels_from_query_comment()
 
         if active_user:
             labels["dbt_invocation_id"] = active_user.invocation_id

From 4976d54e276340b2df348276d2d0141c041783ce Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Mon, 9 Oct 2023 22:05:12 -0700
Subject: [PATCH 720/860] test against 1.7.latest release branch and fix semver
 (#961)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* test against 1.6.latest release branch and fix semver

* test against 1.6.latest
---
 .github/workflows/nightly-release.yml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/.github/workflows/nightly-release.yml b/.github/workflows/nightly-release.yml
index 46db5b749..ef210dacd 100644
--- a/.github/workflows/nightly-release.yml
+++ b/.github/workflows/nightly-release.yml
@@ -26,7 +26,7 @@ defaults:
     shell: bash
 
 env:
-  RELEASE_BRANCH: "1.5.latest" # must test against most recent .latest branch to have parity for dependency with core
+  RELEASE_BRANCH: "1.6.latest" # must test against most recent .latest branch to have parity for dependency with core
 
 jobs:
   aggregate-release-data:
@@ -75,7 +75,7 @@ jobs:
       - name: "Generate Nightly Release Version Number"
         id: nightly-release-version
         run: |
-          number="${{ steps.semver.outputs.major }}.${{ steps.semver.outputs.minor }}.${{ steps.bump_patch.outputs.patch }}.dev${{ steps.current-date.outputs.date }}"
+          number="${{ steps.semver.outputs.major }}.${{ steps.semver.outputs.minor }}.${{ steps.bump_patch.outputs.patch }}+dev${{ steps.current-date.outputs.date }}"
           echo "number=$number" >> $GITHUB_OUTPUT
 
       - name: "Audit Nightly Release Version And Parse Into Parts"

From 5396aa0918abe033a2c677579052928ab164c897 Mon Sep 17 00:00:00 2001
From: Gerda Shank <gerda@dbtlabs.com>
Date: Tue, 10 Oct 2023 13:24:38 -0400
Subject: [PATCH 721/860] Support type aliasing in contracts (#954)

---
 .../Under the Hood-20231005-115800.yaml         |  6 ++++++
 dbt/adapters/bigquery/column.py                 |  5 +----
 .../adapter/constraints/test_constraints.py     | 17 +++++++----------
 3 files changed, 14 insertions(+), 14 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20231005-115800.yaml

diff --git a/.changes/unreleased/Under the Hood-20231005-115800.yaml b/.changes/unreleased/Under the Hood-20231005-115800.yaml
new file mode 100644
index 000000000..a5b56ed72
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20231005-115800.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Support for use of type aliases in contract column data_type
+time: 2023-10-05T11:58:00.719136-04:00
+custom:
+  Author: gshank
+  Issue: "953"
diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py
index 1820a6ba7..1bdf4323d 100644
--- a/dbt/adapters/bigquery/column.py
+++ b/dbt/adapters/bigquery/column.py
@@ -13,12 +13,9 @@
 @dataclass(init=False)
 class BigQueryColumn(Column):
     TYPE_LABELS = {
-        "STRING": "STRING",
-        "TIMESTAMP": "TIMESTAMP",
+        "TEXT": "STRING",
         "FLOAT": "FLOAT64",
         "INTEGER": "INT64",
-        "BOOLEAN": "BOOLEAN",
-        "RECORD": "RECORD",
     }
     fields: List[Self]  # type: ignore
     mode: str  # type: ignore
diff --git a/tests/functional/adapter/constraints/test_constraints.py b/tests/functional/adapter/constraints/test_constraints.py
index c8c0c11e2..013f2948b 100644
--- a/tests/functional/adapter/constraints/test_constraints.py
+++ b/tests/functional/adapter/constraints/test_constraints.py
@@ -47,7 +47,7 @@
 
 _expected_sql_bigquery = """
 create or replace table <model_identifier> (
-    id integer not null primary key not enforced references <foreign_key_model_identifier> (id) not enforced,
+    id INT64 not null primary key not enforced references <foreign_key_model_identifier> (id) not enforced,
     color string,
     date_day string
 )
@@ -79,12 +79,9 @@
 """
 
 # Different on BigQuery:
-# - does not support a data type named 'text' (TODO handle this via type translation/aliasing!)
-constraints_yml = model_schema_yml.replace("text", "string")
-model_constraints_yml = constrained_model_schema_yml.replace("text", "string")
-model_contract_header_schema_yml = model_contract_header_schema_yml.replace("text", "string")
-model_fk_constraint_schema_yml = model_fk_constraint_schema_yml.replace("text", "string")
-constrained_model_schema_yml = constrained_model_schema_yml.replace("text", "string")
+# Switch from text to string handled by aliasing
+constraints_yml = model_schema_yml
+model_constraints_yml = constrained_model_schema_yml
 
 my_model_contract_sql_header_sql = """
 {{
@@ -334,7 +331,7 @@ def models(self):
     def expected_sql(self):
         return """
 create or replace table <model_identifier> (
-    id integer not null,
+    id INT64 not null,
     color string,
     date_day string,
     primary key (id) not enforced,
@@ -361,14 +358,14 @@ class TestBigQueryConstraintQuotedColumn(BaseConstraintQuotedColumn):
     def models(self):
         return {
             "my_model.sql": my_model_with_quoted_column_name_sql,
-            "constraints_schema.yml": model_quoted_column_schema_yml.replace("text", "string"),
+            "constraints_schema.yml": model_quoted_column_schema_yml,
         }
 
     @pytest.fixture(scope="class")
     def expected_sql(self):
         return """
 create or replace table <model_identifier> (
-    id integer not null,
+    id INT64 not null,
     `from` string not null,
     date_day string
 )

From 2c1d4ea04b64bba64c3bd367c89f2104a58d53a3 Mon Sep 17 00:00:00 2001
From: Martin Scheubrein <m.sche@seznam.cz>
Date: Wed, 11 Oct 2023 02:46:41 +0200
Subject: [PATCH 722/860] Fix bigquery copy materialization (#910)

* Fix bigquery copy materialization

---------

Co-authored-by: Martin Scheubrein <m-sche@users.noreply.github.com>
Co-authored-by: Anders <anders.swanson@dbtlabs.com>
Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .../unreleased/Fixes-20230906-141819.yaml     |  6 ++
 .../bigquery/macros/materializations/copy.sql |  2 +-
 .../adapter/test_copy_materialization.py      | 62 +++++++++++++++++++
 3 files changed, 69 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20230906-141819.yaml
 create mode 100644 tests/functional/adapter/test_copy_materialization.py

diff --git a/.changes/unreleased/Fixes-20230906-141819.yaml b/.changes/unreleased/Fixes-20230906-141819.yaml
new file mode 100644
index 000000000..f40944670
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230906-141819.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fix bigquery copy materialization
+time: 2023-09-06T14:18:19.445262+02:00
+custom:
+  Author: m-sche
+  Issue: "820"
diff --git a/dbt/include/bigquery/macros/materializations/copy.sql b/dbt/include/bigquery/macros/materializations/copy.sql
index 8285dc845..3ce2de184 100644
--- a/dbt/include/bigquery/macros/materializations/copy.sql
+++ b/dbt/include/bigquery/macros/materializations/copy.sql
@@ -9,7 +9,7 @@
   {# cycle over ref() and source() to create source tables array #}
   {% set source_array = [] %}
   {% for ref_table in model.refs %}
-    {{ source_array.append(ref(*ref_table)) }}
+    {{ source_array.append(ref(ref_table.get('package'), ref_table.name, version=ref_table.get('version'))) }}
   {% endfor %}
 
   {% for src_table in model.sources %}
diff --git a/tests/functional/adapter/test_copy_materialization.py b/tests/functional/adapter/test_copy_materialization.py
new file mode 100644
index 000000000..77d5f4af6
--- /dev/null
+++ b/tests/functional/adapter/test_copy_materialization.py
@@ -0,0 +1,62 @@
+import pytest
+from pathlib import Path
+from dbt.tests.util import run_dbt, write_file, check_relations_equal
+
+_SEED_A = """
+load_date,id,first_name,last_name,email,gender,ip_address
+2021-03-05,1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
+2021-03-05,2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
+2021-03-05,3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
+""".lstrip()
+
+_SEED_B = """
+load_date,id,first_name,last_name,email,gender,ip_address
+2021-03-05,4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
+2021-03-05,5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
+""".lstrip()
+
+_EXPECTED_RESULT = """
+load_date,id,first_name,last_name,email,gender,ip_address
+2021-03-05,1,Jack,Hunter,jhunter0@pbs.org,Male,59.80.20.168
+2021-03-05,2,Kathryn,Walker,kwalker1@ezinearticles.com,Female,194.121.179.35
+2021-03-05,3,Gerald,Ryan,gryan2@com.com,Male,11.3.212.243
+2021-03-05,4,Bonnie,Spencer,bspencer3@ameblo.jp,Female,216.32.196.175
+2021-03-05,5,Harold,Taylor,htaylor4@people.com.cn,Male,253.10.246.136
+""".lstrip()
+
+_COPY_MODEL = """
+{{ config(
+    materialized="copy",
+    copy_materialization="incremental",
+) }}
+
+SELECT * FROM {{ ref("seed") }}
+"""
+
+
+class BaseCopyModelConfig:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {"copy_model.sql": _COPY_MODEL}
+
+    @pytest.fixture(scope="class")
+    def seeds(self):
+        return {
+            "seed.csv": _SEED_A,
+            "expected_result.csv": _EXPECTED_RESULT,
+        }
+
+
+class TestCopyMaterialization(BaseCopyModelConfig):
+    def test_incremental_copy(self, project):
+        run_dbt(["seed"])
+        run_dbt(["run"])
+
+        # Replace original seed _SEED_A with _SEED_B
+        seed_file = project.project_root / Path("seeds") / Path("seed.csv")
+        write_file(_SEED_B, seed_file)
+
+        run_dbt(["seed"])
+        run_dbt(["run"])
+
+        check_relations_equal(project.adapter, ["copy_model", "expected_result"])

From 593752fe1e038302fbc96d62d775036c565263a8 Mon Sep 17 00:00:00 2001
From: KalanyuZ <kalanyuz@users.noreply.github.com>
Date: Wed, 11 Oct 2023 14:05:06 +0900
Subject: [PATCH 723/860] [ADAP-492] Support partition_by and cluster_by on
 python models (#680) (#681)

* support partition_by and cluster_by on python models when supplied in model configurations
* add integration test for partitioned models
* fix typo on partitionType field
---
 .../unreleased/Features-20230426-152526.yaml  |  7 ++
 .../macros/materializations/table.sql         | 17 +++-
 tests/functional/adapter/test_python_model.py | 95 +++++++++++++++++++
 3 files changed, 115 insertions(+), 4 deletions(-)
 create mode 100644 .changes/unreleased/Features-20230426-152526.yaml

diff --git a/.changes/unreleased/Features-20230426-152526.yaml b/.changes/unreleased/Features-20230426-152526.yaml
new file mode 100644
index 000000000..8481f1f74
--- /dev/null
+++ b/.changes/unreleased/Features-20230426-152526.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: Support partition_by and cluster_by on python models when supplied in model
+  configurations
+time: 2023-04-26T15:25:26.285021+09:00
+custom:
+  Author: kalanyuz
+  Issue: "680"
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index c8c88aa3c..16b454351 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -45,11 +45,11 @@
 
 {% endmaterialization %}
 
--- TODO dataproc requires a temp bucket to perform BQ write
--- this is hard coded to internal testing ATM. need to adjust to render
--- or find another way around
 {% macro py_write_table(compiled_code, target_relation) %}
 from pyspark.sql import SparkSession
+{%- set raw_partition_by = config.get('partition_by', none) -%}
+{%- set raw_cluster_by = config.get('cluster_by', none) -%}
+{%- set partition_config = adapter.parse_partition_by(raw_partition_by) %}
 
 spark = SparkSession.builder.appName('smallTest').getOrCreate()
 
@@ -109,6 +109,15 @@ else:
 df.write \
   .mode("overwrite") \
   .format("bigquery") \
-  .option("writeMethod", "direct").option("writeDisposition", 'WRITE_TRUNCATE') \
+  .option("writeMethod", "indirect").option("writeDisposition", 'WRITE_TRUNCATE') \
+  {%- if partition_config.data_type | lower in ('date','timestamp','datetime') %}
+  .option("partitionField", "{{- partition_config.field -}}") \
+  {%- if partition_config.granularity is not none %}
+  .option("partitionType", "{{- partition_config.granularity -}}") \
+  {%- endif %}
+  {%- endif %}
+  {%- if raw_cluster_by is not none %}
+  .option("clusteredFields", "{{- raw_cluster_by|join(',') -}}") \
+  {%- endif %}
   .save("{{target_relation}}")
 {% endmacro %}
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
index b389fe8aa..b67384667 100644
--- a/tests/functional/adapter/test_python_model.py
+++ b/tests/functional/adapter/test_python_model.py
@@ -54,6 +54,101 @@ def model(dbt, spark):
     return spark.createDataFrame(data, schema=['test', 'test2'])
 """
 
+macro__partition_count_sql = """
+{% test number_partitions(model, expected) %}
+
+    {%- set result = get_partitions_metadata(model) %}
+
+    {% if result %}
+        {% set partitions = result.columns['partition_id'].values() %}
+    {% else %}
+        {% set partitions = () %}
+    {% endif %}
+
+    {% set actual = partitions | length %}
+    {% set success = 1 if model and actual == expected else 0 %}
+
+    select 'Expected {{ expected }}, but got {{ actual }}' as validation_error
+    from (select true)
+    where {{ success }} = 0
+
+{% endtest %}
+"""
+
+models__partitioned_model_python = """
+import pandas as pd
+
+def model(dbt, spark):
+    dbt.config(
+        materialized='table',
+        partition_by={
+                "field": "C",
+                "data_type": "timestamp",
+                "granularity": "day",
+            },
+        cluster_by=["A"],
+    )
+    random_array = [
+        ["A", -157.9871329592354],
+        ["B", -528.9769041860632],
+        ["B", 941.0504221837489],
+        ["B", 919.5903586746183],
+        ["A", -121.25678519054622],
+        ["A", 254.9985130814921],
+        ["A", 833.2963094260072],
+    ]
+
+    df = pd.DataFrame(random_array, columns=["A", "B"])
+
+    df["C"] = pd.to_datetime('now')
+
+    final_df = df[["A", "B", "C"]]
+
+    return final_df
+"""
+
+models__partitioned_model_yaml = """
+models:
+  - name: python_partitioned_model
+    description: A random table with a calculated column defined in python.
+    config:
+      batch_id: '{{ run_started_at.strftime("%Y-%m-%d-%H-%M-%S") }}-python-partitioned'
+    tests:
+      - number_partitions:
+          expected: "{{ var('expected', 1) }}"
+    columns:
+      - name: A
+        description: Column A
+      - name: B
+        description: Column B
+      - name: C
+        description: Column C
+"""
+
+
+class TestPythonPartitionedModels:
+    @pytest.fixture(scope="class")
+    def macros(self):
+        return {"partition_metadata.sql": macro__partition_count_sql}
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "python_partitioned_model.py": models__partitioned_model_python,
+            "python_partitioned_model.yml": models__partitioned_model_yaml,
+        }
+
+    def test_multiple_named_python_models(self, project):
+        result = run_dbt(["run"])
+        assert len(result) == 1
+
+        test_results = run_dbt(["test"])
+        for result in test_results:
+            assert result.status == "pass"
+            assert not result.skipped
+            assert result.failures == 0
+
+
 models__simple_python_model_v2 = """
 import pandas
 

From 2d5c5a5d8ec99352afb12fd8bdebcd0b477fad08 Mon Sep 17 00:00:00 2001
From: CM Lubinski <cmc333333@gmail.com>
Date: Wed, 11 Oct 2023 12:19:47 -0400
Subject: [PATCH 724/860] Remove impersonation lifetime. (#882)

* Remove impersonation lifetime.

Per https://github.com/dbt-labs/dbt-bigquery/issues/769 , we shouldn't
need to set the lifetime of an impersonated token. In some edge cases,
attempting to set it to the job execution timeout (> 300 seconds) can
result in authentication failures, even though the job will complete
successfully and authentication creds will be correctly refreshed.

* Changie data.

---------

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20230817-095527.yaml | 6 ++++++
 dbt/adapters/bigquery/connections.py           | 1 -
 2 files changed, 6 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20230817-095527.yaml

diff --git a/.changes/unreleased/Fixes-20230817-095527.yaml b/.changes/unreleased/Fixes-20230817-095527.yaml
new file mode 100644
index 000000000..34f1bf908
--- /dev/null
+++ b/.changes/unreleased/Fixes-20230817-095527.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Avoid setting lifetime on impersonation tokens.
+time: 2023-08-17T09:55:27.333914673-04:00
+custom:
+  Author: cmc333333
+  Issue: "769"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index ff544f0d0..132854748 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -356,7 +356,6 @@ def get_impersonated_credentials(cls, profile_credentials):
             source_credentials=source_credentials,
             target_principal=profile_credentials.impersonate_service_account,
             target_scopes=list(profile_credentials.scopes),
-            lifetime=(profile_credentials.job_execution_timeout_seconds or 300),
         )
 
     @classmethod

From 30b8bc2259e126f9e8e21733036a98ab3e0d4a51 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 11 Oct 2023 12:35:36 -0400
Subject: [PATCH 725/860] ADAP-892: Support test results as views (#932)

* implement tests for persist test results as view

* update changelog entry to reflect renamed parameter

* update test names to reflect renamed parameter

* implement store-failures-as tests

* revert dev requirements to point back to main on dbt-core

* update tests to include recent tests from dbt-core
---
 .../unreleased/Features-20230921-175106.yaml  |  6 +++++
 Makefile                                      |  1 +
 .../test_store_test_failures.py               | 26 +++++++++++++++++++
 3 files changed, 33 insertions(+)
 create mode 100644 .changes/unreleased/Features-20230921-175106.yaml

diff --git a/.changes/unreleased/Features-20230921-175106.yaml b/.changes/unreleased/Features-20230921-175106.yaml
new file mode 100644
index 000000000..cd833de67
--- /dev/null
+++ b/.changes/unreleased/Features-20230921-175106.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Support storing test failures as views
+time: 2023-09-21T17:51:06.389529-04:00
+custom:
+  Author: mikealfare
+  Issue: "6914"
diff --git a/Makefile b/Makefile
index c75b0c6a9..fc6536f98 100644
--- a/Makefile
+++ b/Makefile
@@ -9,6 +9,7 @@ dev: ## Installs adapter in develop mode along with development dependencies
 dev-uninstall: ## Uninstalls all packages while maintaining the virtual environment
                ## Useful when updating versions, or if you accidentally installed into the system interpreter
 	pip freeze | grep -v "^-e" | cut -d "@" -f1 | xargs pip uninstall -y
+	pip uninstall -y dbt-bigquery
 
 .PHONY: ubuntu-py311
 ubuntu-py311: ## Builds and runs an Ubuntu Python 3.11 development container
diff --git a/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py b/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
index d22dab3ae..b9de2b80b 100644
--- a/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
+++ b/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
@@ -1,4 +1,6 @@
 import pytest
+
+from dbt.tests.adapter.store_test_failures_tests import basic
 from dbt.tests.adapter.store_test_failures_tests.test_store_test_failures import (
     StoreTestFailuresBase,
 )
@@ -20,3 +22,27 @@ def teardown_method(self, project):
     def test_store_and_assert(self, project):
         self.run_tests_store_one_failure(project)
         self.run_tests_store_failures_and_assert(project)
+
+
+class TestStoreTestFailuresAsInteractions(basic.StoreTestFailuresAsInteractions):
+    pass
+
+
+class TestStoreTestFailuresAsProjectLevelOff(basic.StoreTestFailuresAsProjectLevelOff):
+    pass
+
+
+class TestStoreTestFailuresAsProjectLevelView(basic.StoreTestFailuresAsProjectLevelView):
+    pass
+
+
+class TestStoreTestFailuresAsGeneric(basic.StoreTestFailuresAsGeneric):
+    pass
+
+
+class TestStoreTestFailuresAsProjectLevelEphemeral(basic.StoreTestFailuresAsProjectLevelEphemeral):
+    pass
+
+
+class TestStoreTestFailuresAsExceptions(basic.StoreTestFailuresAsExceptions):
+    pass

From b06414f50c852e21a50adb89acfceec3e41a9a38 Mon Sep 17 00:00:00 2001
From: Quigley Malcolm <QMalcolm@users.noreply.github.com>
Date: Wed, 11 Oct 2023 11:26:21 -0700
Subject: [PATCH 726/860] Add tests for `date_spine` macro, and sub macros
 (#943)

* Add tests for `date_spine` macro, and sub macros

* fix test fixtures

* fix typo

---------

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Co-authored-by: Colin <colin.rogers@dbtlabs.com>
---
 .../utils/fixture_get_intervals_between.py    | 16 ++++++++++
 tests/functional/adapter/utils/test_utils.py  | 31 +++++++++++++++++++
 2 files changed, 47 insertions(+)
 create mode 100644 tests/functional/adapter/utils/fixture_get_intervals_between.py

diff --git a/tests/functional/adapter/utils/fixture_get_intervals_between.py b/tests/functional/adapter/utils/fixture_get_intervals_between.py
new file mode 100644
index 000000000..be25d6bd9
--- /dev/null
+++ b/tests/functional/adapter/utils/fixture_get_intervals_between.py
@@ -0,0 +1,16 @@
+models__bq_test_get_intervals_between_sql = """
+SELECT
+  {{ get_intervals_between("'2023-09-01'", "'2023-09-12'", "day") }} as intervals,
+  11 as expected
+
+"""
+
+models___bq_test_get_intervals_between_yml = """
+version: 2
+models:
+  - name: test_get_intervals_between
+    tests:
+      - assert_equal:
+          actual: intervals
+          expected: expected
+"""
diff --git a/tests/functional/adapter/utils/test_utils.py b/tests/functional/adapter/utils/test_utils.py
index dc42c4db0..58cecdc7e 100644
--- a/tests/functional/adapter/utils/test_utils.py
+++ b/tests/functional/adapter/utils/test_utils.py
@@ -13,9 +13,13 @@
 from dbt.tests.adapter.utils.test_current_timestamp import BaseCurrentTimestampAware
 from dbt.tests.adapter.utils.test_dateadd import BaseDateAdd
 from dbt.tests.adapter.utils.test_datediff import BaseDateDiff
+from dbt.tests.adapter.utils.test_date_spine import BaseDateSpine
 from dbt.tests.adapter.utils.test_date_trunc import BaseDateTrunc
 from dbt.tests.adapter.utils.test_escape_single_quotes import BaseEscapeSingleQuotesBackslash
 from dbt.tests.adapter.utils.test_except import BaseExcept
+from dbt.tests.adapter.utils.test_generate_series import BaseGenerateSeries
+from dbt.tests.adapter.utils.test_get_intervals_between import BaseGetIntervalsBetween
+from dbt.tests.adapter.utils.test_get_powers_of_two import BaseGetPowersOfTwo
 from dbt.tests.adapter.utils.test_hash import BaseHash
 from dbt.tests.adapter.utils.test_intersect import BaseIntersect
 from dbt.tests.adapter.utils.test_last_day import BaseLastDay
@@ -41,6 +45,10 @@
     models__array_construct_expected_sql,
     macros__array_to_string_sql,
 )
+from tests.functional.adapter.utils.fixture_get_intervals_between import (
+    models__bq_test_get_intervals_between_sql,
+    models___bq_test_get_intervals_between_yml,
+)
 
 
 class TestAnyValue(BaseAnyValue):
@@ -117,6 +125,10 @@ class TestDateDiff(BaseDateDiff):
     pass
 
 
+class TestDateSpine(BaseDateSpine):
+    pass
+
+
 class TestDateTrunc(BaseDateTrunc):
     pass
 
@@ -129,6 +141,25 @@ class TestExcept(BaseExcept):
     pass
 
 
+class TestGenerateSeries(BaseGenerateSeries):
+    pass
+
+
+class TestGetIntervalsBetween(BaseGetIntervalsBetween):
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "test_get_intervals_between.yml": models___bq_test_get_intervals_between_yml,
+            "test_get_intervals_between.sql": self.interpolate_macro_namespace(
+                models__bq_test_get_intervals_between_sql, "get_intervals_between"
+            ),
+        }
+
+
+class TestGetPowersOfTwo(BaseGetPowersOfTwo):
+    pass
+
+
 class TestHash(BaseHash):
     pass
 

From e72cc51e998e6cb9b84fe13c4c093d3523faed9d Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 11 Oct 2023 17:43:57 -0500
Subject: [PATCH 727/860] [ADAP-394] support change monitoring for materialized
 views (#914)

* init push of ADAP-394

* update lots of framework and update to main

* updating based on feedback

* add changelog

* remove in preivew option

* fill out changeset and config change classes for specific options

* change partition_by and cluster_by to FrozenSet, initial attempt at describe.sql

* create utility.py to add bool_setting method, update parse_model_node to use new method

* update describe.sql query

* update describe sql to be able to create list of cluster by field names

* initital attempt at modifying get_alter_materialized_view_as_sql

* update to main and add space

* initial build out of mini classes for bigquery cluster, partition, auto_refresh in materialized_view dict. still need to fix some mypy issues

* remove local package (dbt-bigquery) on `make dev-uninstall`

* update changelog entry to encompass all features in this branch

* remove alteration to setup/teardown for materialized view materialization

* fix spelling error, prepend underscore on base class module to mark as private to its package

* update call to relation to include quote and include policies, update case to match convention

* update create statement to include partition, cluster, and options clauses

* update partition config to align with existing dbt-bigquery table config

* update cluster config to align with existing dbt-bigquery table config

* update auto refresh config to align with other configs

* revert parse results to accept an agate Row

* update how defaults are handled

* add description option to materialized view since it is handled for tables

* add description option to materialized view since it is handled for tables

* fix method call chain in parse_relation_results on cluster, partition, and auto_refresh

* move PartitionConfig into relation_configs to be used by materialized views, update references

* move PartitionConfig into relation_configs to be used by materialized views, update references; add get_materialized_view_options in alignment with get_table_options; fix wild import order; add factory method for materialized views to be used in the jinja template; update expiration timestamp attribute;

* update create materialized view to use the relation config

* condition on existence of properties before templating them

* allow for "drop if exists" functionality via the google sdk

* remove unnecessary trailing semicolon

* implement replace based on create

* implement clustering, partitioning, and auto refresh for materialized views

* remove include_policy from BigQueryRelation, it's causing unit tests to fail and is not used at the moment

* partition type cannot be queried for materialized views, adjust the describe query and equality check to account for that

* add describe_relation for materialized views

* break out common utilities into a mixin for materialized view tests

* change refresh_interval_minutes from an int to a float to match the bigquery docs

* make partition optional on relation results since it cannot be queried yet

* initial draft of materialized view change tests

* build changeset for materialized view

* implement change monitoring for autorefresh and clustering on materialized views, add describe_relation method on BigQueryAdapter for utility

* committing to park changes and wrap up other 1.7 items

* update describe to use the sdk instead of sql to pick up partition information

* basic tests pass

* existing change monitoring tests pass

* partition change monitoring tests pass

* ADAP-940: Add change monitoring for partitioning clause (#962)

* committing to park changes and wrap up other 1.7 items

* update describe to use the sdk instead of sql to pick up partition information

* basic tests pass

* existing change monitoring tests pass

* partition change monitoring tests pass

---------

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>

* implement PR review feedback

* delete empty file

* add MV tests for cluster and partition alone, update combined tests to perform all checks

---------

Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .../unreleased/Features-20230913-130445.yaml  |   6 +
 .flake8                                       |   2 +
 dbt/adapters/bigquery/impl.py                 | 197 ++++++------------
 dbt/adapters/bigquery/relation.py             |  58 +++++-
 .../bigquery/relation_configs/__init__.py     |  21 ++
 .../bigquery/relation_configs/_base.py        |  62 ++++++
 .../bigquery/relation_configs/_cluster.py     |  54 +++++
 .../relation_configs/_materialized_view.py    | 128 ++++++++++++
 .../bigquery/relation_configs/_options.py     | 154 ++++++++++++++
 .../bigquery/relation_configs/_partition.py   | 159 ++++++++++++++
 .../bigquery/relation_configs/_policies.py    |  16 ++
 dbt/adapters/bigquery/utility.py              |  45 ++++
 .../materializations/materialized_view.sql    |   8 -
 .../relations/materialized_view/_replace.sql  |   7 -
 .../relations/materialized_view/alter.sql     |  22 +-
 .../relations/materialized_view/create.sql    |  10 +-
 .../relations/materialized_view/refresh.sql   |   2 +-
 .../relations/materialized_view/replace.sql   |  11 +
 .../adapter/describe_relation/_files.py       |  88 ++++++++
 .../test_describe_relation.py                 | 100 +++++++++
 .../adapter/materialized_view_tests/_files.py |  69 ++++++
 .../adapter/materialized_view_tests/_mixin.py | 104 +++++++++
 .../test_materialized_view.py                 |  62 +-----
 .../test_materialized_view_changes.py         |  95 +++++++++
 .../test_materialized_view_cluster_changes.py |  69 ++++++
 ...est_materialized_view_partition_changes.py |  93 +++++++++
 .../adapter/materialized_view_tests/utils.py  |  13 --
 tests/unit/test_bigquery_adapter.py           |   6 +-
 28 files changed, 1424 insertions(+), 237 deletions(-)
 create mode 100644 .changes/unreleased/Features-20230913-130445.yaml
 create mode 100644 dbt/adapters/bigquery/relation_configs/__init__.py
 create mode 100644 dbt/adapters/bigquery/relation_configs/_base.py
 create mode 100644 dbt/adapters/bigquery/relation_configs/_cluster.py
 create mode 100644 dbt/adapters/bigquery/relation_configs/_materialized_view.py
 create mode 100644 dbt/adapters/bigquery/relation_configs/_options.py
 create mode 100644 dbt/adapters/bigquery/relation_configs/_partition.py
 create mode 100644 dbt/adapters/bigquery/relation_configs/_policies.py
 create mode 100644 dbt/adapters/bigquery/utility.py
 delete mode 100644 dbt/include/bigquery/macros/materializations/materialized_view.sql
 delete mode 100644 dbt/include/bigquery/macros/relations/materialized_view/_replace.sql
 create mode 100644 dbt/include/bigquery/macros/relations/materialized_view/replace.sql
 create mode 100644 tests/functional/adapter/describe_relation/_files.py
 create mode 100644 tests/functional/adapter/describe_relation/test_describe_relation.py
 create mode 100644 tests/functional/adapter/materialized_view_tests/_files.py
 create mode 100644 tests/functional/adapter/materialized_view_tests/_mixin.py
 create mode 100644 tests/functional/adapter/materialized_view_tests/test_materialized_view_changes.py
 create mode 100644 tests/functional/adapter/materialized_view_tests/test_materialized_view_cluster_changes.py
 create mode 100644 tests/functional/adapter/materialized_view_tests/test_materialized_view_partition_changes.py
 delete mode 100644 tests/functional/adapter/materialized_view_tests/utils.py

diff --git a/.changes/unreleased/Features-20230913-130445.yaml b/.changes/unreleased/Features-20230913-130445.yaml
new file mode 100644
index 000000000..42311fd92
--- /dev/null
+++ b/.changes/unreleased/Features-20230913-130445.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: "Support change monitoring for materialized views, including: autorefresh, clustering, partitioning"
+time: 2023-09-13T13:04:45.761294-05:00
+custom:
+  Author: McKnight-42
+  Issue: "924"
diff --git a/.flake8 b/.flake8
index 11baa8ee0..da7e039fd 100644
--- a/.flake8
+++ b/.flake8
@@ -12,3 +12,5 @@ ignore =
     E741,
     E501,
 exclude = tests
+per-file-ignores =
+    */__init__.py: F401
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 8fc1b69bb..50ce21f11 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -1,58 +1,56 @@
 from dataclasses import dataclass
+import json
 import threading
-from typing import Dict, List, Optional, Any, Set, Union, Type
-
-from dbt.contracts.connection import AdapterResponse
-from dbt.contracts.graph.nodes import ColumnLevelConstraint, ModelLevelConstraint, ConstraintType  # type: ignore
-from dbt.dataclass_schema import dbtClassMixin, ValidationError
-
-import dbt.deprecations
-import dbt.exceptions
-import dbt.clients.agate_helper
+import time
+from typing import Any, Dict, List, Optional, Type, Set, Union
 
+import agate
 from dbt import ui  # type: ignore
 from dbt.adapters.base import (  # type: ignore
+    AdapterConfig,
     BaseAdapter,
+    BaseRelation,
     ConstraintSupport,
-    available,
+    PythonJobHelper,
     RelationType,
-    BaseRelation,
     SchemaSearchMap,
-    AdapterConfig,
-    PythonJobHelper,
+    available,
 )
-
 from dbt.adapters.cache import _make_ref_key_dict  # type: ignore
-
-from dbt.adapters.bigquery.column import get_nested_column_data_types
-from dbt.adapters.bigquery.relation import BigQueryRelation
-from dbt.adapters.bigquery.dataset import add_access_entry_to_dataset, is_access_entry_in_dataset
-from dbt.adapters.bigquery import BigQueryColumn
-from dbt.adapters.bigquery import BigQueryConnectionManager
-from dbt.adapters.bigquery.python_submissions import (
-    ClusterDataprocHelper,
-    ServerlessDataProcHelper,
-)
-from dbt.adapters.bigquery.connections import BigQueryAdapterResponse
+import dbt.clients.agate_helper
+from dbt.contracts.connection import AdapterResponse
 from dbt.contracts.graph.manifest import Manifest
-from dbt.events import (
-    AdapterLogger,
-)
+from dbt.contracts.graph.nodes import ColumnLevelConstraint, ConstraintType, ModelLevelConstraint  # type: ignore
+from dbt.dataclass_schema import dbtClassMixin
+import dbt.deprecations
+from dbt.events import AdapterLogger
 from dbt.events.functions import fire_event
 from dbt.events.types import SchemaCreation, SchemaDrop
+import dbt.exceptions
 from dbt.utils import filter_null_values
-
-import google.auth
 import google.api_core
+import google.auth
 import google.oauth2
-import google.cloud.exceptions
 import google.cloud.bigquery
+from google.cloud.bigquery import AccessEntry, SchemaField, Table as BigQueryTable
+import google.cloud.exceptions
 
-from google.cloud.bigquery import AccessEntry, SchemaField
+from dbt.adapters.bigquery import BigQueryColumn, BigQueryConnectionManager
+from dbt.adapters.bigquery.column import get_nested_column_data_types
+from dbt.adapters.bigquery.connections import BigQueryAdapterResponse
+from dbt.adapters.bigquery.dataset import add_access_entry_to_dataset, is_access_entry_in_dataset
+from dbt.adapters.bigquery.python_submissions import (
+    ClusterDataprocHelper,
+    ServerlessDataProcHelper,
+)
+from dbt.adapters.bigquery.relation import BigQueryRelation
+from dbt.adapters.bigquery.relation_configs import (
+    BigQueryBaseRelationConfig,
+    BigQueryMaterializedViewConfig,
+    PartitionConfig,
+)
+from dbt.adapters.bigquery.utility import sql_escape
 
-import time
-import agate
-import json
 
 logger = AdapterLogger("BigQuery")
 
@@ -64,105 +62,6 @@
 _dataset_lock = threading.Lock()
 
 
-def sql_escape(string):
-    if not isinstance(string, str):
-        raise dbt.exceptions.CompilationError(f"cannot escape a non-string: {string}")
-    return json.dumps(string)[1:-1]
-
-
-@dataclass
-class PartitionConfig(dbtClassMixin):
-    field: str
-    data_type: str = "date"
-    granularity: str = "day"
-    range: Optional[Dict[str, Any]] = None
-    time_ingestion_partitioning: bool = False
-    copy_partitions: bool = False
-
-    PARTITION_DATE = "_PARTITIONDATE"
-    PARTITION_TIME = "_PARTITIONTIME"
-
-    def data_type_for_partition(self):
-        """Return the data type of partitions for replacement.
-        When time_ingestion_partitioning is enabled, the data type supported are date & timestamp.
-        """
-        if not self.time_ingestion_partitioning:
-            return self.data_type
-
-        return "date" if self.data_type == "date" else "timestamp"
-
-    def reject_partition_field_column(self, columns: List[Any]) -> List[str]:
-        return [c for c in columns if not c.name.upper() == self.field.upper()]
-
-    def data_type_should_be_truncated(self):
-        """Return true if the data type should be truncated instead of cast to the data type."""
-        return not (
-            self.data_type == "int64" or (self.data_type == "date" and self.granularity == "day")
-        )
-
-    def time_partitioning_field(self) -> str:
-        """Return the time partitioning field name based on the data type.
-        The default is _PARTITIONTIME, but for date it is _PARTITIONDATE
-        else it will fail statements for type mismatch."""
-        if self.data_type == "date":
-            return self.PARTITION_DATE
-        else:
-            return self.PARTITION_TIME
-
-    def insertable_time_partitioning_field(self) -> str:
-        """Return the insertable time partitioning field name based on the data type.
-        Practically, only _PARTITIONTIME works so far.
-        The function is meant to keep the call sites consistent as it might evolve."""
-        return self.PARTITION_TIME
-
-    def render(self, alias: Optional[str] = None):
-        column: str = (
-            self.field if not self.time_ingestion_partitioning else self.time_partitioning_field()
-        )
-        if alias:
-            column = f"{alias}.{column}"
-
-        if self.data_type_should_be_truncated():
-            return f"{self.data_type}_trunc({column}, {self.granularity})"
-        else:
-            return column
-
-    def render_wrapped(self, alias: Optional[str] = None):
-        """Wrap the partitioning column when time involved to ensure it is properly cast to matching time."""
-        # if data type is going to be truncated, no need to wrap
-        if (
-            self.data_type in ("date", "timestamp", "datetime")
-            and not self.data_type_should_be_truncated()
-            and not (
-                self.time_ingestion_partitioning and self.data_type == "date"
-            )  # _PARTITIONDATE is already a date
-        ):
-            return f"{self.data_type}({self.render(alias)})"
-        else:
-            return self.render(alias)
-
-    @classmethod
-    def parse(cls, raw_partition_by) -> Optional["PartitionConfig"]:
-        if raw_partition_by is None:
-            return None
-        try:
-            cls.validate(raw_partition_by)
-            return cls.from_dict(
-                {
-                    key: (value.lower() if isinstance(value, str) else value)
-                    for key, value in raw_partition_by.items()
-                }
-            )
-        except ValidationError as exc:
-            raise dbt.exceptions.DbtValidationError("Could not parse partition config") from exc
-        except TypeError:
-            raise dbt.exceptions.CompilationError(
-                f"Invalid partition_by config:\n"
-                f"  Got: {raw_partition_by}\n"
-                f'  Expected a dictionary with "field" and "data_type" keys'
-            )
-
-
 @dataclass
 class GrantTarget(dbtClassMixin):
     dataset: str
@@ -241,7 +140,9 @@ def drop_relation(self, relation: BigQueryRelation) -> None:
         conn = self.connections.get_thread_connection()
 
         table_ref = self.get_table_ref_from_relation(relation)
-        conn.handle.delete_table(table_ref)
+
+        # mimic "drop if exists" functionality that's ubiquitous in most sql implementations
+        conn.handle.delete_table(table_ref, not_found_ok=True)
 
     def truncate_relation(self, relation: BigQueryRelation) -> None:
         raise dbt.exceptions.NotImplementedError("`truncate` is not implemented for this adapter!")
@@ -849,6 +750,32 @@ def get_view_options(self, config: Dict[str, Any], node: Dict[str, Any]) -> Dict
         opts = self.get_common_options(config, node)
         return opts
 
+    @available.parse(lambda *a, **k: True)
+    def get_bq_table(self, relation: BigQueryRelation) -> Optional[BigQueryTable]:
+        try:
+            table = self.connections.get_bq_table(
+                relation.database, relation.schema, relation.identifier
+            )
+        except google.cloud.exceptions.NotFound:
+            table = None
+        return table
+
+    @available.parse(lambda *a, **k: True)
+    def describe_relation(
+        self, relation: BigQueryRelation
+    ) -> Optional[BigQueryBaseRelationConfig]:
+        if relation.type == RelationType.MaterializedView:
+            bq_table = self.get_bq_table(relation)
+            parser = BigQueryMaterializedViewConfig
+        else:
+            raise dbt.exceptions.DbtRuntimeError(
+                f"The method `BigQueryAdapter.describe_relation` is not implemented "
+                f"for the relation type: {relation.type}"
+            )
+        if bq_table:
+            return parser.from_bq_table(bq_table)
+        return None
+
     @available.parse_none
     def grant_access_to(self, entity, entity_type, role, grant_target_dict):
         """
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index 37ba33632..a689e76fc 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -1,13 +1,21 @@
 from dataclasses import dataclass
-from typing import Optional
+from typing import FrozenSet, Optional, TypeVar
 
 from itertools import chain, islice
-
+from dbt.context.providers import RuntimeConfigObject
 from dbt.adapters.base.relation import BaseRelation, ComponentName, InformationSchema
+from dbt.adapters.relation_configs import RelationConfigChangeAction
+from dbt.adapters.bigquery.relation_configs import (
+    BigQueryClusterConfigChange,
+    BigQueryMaterializedViewConfig,
+    BigQueryMaterializedViewConfigChangeset,
+    BigQueryOptionsConfigChange,
+    BigQueryPartitionConfigChange,
+)
+from dbt.contracts.graph.nodes import ModelNode
 from dbt.contracts.relation import RelationType
 from dbt.exceptions import CompilationError
 from dbt.utils import filter_null_values
-from typing import TypeVar
 
 
 Self = TypeVar("Self", bound="BigQueryRelation")
@@ -17,9 +25,10 @@
 class BigQueryRelation(BaseRelation):
     quote_character: str = "`"
     location: Optional[str] = None
-    # why do we need to use default_factory here but we can assign it directly in dbt-postgres?
-    renameable_relations = frozenset({RelationType.Table})
-    replaceable_relations = frozenset({RelationType.Table, RelationType.View})
+    renameable_relations: FrozenSet[RelationType] = frozenset({RelationType.Table})
+    replaceable_relations: FrozenSet[RelationType] = frozenset(
+        {RelationType.Table, RelationType.View}
+    )
 
     def matches(
         self,
@@ -53,6 +62,43 @@ def project(self):
     def dataset(self):
         return self.schema
 
+    @classmethod
+    def materialized_view_from_model_node(
+        cls, model_node: ModelNode
+    ) -> BigQueryMaterializedViewConfig:
+        return BigQueryMaterializedViewConfig.from_model_node(model_node)  # type: ignore
+
+    @classmethod
+    def materialized_view_config_changeset(
+        cls,
+        existing_materialized_view: BigQueryMaterializedViewConfig,
+        runtime_config: RuntimeConfigObject,
+    ) -> Optional[BigQueryMaterializedViewConfigChangeset]:
+        config_change_collection = BigQueryMaterializedViewConfigChangeset()
+        new_materialized_view = cls.materialized_view_from_model_node(runtime_config.model)
+
+        if new_materialized_view.options != existing_materialized_view.options:
+            config_change_collection.options = BigQueryOptionsConfigChange(
+                action=RelationConfigChangeAction.alter,
+                context=new_materialized_view.options,
+            )
+
+        if new_materialized_view.partition != existing_materialized_view.partition:
+            config_change_collection.partition = BigQueryPartitionConfigChange(
+                action=RelationConfigChangeAction.alter,
+                context=new_materialized_view.partition,
+            )
+
+        if new_materialized_view.cluster != existing_materialized_view.cluster:
+            config_change_collection.cluster = BigQueryClusterConfigChange(
+                action=RelationConfigChangeAction.alter,
+                context=new_materialized_view.cluster,
+            )
+
+        if config_change_collection:
+            return config_change_collection
+        return None
+
     def information_schema(self, identifier: Optional[str] = None) -> "BigQueryInformationSchema":
         return BigQueryInformationSchema.from_relation(self, identifier)
 
diff --git a/dbt/adapters/bigquery/relation_configs/__init__.py b/dbt/adapters/bigquery/relation_configs/__init__.py
new file mode 100644
index 000000000..9ccdec1e0
--- /dev/null
+++ b/dbt/adapters/bigquery/relation_configs/__init__.py
@@ -0,0 +1,21 @@
+from dbt.adapters.bigquery.relation_configs._base import BigQueryBaseRelationConfig
+from dbt.adapters.bigquery.relation_configs._cluster import (
+    BigQueryClusterConfig,
+    BigQueryClusterConfigChange,
+)
+from dbt.adapters.bigquery.relation_configs._materialized_view import (
+    BigQueryMaterializedViewConfig,
+    BigQueryMaterializedViewConfigChangeset,
+)
+from dbt.adapters.bigquery.relation_configs._options import (
+    BigQueryOptionsConfig,
+    BigQueryOptionsConfigChange,
+)
+from dbt.adapters.bigquery.relation_configs._partition import (
+    PartitionConfig,
+    BigQueryPartitionConfigChange,
+)
+from dbt.adapters.bigquery.relation_configs._policies import (
+    BigQueryIncludePolicy,
+    BigQueryQuotePolicy,
+)
diff --git a/dbt/adapters/bigquery/relation_configs/_base.py b/dbt/adapters/bigquery/relation_configs/_base.py
new file mode 100644
index 000000000..c92f7c01a
--- /dev/null
+++ b/dbt/adapters/bigquery/relation_configs/_base.py
@@ -0,0 +1,62 @@
+from dataclasses import dataclass
+from typing import Optional
+
+import agate
+from dbt.adapters.base.relation import Policy
+from dbt.adapters.relation_configs import RelationConfigBase
+from google.cloud.bigquery import Table as BigQueryTable
+
+from dbt.adapters.bigquery.relation_configs._policies import (
+    BigQueryIncludePolicy,
+    BigQueryQuotePolicy,
+)
+from dbt.contracts.graph.nodes import ModelNode
+from dbt.contracts.relation import ComponentName
+
+
+@dataclass(frozen=True, eq=True, unsafe_hash=True)
+class BigQueryBaseRelationConfig(RelationConfigBase):
+    @classmethod
+    def include_policy(cls) -> Policy:
+        return BigQueryIncludePolicy()
+
+    @classmethod
+    def quote_policy(cls) -> Policy:
+        return BigQueryQuotePolicy()
+
+    @classmethod
+    def from_model_node(cls, model_node: ModelNode) -> "BigQueryBaseRelationConfig":
+        relation_config = cls.parse_model_node(model_node)
+        relation = cls.from_dict(relation_config)
+        return relation  # type: ignore
+
+    @classmethod
+    def parse_model_node(cls, model_node: ModelNode) -> dict:
+        raise NotImplementedError(
+            "`parse_model_node()` needs to be implemented on this RelationConfigBase instance"
+        )
+
+    @classmethod
+    def from_bq_table(cls, table: BigQueryTable) -> "BigQueryBaseRelationConfig":
+        relation_config = cls.parse_bq_table(table)
+        relation = cls.from_dict(relation_config)
+        return relation  # type: ignore
+
+    @classmethod
+    def parse_bq_table(cls, table: BigQueryTable) -> dict:
+        raise NotImplementedError("`parse_bq_table()` is not implemented for this relation type")
+
+    @classmethod
+    def _render_part(cls, component: ComponentName, value: Optional[str]) -> Optional[str]:
+        if cls.include_policy().get_part(component) and value:
+            if cls.quote_policy().get_part(component):
+                return f'"{value}"'
+            return value.lower()
+        return None
+
+    @classmethod
+    def _get_first_row(cls, results: agate.Table) -> agate.Row:
+        try:
+            return results.rows[0]
+        except IndexError:
+            return agate.Row(values=set())
diff --git a/dbt/adapters/bigquery/relation_configs/_cluster.py b/dbt/adapters/bigquery/relation_configs/_cluster.py
new file mode 100644
index 000000000..c7161a96a
--- /dev/null
+++ b/dbt/adapters/bigquery/relation_configs/_cluster.py
@@ -0,0 +1,54 @@
+from dataclasses import dataclass
+from typing import Any, Dict, FrozenSet, Optional
+
+from dbt.adapters.relation_configs import RelationConfigChange
+from dbt.contracts.graph.nodes import ModelNode
+from google.cloud.bigquery import Table as BigQueryTable
+
+from dbt.adapters.bigquery.relation_configs._base import BigQueryBaseRelationConfig
+
+
+@dataclass(frozen=True, eq=True, unsafe_hash=True)
+class BigQueryClusterConfig(BigQueryBaseRelationConfig):
+    """
+    This config manages table options supporting clustering. See the following for more information:
+        - https://docs.getdbt.com/reference/resource-configs/bigquery-configs#using-table-partitioning-and-clustering
+        - https://cloud.google.com/bigquery/docs/reference/standard-sql/data-definition-language#clustering_column_list
+
+    - fields: set of columns to cluster on
+        - Note: can contain up to four columns
+    """
+
+    fields: FrozenSet[str]
+
+    @classmethod
+    def from_dict(cls, config_dict: Dict[str, Any]) -> "BigQueryClusterConfig":
+        kwargs_dict = {"fields": config_dict.get("fields")}
+        cluster: "BigQueryClusterConfig" = super().from_dict(kwargs_dict)  # type: ignore
+        return cluster
+
+    @classmethod
+    def parse_model_node(cls, model_node: ModelNode) -> Dict[str, Any]:
+        config_dict = {}
+
+        if cluster_by := model_node.config.extra.get("cluster_by"):
+            # users may input a single field as a string
+            if isinstance(cluster_by, str):
+                cluster_by = [cluster_by]
+            config_dict.update({"fields": frozenset(cluster_by)})
+
+        return config_dict
+
+    @classmethod
+    def parse_bq_table(cls, table: BigQueryTable) -> Dict[str, Any]:  # type: ignore
+        config_dict = {"fields": frozenset(table.clustering_fields)}
+        return config_dict
+
+
+@dataclass(frozen=True, eq=True, unsafe_hash=True)
+class BigQueryClusterConfigChange(RelationConfigChange):
+    context: Optional[BigQueryClusterConfig]
+
+    @property
+    def requires_full_refresh(self) -> bool:
+        return True
diff --git a/dbt/adapters/bigquery/relation_configs/_materialized_view.py b/dbt/adapters/bigquery/relation_configs/_materialized_view.py
new file mode 100644
index 000000000..a9baa3ab0
--- /dev/null
+++ b/dbt/adapters/bigquery/relation_configs/_materialized_view.py
@@ -0,0 +1,128 @@
+from dataclasses import dataclass
+from typing import Any, Dict, Optional
+
+from dbt.contracts.graph.nodes import ModelNode
+from dbt.contracts.relation import ComponentName
+from google.cloud.bigquery import Table as BigQueryTable
+
+from dbt.adapters.bigquery.relation_configs._base import BigQueryBaseRelationConfig
+from dbt.adapters.bigquery.relation_configs._options import (
+    BigQueryOptionsConfig,
+    BigQueryOptionsConfigChange,
+)
+from dbt.adapters.bigquery.relation_configs._partition import (
+    BigQueryPartitionConfigChange,
+    PartitionConfig,
+)
+from dbt.adapters.bigquery.relation_configs._cluster import (
+    BigQueryClusterConfig,
+    BigQueryClusterConfigChange,
+)
+
+
+@dataclass(frozen=True, eq=True, unsafe_hash=True)
+class BigQueryMaterializedViewConfig(BigQueryBaseRelationConfig):
+    """
+    This config follow the specs found here:
+    https://cloud.google.com/bigquery/docs/reference/standard-sql/data-definition-language#create_materialized_view_statement
+
+    The following parameters are configurable by dbt:
+    - table_id: name of the materialized view
+    - dataset_id: dataset name of the materialized view
+    - project_id: project name of the database
+    - options: options that get set in `SET OPTIONS()` clause
+    - partition: object containing partition information
+    - cluster: object containing cluster information
+    """
+
+    table_id: str
+    dataset_id: str
+    project_id: str
+    options: BigQueryOptionsConfig
+    partition: Optional[PartitionConfig] = None
+    cluster: Optional[BigQueryClusterConfig] = None
+
+    @classmethod
+    def from_dict(cls, config_dict: Dict[str, Any]) -> "BigQueryMaterializedViewConfig":
+        # required
+        kwargs_dict: Dict[str, Any] = {
+            "table_id": cls._render_part(ComponentName.Identifier, config_dict["table_id"]),
+            "dataset_id": cls._render_part(ComponentName.Schema, config_dict["dataset_id"]),
+            "project_id": cls._render_part(ComponentName.Database, config_dict["project_id"]),
+            "options": BigQueryOptionsConfig.from_dict(config_dict["options"]),
+        }
+
+        # optional
+        if partition := config_dict.get("partition"):
+            kwargs_dict.update({"partition": PartitionConfig.parse(partition)})
+
+        if cluster := config_dict.get("cluster"):
+            kwargs_dict.update({"cluster": BigQueryClusterConfig.from_dict(cluster)})
+
+        materialized_view: "BigQueryMaterializedViewConfig" = super().from_dict(kwargs_dict)  # type: ignore
+        return materialized_view
+
+    @classmethod
+    def parse_model_node(cls, model_node: ModelNode) -> Dict[str, Any]:
+        config_dict = {
+            "table_id": model_node.identifier,
+            "dataset_id": model_node.schema,
+            "project_id": model_node.database,
+            # despite this being a foreign object, there will always be options because of defaults
+            "options": BigQueryOptionsConfig.parse_model_node(model_node),
+        }
+
+        # optional
+        if "partition_by" in model_node.config:
+            config_dict.update({"partition": PartitionConfig.parse_model_node(model_node)})
+
+        if "cluster_by" in model_node.config:
+            config_dict.update({"cluster": BigQueryClusterConfig.parse_model_node(model_node)})
+
+        return config_dict
+
+    @classmethod
+    def parse_bq_table(cls, table: BigQueryTable) -> Dict[str, Any]:
+        config_dict = {
+            "table_id": table.table_id,
+            "dataset_id": table.dataset_id,
+            "project_id": table.project,
+            # despite this being a foreign object, there will always be options because of defaults
+            "options": BigQueryOptionsConfig.parse_bq_table(table),
+        }
+
+        # optional
+        if table.time_partitioning or table.range_partitioning:
+            config_dict.update({"partition": PartitionConfig.parse_bq_table(table)})
+
+        if table.clustering_fields:
+            config_dict.update({"cluster": BigQueryClusterConfig.parse_bq_table(table)})
+
+        return config_dict
+
+
+@dataclass
+class BigQueryMaterializedViewConfigChangeset:
+    options: Optional[BigQueryOptionsConfigChange] = None
+    partition: Optional[BigQueryPartitionConfigChange] = None
+    cluster: Optional[BigQueryClusterConfigChange] = None
+
+    @property
+    def requires_full_refresh(self) -> bool:
+        return any(
+            {
+                self.options.requires_full_refresh if self.options else False,
+                self.partition.requires_full_refresh if self.partition else False,
+                self.cluster.requires_full_refresh if self.cluster else False,
+            }
+        )
+
+    @property
+    def has_changes(self) -> bool:
+        return any(
+            {
+                self.options if self.options else False,
+                self.partition if self.partition else False,
+                self.cluster if self.cluster else False,
+            }
+        )
diff --git a/dbt/adapters/bigquery/relation_configs/_options.py b/dbt/adapters/bigquery/relation_configs/_options.py
new file mode 100644
index 000000000..51774e3fb
--- /dev/null
+++ b/dbt/adapters/bigquery/relation_configs/_options.py
@@ -0,0 +1,154 @@
+from dataclasses import dataclass
+from datetime import datetime, timedelta
+from typing import Any, Dict, Optional
+
+from dbt.adapters.relation_configs import RelationConfigChange
+from dbt.contracts.graph.nodes import ModelNode
+from google.cloud.bigquery import Table as BigQueryTable
+
+from dbt.adapters.bigquery.relation_configs._base import BigQueryBaseRelationConfig
+from dbt.adapters.bigquery.utility import bool_setting, float_setting, sql_escape
+
+
+@dataclass(frozen=True, eq=True, unsafe_hash=True)
+class BigQueryOptionsConfig(BigQueryBaseRelationConfig):
+    """
+    This config manages materialized view options. See the following for more information:
+    - https://cloud.google.com/bigquery/docs/reference/standard-sql/data-definition-language#materialized_view_option_list
+    """
+
+    enable_refresh: Optional[bool] = True
+    refresh_interval_minutes: Optional[float] = 30
+    expiration_timestamp: Optional[datetime] = None
+    max_staleness: Optional[str] = None
+    kms_key_name: Optional[str] = None
+    description: Optional[str] = None
+    labels: Optional[Dict[str, str]] = None
+
+    def as_ddl_dict(self) -> Dict[str, Any]:
+        """
+        Reformat `options_dict` so that it can be passed into the `bigquery_options()` macro.
+
+        Options should be flattened and filtered prior to passing into this method. For example:
+        - the "auto refresh" set of options should be flattened into the root instead of stuck under "auto_refresh"
+        - any option that comes in set as `None` will be unset; this happens mostly due to config changes
+        """
+
+        def boolean(x):
+            return x
+
+        def numeric(x):
+            return x
+
+        def string(x):
+            return f"'{x}'"
+
+        def escaped_string(x):
+            return f'"""{sql_escape(x)}"""'
+
+        def interval(x):
+            return x
+
+        def array(x):
+            return list(x.items())
+
+        option_formatters = {
+            "enable_refresh": boolean,
+            "refresh_interval_minutes": numeric,
+            "expiration_timestamp": interval,
+            "max_staleness": interval,
+            "kms_key_name": string,
+            "description": escaped_string,
+            "labels": array,
+        }
+
+        def formatted_option(name: str) -> Optional[Any]:
+            value = getattr(self, name)
+            if value is not None:
+                formatter = option_formatters[name]
+                return formatter(value)
+            return None
+
+        options = {
+            option: formatted_option(option)
+            for option, option_formatter in option_formatters.items()
+            if formatted_option(option) is not None
+        }
+
+        return options
+
+    @classmethod
+    def from_dict(cls, config_dict: Dict[str, Any]) -> "BigQueryOptionsConfig":
+        setting_formatters = {
+            "enable_refresh": bool_setting,
+            "refresh_interval_minutes": float_setting,
+            "expiration_timestamp": None,
+            "max_staleness": None,
+            "kms_key_name": None,
+            "description": None,
+            "labels": None,
+        }
+
+        def formatted_setting(name: str) -> Any:
+            value = config_dict.get(name)
+            if formatter := setting_formatters[name]:
+                return formatter(value)
+            return value
+
+        kwargs_dict = {attribute: formatted_setting(attribute) for attribute in setting_formatters}
+
+        # avoid picking up defaults on dependent options
+        # e.g. don't set `refresh_interval_minutes` = 30 when the user has `enable_refresh` = False
+        if kwargs_dict["enable_refresh"] is False:
+            kwargs_dict.update({"refresh_interval_minutes": None, "max_staleness": None})
+
+        options: "BigQueryOptionsConfig" = super().from_dict(kwargs_dict)  # type: ignore
+        return options
+
+    @classmethod
+    def parse_model_node(cls, model_node: ModelNode) -> Dict[str, Any]:
+        config_dict = {
+            option: model_node.config.extra.get(option)
+            for option in [
+                "enable_refresh",
+                "refresh_interval_minutes",
+                "expiration_timestamp",
+                "max_staleness",
+                "kms_key_name",
+                "description",
+                "labels",
+            ]
+        }
+
+        # update dbt-specific versions of these settings
+        if hours_to_expiration := model_node.config.extra.get("hours_to_expiration"):
+            config_dict.update(
+                {"expiration_timestamp": datetime.now() + timedelta(hours=hours_to_expiration)}
+            )
+        if not model_node.config.persist_docs:
+            del config_dict["description"]
+
+        return config_dict
+
+    @classmethod
+    def parse_bq_table(cls, table: BigQueryTable) -> Dict[str, Any]:
+        config_dict = {
+            "enable_refresh": table.mview_enable_refresh,
+            "refresh_interval_minutes": table.mview_refresh_interval.seconds / 60,
+            "expiration_timestamp": table.expires,
+            "max_staleness": None,
+            "description": table.description,
+            "labels": table.labels,
+        }
+        if encryption_configuration := table.encryption_configuration:
+            config_dict.update({"kms_key_name": encryption_configuration.kms_key_name})
+        return config_dict
+
+
+@dataclass(frozen=True, eq=True, unsafe_hash=True)
+class BigQueryOptionsConfigChange(RelationConfigChange):
+    context: BigQueryOptionsConfig
+
+    @property
+    def requires_full_refresh(self) -> bool:
+        return False
diff --git a/dbt/adapters/bigquery/relation_configs/_partition.py b/dbt/adapters/bigquery/relation_configs/_partition.py
new file mode 100644
index 000000000..094e4f1c4
--- /dev/null
+++ b/dbt/adapters/bigquery/relation_configs/_partition.py
@@ -0,0 +1,159 @@
+from dataclasses import dataclass
+from typing import Any, Dict, List, Optional
+
+from dbt.adapters.relation_configs import RelationConfigChange
+from dbt.contracts.graph.nodes import ModelNode
+from dbt.dataclass_schema import dbtClassMixin, ValidationError
+import dbt.exceptions
+from google.cloud.bigquery.table import Table as BigQueryTable
+
+
+@dataclass
+class PartitionConfig(dbtClassMixin):
+    field: str
+    data_type: str = "date"
+    granularity: str = "day"
+    range: Optional[Dict[str, Any]] = None
+    time_ingestion_partitioning: bool = False
+    copy_partitions: bool = False
+
+    PARTITION_DATE = "_PARTITIONDATE"
+    PARTITION_TIME = "_PARTITIONTIME"
+
+    def data_type_for_partition(self):
+        """Return the data type of partitions for replacement.
+        When time_ingestion_partitioning is enabled, the data type supported are date & timestamp.
+        """
+        if not self.time_ingestion_partitioning:
+            return self.data_type
+
+        return "date" if self.data_type == "date" else "timestamp"
+
+    def reject_partition_field_column(self, columns: List[Any]) -> List[str]:
+        return [c for c in columns if not c.name.upper() == self.field.upper()]
+
+    def data_type_should_be_truncated(self):
+        """Return true if the data type should be truncated instead of cast to the data type."""
+        return not (
+            self.data_type == "int64" or (self.data_type == "date" and self.granularity == "day")
+        )
+
+    def time_partitioning_field(self) -> str:
+        """Return the time partitioning field name based on the data type.
+        The default is _PARTITIONTIME, but for date it is _PARTITIONDATE
+        else it will fail statements for type mismatch."""
+        if self.data_type == "date":
+            return self.PARTITION_DATE
+        else:
+            return self.PARTITION_TIME
+
+    def insertable_time_partitioning_field(self) -> str:
+        """Return the insertable time partitioning field name based on the data type.
+        Practically, only _PARTITIONTIME works so far.
+        The function is meant to keep the call sites consistent as it might evolve."""
+        return self.PARTITION_TIME
+
+    def render(self, alias: Optional[str] = None):
+        column: str = (
+            self.field if not self.time_ingestion_partitioning else self.time_partitioning_field()
+        )
+        if alias:
+            column = f"{alias}.{column}"
+
+        if self.data_type_should_be_truncated():
+            return f"{self.data_type}_trunc({column}, {self.granularity})"
+        else:
+            return column
+
+    def render_wrapped(self, alias: Optional[str] = None):
+        """Wrap the partitioning column when time involved to ensure it is properly cast to matching time."""
+        # if data type is going to be truncated, no need to wrap
+        if (
+            self.data_type in ("date", "timestamp", "datetime")
+            and not self.data_type_should_be_truncated()
+            and not (
+                self.time_ingestion_partitioning and self.data_type == "date"
+            )  # _PARTITIONDATE is already a date
+        ):
+            return f"{self.data_type}({self.render(alias)})"
+        else:
+            return self.render(alias)
+
+    @classmethod
+    def parse(cls, raw_partition_by) -> Optional["PartitionConfig"]:
+        if raw_partition_by is None:
+            return None
+        try:
+            cls.validate(raw_partition_by)
+            return cls.from_dict(
+                {
+                    key: (value.lower() if isinstance(value, str) else value)
+                    for key, value in raw_partition_by.items()
+                }
+            )
+        except ValidationError as exc:
+            raise dbt.exceptions.DbtValidationError("Could not parse partition config") from exc
+        except TypeError:
+            raise dbt.exceptions.CompilationError(
+                f"Invalid partition_by config:\n"
+                f"  Got: {raw_partition_by}\n"
+                f'  Expected a dictionary with "field" and "data_type" keys'
+            )
+
+    @classmethod
+    def parse_model_node(cls, model_node: ModelNode) -> Dict[str, Any]:
+        """
+        Parse model node into a raw config for `PartitionConfig.parse`
+
+        - Note:
+            This doesn't currently collect `time_ingestion_partitioning` and `copy_partitions`
+            because this was built for materialized views, which do not support those settings.
+        """
+        config_dict = model_node.config.extra.get("partition_by")
+        if "time_ingestion_partitioning" in config_dict:
+            del config_dict["time_ingestion_partitioning"]
+        if "copy_partitions" in config_dict:
+            del config_dict["copy_partitions"]
+        return config_dict
+
+    @classmethod
+    def parse_bq_table(cls, table: BigQueryTable) -> Dict[str, Any]:
+        """
+        Parse the BQ Table object into a raw config for `PartitionConfig.parse`
+
+        - Note:
+            This doesn't currently collect `time_ingestion_partitioning` and `copy_partitions`
+            because this was built for materialized views, which do not support those settings.
+        """
+        if time_partitioning := table.time_partitioning:
+            field_types = {field.name: field.field_type.lower() for field in table.schema}
+            config_dict = {
+                "field": time_partitioning.field,
+                "data_type": field_types[time_partitioning.field],
+                "granularity": time_partitioning.type_,
+            }
+
+        elif range_partitioning := table.range_partitioning:
+            config_dict = {
+                "field": range_partitioning.field,
+                "data_type": "int64",
+                "range": {
+                    "start": range_partitioning.range_.start,
+                    "end": range_partitioning.range_.end,
+                    "interval": range_partitioning.range_.interval,
+                },
+            }
+
+        else:
+            config_dict = {}
+
+        return config_dict
+
+
+@dataclass(frozen=True, eq=True, unsafe_hash=True)
+class BigQueryPartitionConfigChange(RelationConfigChange):
+    context: Optional[PartitionConfig]
+
+    @property
+    def requires_full_refresh(self) -> bool:
+        return True
diff --git a/dbt/adapters/bigquery/relation_configs/_policies.py b/dbt/adapters/bigquery/relation_configs/_policies.py
new file mode 100644
index 000000000..4467c4340
--- /dev/null
+++ b/dbt/adapters/bigquery/relation_configs/_policies.py
@@ -0,0 +1,16 @@
+from dataclasses import dataclass
+
+from dbt.adapters.base.relation import Policy
+
+
+class BigQueryIncludePolicy(Policy):
+    database: bool = True
+    schema: bool = True
+    identifier: bool = True
+
+
+@dataclass
+class BigQueryQuotePolicy(Policy):
+    database: bool = True
+    schema: bool = True
+    identifier: bool = True
diff --git a/dbt/adapters/bigquery/utility.py b/dbt/adapters/bigquery/utility.py
new file mode 100644
index 000000000..c66b2d97d
--- /dev/null
+++ b/dbt/adapters/bigquery/utility.py
@@ -0,0 +1,45 @@
+import json
+from typing import Any, Optional
+
+import dbt.exceptions
+
+
+def bool_setting(value: Optional[Any] = None) -> Optional[bool]:
+    if value is None:
+        return None
+    elif isinstance(value, bool):
+        return value
+    elif isinstance(value, str):
+        # don't do bool(value) as that is equivalent to: len(value) > 0
+        if value.lower() == "true":
+            return True
+        elif value.lower() == "false":
+            return False
+        else:
+            raise ValueError(
+                f"Invalid input, "
+                f"expecting `bool` or `str` ex. (True, False, 'true', 'False'), received: {value}"
+            )
+    else:
+        raise TypeError(
+            f"Invalid type for bool evaluation, "
+            f"expecting `bool` or `str`, received: {type(value)}"
+        )
+
+
+def float_setting(value: Optional[Any] = None) -> Optional[float]:
+    if value is None:
+        return None
+    elif any(isinstance(value, i) for i in [int, float, str]):
+        return float(value)
+    else:
+        raise TypeError(
+            f"Invalid type for float evaluation, "
+            f"expecting `int`, `float`, or `str`, received: {type(value)}"
+        )
+
+
+def sql_escape(string):
+    if not isinstance(string, str):
+        raise dbt.exceptions.CompilationError(f"cannot escape a non-string: {string}")
+    return json.dumps(string)[1:-1]
diff --git a/dbt/include/bigquery/macros/materializations/materialized_view.sql b/dbt/include/bigquery/macros/materializations/materialized_view.sql
deleted file mode 100644
index 8b4168aec..000000000
--- a/dbt/include/bigquery/macros/materializations/materialized_view.sql
+++ /dev/null
@@ -1,8 +0,0 @@
-{% macro materialized_view_setup(backup_relation, intermediate_relation, pre_hooks) %}
-    {{ run_hooks(pre_hooks, inside_transaction=False) }}
-{% endmacro %}
-
-
-{% macro materialized_view_teardown(backup_relation, intermediate_relation, post_hooks) %}
-    {{ run_hooks(post_hooks, inside_transaction=False) }}
-{% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/_replace.sql b/dbt/include/bigquery/macros/relations/materialized_view/_replace.sql
deleted file mode 100644
index c9df0aef6..000000000
--- a/dbt/include/bigquery/macros/relations/materialized_view/_replace.sql
+++ /dev/null
@@ -1,7 +0,0 @@
-{% macro bigquery__get_replace_materialized_view_as_sql(
-    relation,
-    sql
-) %}
-    {{ get_drop_sql(existing_relation) }}
-    {{ get_create_materialized_view_as_sql(relation, sql) }}
-{% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/alter.sql b/dbt/include/bigquery/macros/relations/materialized_view/alter.sql
index 27819c948..7320addde 100644
--- a/dbt/include/bigquery/macros/relations/materialized_view/alter.sql
+++ b/dbt/include/bigquery/macros/relations/materialized_view/alter.sql
@@ -2,14 +2,24 @@
     relation,
     configuration_changes,
     sql,
-    existing_relation
+    existing_relation,
+    backup_relation,
+    intermediate_relation
 ) %}
-    bigquery__get_replace_materialized_view_as_sql(
-        relation,
-        sql
-    )
+
+    {% if configuration_changes.requires_full_refresh %}
+        {{ get_replace_sql(existing_relation, relation, sql) }}
+    {% else %}
+
+        alter materialized view {{ relation }}
+            set {{ bigquery_options(configuration_changes.options.context.as_ddl_dict()) }}
+
+    {%- endif %}
+
 {% endmacro %}
 
 {% macro bigquery__get_materialized_view_configuration_changes(existing_relation, new_config) %}
-    {{- return(None) -}}
+    {% set _existing_materialized_view = adapter.describe_relation(existing_relation) %}
+    {% set _configuration_changes = existing_relation.materialized_view_config_changeset(_existing_materialized_view, new_config) %}
+    {% do return(_configuration_changes) %}
 {% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/create.sql b/dbt/include/bigquery/macros/relations/materialized_view/create.sql
index 198a3a04a..d2b6c5be5 100644
--- a/dbt/include/bigquery/macros/relations/materialized_view/create.sql
+++ b/dbt/include/bigquery/macros/relations/materialized_view/create.sql
@@ -1,3 +1,11 @@
 {% macro bigquery__get_create_materialized_view_as_sql(relation, sql) %}
-    create materialized view if not exists {{ relation }} as {{ sql }}
+
+    {%- set materialized_view = adapter.Relation.materialized_view_from_model_node(config.model) -%}
+
+    create materialized view if not exists {{ relation }}
+    {% if materialized_view.partition %}{{ partition_by(materialized_view.partition) }}{% endif %}
+    {% if materialized_view.cluster %}{{ cluster_by(materialized_view.cluster.fields) }}{% endif %}
+    {{ bigquery_options(materialized_view.options.as_ddl_dict()) }}
+    as {{ sql }}
+
 {% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql b/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql
index 54ad76e1e..82bf819cd 100644
--- a/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql
+++ b/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql
@@ -1,3 +1,3 @@
 {% macro bigquery__refresh_materialized_view(relation) %}
-    CALL BQ.REFRESH_MATERIALIZED_VIEW('{{ relation.database }}.{{ relation.schema }}.{{ relation.identifier }}');
+    call bq.refresh_materialized_view('{{ relation }}')
 {% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/replace.sql b/dbt/include/bigquery/macros/relations/materialized_view/replace.sql
new file mode 100644
index 000000000..4fdfd21f6
--- /dev/null
+++ b/dbt/include/bigquery/macros/relations/materialized_view/replace.sql
@@ -0,0 +1,11 @@
+{% macro bigquery__get_replace_materialized_view_as_sql(relation, sql) %}
+
+    {%- set materialized_view = adapter.Relation.materialized_view_from_model_node(config.model) -%}
+
+    create or replace materialized view if not exists {{ relation }}
+    {% if materialized_view.partition %}{{ partition_by(materialized_view.partition) }}{% endif %}
+    {% if materialized_view.cluster %}{{ cluster_by(materialized_view.cluster.fields) }}{% endif %}
+    {{ bigquery_options(materialized_view.options.as_ddl_dict()) }}
+    as {{ sql }}
+
+{% endmacro %}
diff --git a/tests/functional/adapter/describe_relation/_files.py b/tests/functional/adapter/describe_relation/_files.py
new file mode 100644
index 000000000..ac0203049
--- /dev/null
+++ b/tests/functional/adapter/describe_relation/_files.py
@@ -0,0 +1,88 @@
+MY_SEED = """
+id,value,record_date
+1,100,2023-01-01 12:00:00
+2,200,2023-01-02 12:00:00
+3,300,2023-01-02 12:00:00
+""".strip()
+
+
+MY_BASE_TABLE = """
+{{ config(
+    materialized='table',
+    partition_by={
+        "field": "record_date",
+        "data_type": "datetime",
+        "granularity": "day"
+    },
+    cluster_by=["id", "value"]
+) }}
+select
+    id,
+    value,
+    record_date
+from {{ ref('my_seed') }}
+"""
+
+
+MY_MATERIALIZED_VIEW = """
+{{ config(
+    materialized='materialized_view',
+    partition_by={
+        "field": "record_date",
+        "data_type": "datetime",
+        "granularity": "day"
+    },
+    cluster_by="id",
+) }}
+select
+    id,
+    value,
+    record_date
+from {{ ref('my_base_table') }}
+"""
+
+
+MY_OTHER_BASE_TABLE = """
+{{ config(
+    materialized='table',
+    partition_by={
+        "field": "value",
+        "data_type": "int64",
+        "range": {
+            "start": 0,
+            "end": 500,
+            "interval": 50
+        }
+    },
+    cluster_by=["id", "value"]
+) }}
+select
+    id,
+    value,
+    record_date
+from {{ ref('my_seed') }}
+"""
+
+
+MY_OTHER_MATERIALIZED_VIEW = """
+{{ config(
+    materialized='materialized_view',
+    partition_by={
+        "field": "value",
+        "data_type": "int64",
+        "range": {
+            "start": 0,
+            "end": 500,
+            "interval": 50
+        }
+    },
+    cluster_by="id",
+    enable_refresh=False,
+    refresh_interval_minutes=60
+) }}
+select
+    id,
+    value,
+    record_date
+from {{ ref('my_other_base_table') }}
+"""
diff --git a/tests/functional/adapter/describe_relation/test_describe_relation.py b/tests/functional/adapter/describe_relation/test_describe_relation.py
new file mode 100644
index 000000000..adccd5126
--- /dev/null
+++ b/tests/functional/adapter/describe_relation/test_describe_relation.py
@@ -0,0 +1,100 @@
+import pytest
+
+from dbt.adapters.base.relation import BaseRelation
+from dbt.contracts.relation import RelationType
+from dbt.tests.util import get_connection, run_dbt
+
+from dbt.adapters.bigquery.relation_configs import BigQueryMaterializedViewConfig
+from tests.functional.adapter.describe_relation import _files
+
+
+class TestDescribeRelation:
+    @pytest.fixture(scope="class", autouse=True)
+    def seeds(self):
+        return {"my_seed.csv": _files.MY_SEED}
+
+    @pytest.fixture(scope="class", autouse=True)
+    def models(self):
+        yield {
+            "my_base_table.sql": _files.MY_BASE_TABLE,
+            "my_materialized_view.sql": _files.MY_MATERIALIZED_VIEW,
+            "my_other_base_table.sql": _files.MY_OTHER_BASE_TABLE,
+            "my_other_materialized_view.sql": _files.MY_OTHER_MATERIALIZED_VIEW,
+        }
+
+    @pytest.fixture(scope="class")
+    def my_seed(self, project) -> BaseRelation:
+        return project.adapter.Relation.create(
+            identifier="my_seed",
+            schema=project.test_schema,
+            database=project.database,
+            type=RelationType.Table,
+        )
+
+    @pytest.fixture(scope="class")
+    def my_base_table(self, project) -> BaseRelation:
+        """
+        The base table for a materialized view needs to be partitioned in
+        the same way as the materialized view. So if we want to create a partitioned
+        materialized view, we need to partition the base table. This table is a
+        select * on the seed table, plus a partition.
+        """
+        return project.adapter.Relation.create(
+            identifier="my_base_table",
+            schema=project.test_schema,
+            database=project.database,
+            type=RelationType.Table,
+        )
+
+    @pytest.fixture(scope="class")
+    def my_materialized_view(self, project) -> BaseRelation:
+        return project.adapter.Relation.create(
+            identifier="my_materialized_view",
+            schema=project.test_schema,
+            database=project.database,
+            type=RelationType.MaterializedView,
+        )
+
+    @pytest.fixture(scope="class")
+    def my_other_materialized_view(self, project) -> BaseRelation:
+        return project.adapter.Relation.create(
+            identifier="my_other_materialized_view",
+            schema=project.test_schema,
+            database=project.database,
+            type=RelationType.MaterializedView,
+        )
+
+    @pytest.fixture(scope="class", autouse=True)
+    def setup(self, project, my_base_table, my_materialized_view):
+        run_dbt(["seed"])
+        run_dbt(["run"])
+        yield
+        project.run_sql(f"drop schema if exists {project.test_schema} cascade")
+
+    def test_describe_materialized_view(self, project, my_materialized_view):
+        with get_connection(project.adapter):
+            results = project.adapter.describe_relation(my_materialized_view)
+        assert isinstance(results, BigQueryMaterializedViewConfig)
+        assert results.table_id == f'"{my_materialized_view.identifier}"'
+        assert results.dataset_id == f'"{my_materialized_view.schema}"'
+        assert results.project_id == f'"{my_materialized_view.database}"'
+        assert results.partition.field == "record_date"
+        assert results.partition.data_type == "datetime"
+        assert results.partition.granularity == "day"
+        assert results.cluster.fields == frozenset({"id"})
+        assert results.options.enable_refresh is True
+        assert results.options.refresh_interval_minutes == 30
+
+    def test_describe_other_materialized_view(self, project, my_other_materialized_view):
+        with get_connection(project.adapter):
+            results = project.adapter.describe_relation(my_other_materialized_view)
+        assert isinstance(results, BigQueryMaterializedViewConfig)
+        assert results.table_id == f'"{my_other_materialized_view.identifier}"'
+        assert results.dataset_id == f'"{my_other_materialized_view.schema}"'
+        assert results.project_id == f'"{my_other_materialized_view.database}"'
+        assert results.partition.field == "value"
+        assert results.partition.data_type == "int64"
+        assert results.partition.range == {"start": 0, "end": 500, "interval": 50}
+        assert results.cluster.fields == frozenset({"id"})
+        assert results.options.enable_refresh is False
+        assert results.options.refresh_interval_minutes == 30  # BQ returns it to the default
diff --git a/tests/functional/adapter/materialized_view_tests/_files.py b/tests/functional/adapter/materialized_view_tests/_files.py
new file mode 100644
index 000000000..1ee64269d
--- /dev/null
+++ b/tests/functional/adapter/materialized_view_tests/_files.py
@@ -0,0 +1,69 @@
+MY_SEED = """
+id,value,record_valid_date
+1,100,2023-01-01 00:00:00
+2,200,2023-01-02 00:00:00
+3,300,2023-01-02 00:00:00
+""".strip()
+
+
+MY_BASE_TABLE = """
+{{ config(
+    materialized='table',
+    partition_by={
+        "field": "record_valid_date",
+        "data_type": "datetime",
+        "granularity": "day"
+    },
+    cluster_by=["id", "value"]
+) }}
+select
+    id,
+    value,
+    record_valid_date
+from {{ ref('my_seed') }}
+"""
+
+
+# the whitespace to the left on partition matters here
+MY_MATERIALIZED_VIEW = """
+{{ config(
+    materialized='materialized_view',
+    partition_by={
+        "field": "record_valid_date",
+        "data_type": "datetime",
+        "granularity": "day"
+    },
+    cluster_by=["id", "value"],
+    enable_refresh=True,
+    refresh_interval_minutes=60,
+    max_staleness="INTERVAL 45 MINUTE"
+) }}
+select
+    id,
+    value,
+    record_valid_date
+from {{ ref('my_base_table') }}
+"""
+
+
+# the whitespace to the left on partition matters here
+MY_OTHER_BASE_TABLE = """
+{{ config(
+    materialized='table',
+    partition_by={
+        "field": "value",
+        "data_type": "int64",
+        "range": {
+            "start": 0,
+            "end": 500,
+            "interval": 50
+        }
+    },
+    cluster_by=["id", "value"]
+) }}
+select
+    id,
+    value,
+    record_valid_date
+from {{ ref('my_seed') }}
+"""
diff --git a/tests/functional/adapter/materialized_view_tests/_mixin.py b/tests/functional/adapter/materialized_view_tests/_mixin.py
new file mode 100644
index 000000000..5f75c7c04
--- /dev/null
+++ b/tests/functional/adapter/materialized_view_tests/_mixin.py
@@ -0,0 +1,104 @@
+from typing import Optional, Tuple
+
+import pytest
+
+from dbt.adapters.base.relation import BaseRelation
+from dbt.contracts.relation import RelationType
+from dbt.tests.adapter.materialized_view.files import MY_TABLE, MY_VIEW
+from dbt.tests.util import (
+    get_connection,
+    get_model_file,
+    run_dbt,
+    set_model_file,
+)
+
+from tests.functional.adapter.materialized_view_tests import _files
+
+
+class BigQueryMaterializedViewMixin:
+    @pytest.fixture(scope="class")
+    def my_base_table(self, project) -> BaseRelation:
+        """
+        The base table for a materialized view needs to be partitioned in
+        the same way as the materialized view. So if we want to create a partitioned
+        materialized view, we need to partition the base table. This table is a
+        select * on the seed table, plus a partition.
+        """
+        return project.adapter.Relation.create(
+            identifier="my_base_table",
+            schema=project.test_schema,
+            database=project.database,
+            type=RelationType.Table,
+        )
+
+    @pytest.fixture(scope="class")
+    def my_other_base_table(self, project) -> BaseRelation:
+        """
+        Following the sentiment of `my_base_table` above, if we want to alter the partition
+        on the materialized view, we either need to update the partition on the base table,
+        or we need a second table with a different partition.
+        """
+        return project.adapter.Relation.create(
+            identifier="my_other_base_table",
+            schema=project.test_schema,
+            database=project.database,
+            type=RelationType.Table,
+        )
+
+    @pytest.fixture(scope="function", autouse=True)
+    def setup(self, project, my_base_table, my_other_base_table, my_materialized_view):  # type: ignore
+        run_dbt(["seed"])
+        run_dbt(["run", "--full-refresh"])
+
+        # the tests touch these files, store their contents in memory
+        initial_model = get_model_file(project, my_materialized_view)
+
+        yield
+
+        # and then reset them after the test runs
+        set_model_file(project, my_materialized_view, initial_model)
+        project.run_sql(f"drop schema if exists {project.test_schema} cascade")
+
+    @pytest.fixture(scope="class", autouse=True)
+    def seeds(self):
+        return {"my_seed.csv": _files.MY_SEED}
+
+    @pytest.fixture(scope="class", autouse=True)
+    def models(self):
+        yield {
+            "my_table.sql": MY_TABLE,
+            "my_view.sql": MY_VIEW,
+            "my_base_table.sql": _files.MY_BASE_TABLE,
+            "my_other_base_table.sql": _files.MY_OTHER_BASE_TABLE,
+            "my_materialized_view.sql": _files.MY_MATERIALIZED_VIEW,
+        }
+
+    @staticmethod
+    def insert_record(project, table: BaseRelation, record: Tuple[int, int]) -> None:
+        my_id, value = record
+        project.run_sql(f"insert into {table} (id, value) values ({my_id}, {value})")
+
+    @staticmethod
+    def refresh_materialized_view(project, materialized_view: BaseRelation) -> None:
+        sql = f"""
+        call bq.refresh_materialized_view(
+            '{materialized_view.database}.{materialized_view.schema}.{materialized_view.identifier}'
+        )
+        """
+        project.run_sql(sql)
+
+    @staticmethod
+    def query_row_count(project, relation: BaseRelation) -> int:
+        sql = f"select count(*) from {relation}"
+        return project.run_sql(sql, fetch="one")[0]
+
+    # look into syntax
+    @staticmethod
+    def query_relation_type(project, relation: BaseRelation) -> Optional[str]:
+        with get_connection(project.adapter) as conn:
+            table = conn.handle.get_table(
+                project.adapter.connections.get_bq_table(
+                    relation.database, relation.schema, relation.identifier
+                )
+            )
+        return table.table_type.lower()
diff --git a/tests/functional/adapter/materialized_view_tests/test_materialized_view.py b/tests/functional/adapter/materialized_view_tests/test_materialized_view.py
index 03d588cd4..7ca90983e 100644
--- a/tests/functional/adapter/materialized_view_tests/test_materialized_view.py
+++ b/tests/functional/adapter/materialized_view_tests/test_materialized_view.py
@@ -1,61 +1,12 @@
-from typing import Optional, Tuple
-
 import pytest
 
-from dbt.adapters.base.relation import BaseRelation
-from dbt.tests.util import get_connection, run_dbt
+from dbt.tests.util import run_dbt
 from dbt.tests.adapter.materialized_view.basic import MaterializedViewBasic
 
-from dbt.tests.adapter.materialized_view.files import MY_TABLE, MY_VIEW
-
-
-MY_MATERIALIZED_VIEW = """
-{{ config(
-    materialized='materialized_view'
-) }}
-select * from {{ ref('my_seed') }}
-"""
-
-
-class TestBigqueryMaterializedViewsBasic(MaterializedViewBasic):
-    @pytest.fixture(scope="class", autouse=True)
-    def models(self):
-        yield {
-            "my_table.sql": MY_TABLE,
-            "my_view.sql": MY_VIEW,
-            "my_materialized_view.sql": MY_MATERIALIZED_VIEW,
-        }
-
-    @staticmethod
-    def insert_record(project, table: BaseRelation, record: Tuple[int, int]):
-        my_id, value = record
-        project.run_sql(f"insert into {table} (id, value) values ({my_id}, {value})")
-
-    @staticmethod
-    def refresh_materialized_view(project, materialized_view: BaseRelation):
-        sql = f"""
-        call bq.refresh_materialized_view(
-            '{materialized_view.database}.{materialized_view.schema}.{materialized_view.identifier}'
-        )
-        """
-        project.run_sql(sql)
-
-    @staticmethod
-    def query_row_count(project, relation: BaseRelation) -> int:
-        sql = f"select count(*) from {relation}"
-        return project.run_sql(sql, fetch="one")[0]
+from tests.functional.adapter.materialized_view_tests._mixin import BigQueryMaterializedViewMixin
 
-    # look into syntax
-    @staticmethod
-    def query_relation_type(project, relation: BaseRelation) -> Optional[str]:
-        with get_connection(project.adapter) as conn:
-            table = conn.handle.get_table(
-                project.adapter.connections.get_bq_table(
-                    relation.database, relation.schema, relation.identifier
-                )
-            )
-        return table.table_type.lower()
 
+class TestBigqueryMaterializedViewsBasic(BigQueryMaterializedViewMixin, MaterializedViewBasic):
     def test_view_replaces_materialized_view(self, project, my_materialized_view):
         """
         We don't support replacing a view with another object in dbt-bigquery unless we use --full-refresh
@@ -65,13 +16,12 @@ def test_view_replaces_materialized_view(self, project, my_materialized_view):
 
         self.swap_materialized_view_to_view(project, my_materialized_view)
 
-        run_dbt(
-            ["run", "--models", my_materialized_view.identifier, "--full-refresh"]
-        )  # add --full-refresh
+        # add --full-refresh
+        run_dbt(["run", "--models", my_materialized_view.identifier, "--full-refresh"])
         assert self.query_relation_type(project, my_materialized_view) == "view"
 
     @pytest.mark.skip(
-        "It appears BQ updates the materialized view almost immediately, which fails this test."
+        "It looks like BQ updates the materialized view almost immediately, which fails this test."
     )
     def test_materialized_view_only_updates_after_refresh(
         self, project, my_materialized_view, my_seed
diff --git a/tests/functional/adapter/materialized_view_tests/test_materialized_view_changes.py b/tests/functional/adapter/materialized_view_tests/test_materialized_view_changes.py
new file mode 100644
index 000000000..c821c68fc
--- /dev/null
+++ b/tests/functional/adapter/materialized_view_tests/test_materialized_view_changes.py
@@ -0,0 +1,95 @@
+from dbt.tests.adapter.materialized_view.changes import (
+    MaterializedViewChanges,
+    MaterializedViewChangesApplyMixin,
+    MaterializedViewChangesContinueMixin,
+    MaterializedViewChangesFailMixin,
+)
+from dbt.tests.util import get_connection, get_model_file, set_model_file
+
+from dbt.adapters.bigquery.relation_configs import BigQueryMaterializedViewConfig
+
+from tests.functional.adapter.materialized_view_tests._mixin import BigQueryMaterializedViewMixin
+
+
+class BigQueryMaterializedViewChanges(BigQueryMaterializedViewMixin, MaterializedViewChanges):
+    @staticmethod
+    def check_start_state(project, materialized_view):
+        with get_connection(project.adapter):
+            results = project.adapter.describe_relation(materialized_view)
+        assert isinstance(results, BigQueryMaterializedViewConfig)
+        assert results.options.enable_refresh is True
+        assert results.options.refresh_interval_minutes == 60
+        assert results.partition.field == "record_valid_date"
+        assert results.partition.data_type == "datetime"
+        assert results.partition.granularity == "day"
+        assert results.cluster.fields == frozenset({"id", "value"})
+
+    @staticmethod
+    def change_config_via_alter(project, materialized_view):
+        initial_model = get_model_file(project, materialized_view)
+        new_model = initial_model.replace("enable_refresh=True", "enable_refresh=False")
+        set_model_file(project, materialized_view, new_model)
+
+    @staticmethod
+    def check_state_alter_change_is_applied(project, materialized_view):
+        with get_connection(project.adapter):
+            results = project.adapter.describe_relation(materialized_view)
+        assert isinstance(results, BigQueryMaterializedViewConfig)
+        # these change when run manually
+        assert results.options.enable_refresh is False
+        assert results.options.refresh_interval_minutes == 30  # BQ returns it to the default
+
+    @staticmethod
+    def change_config_via_replace(project, materialized_view):
+        initial_model = get_model_file(project, materialized_view)
+        # the whitespace to the left on partition matters here
+        old_partition = """
+    partition_by={
+        "field": "record_valid_date",
+        "data_type": "datetime",
+        "granularity": "day"
+    },"""
+        new_partition = """
+    partition_by={
+        "field": "value",
+        "data_type": "int64",
+        "range": {
+            "start": 0,
+            "end": 500,
+            "interval": 50
+        }
+    },"""
+        new_model = (
+            initial_model.replace(old_partition, new_partition)
+            .replace("'my_base_table'", "'my_other_base_table'")
+            .replace('cluster_by=["id", "value"]', 'cluster_by="id"')
+        )
+        set_model_file(project, materialized_view, new_model)
+
+    @staticmethod
+    def check_state_replace_change_is_applied(project, materialized_view):
+        with get_connection(project.adapter):
+            results = project.adapter.describe_relation(materialized_view)
+        assert isinstance(results, BigQueryMaterializedViewConfig)
+        assert results.partition.field == "value"
+        assert results.partition.data_type == "int64"
+        assert results.partition.range == {"start": 0, "end": 500, "interval": 50}
+        assert results.cluster.fields == frozenset({"id"})
+
+
+class TestBigQueryMaterializedViewChangesApply(
+    BigQueryMaterializedViewChanges, MaterializedViewChangesApplyMixin
+):
+    pass
+
+
+class TestBigQueryMaterializedViewChangesContinue(
+    BigQueryMaterializedViewChanges, MaterializedViewChangesContinueMixin
+):
+    pass
+
+
+class TestBigQueryMaterializedViewChangesFail(
+    BigQueryMaterializedViewChanges, MaterializedViewChangesFailMixin
+):
+    pass
diff --git a/tests/functional/adapter/materialized_view_tests/test_materialized_view_cluster_changes.py b/tests/functional/adapter/materialized_view_tests/test_materialized_view_cluster_changes.py
new file mode 100644
index 000000000..74e174d4f
--- /dev/null
+++ b/tests/functional/adapter/materialized_view_tests/test_materialized_view_cluster_changes.py
@@ -0,0 +1,69 @@
+from dbt.tests.adapter.materialized_view.changes import (
+    MaterializedViewChanges,
+    MaterializedViewChangesApplyMixin,
+    MaterializedViewChangesContinueMixin,
+    MaterializedViewChangesFailMixin,
+)
+from dbt.tests.util import get_connection, get_model_file, set_model_file
+
+from dbt.adapters.bigquery.relation_configs import BigQueryMaterializedViewConfig
+
+from tests.functional.adapter.materialized_view_tests._mixin import BigQueryMaterializedViewMixin
+
+
+class BigQueryMaterializedViewClusterChanges(
+    BigQueryMaterializedViewMixin, MaterializedViewChanges
+):
+    @staticmethod
+    def check_start_state(project, materialized_view):
+        with get_connection(project.adapter):
+            results = project.adapter.describe_relation(materialized_view)
+        assert isinstance(results, BigQueryMaterializedViewConfig)
+        assert results.options.enable_refresh is True
+        assert results.options.refresh_interval_minutes == 60
+        assert results.cluster.fields == frozenset({"id", "value"})
+
+    @staticmethod
+    def change_config_via_alter(project, materialized_view):
+        initial_model = get_model_file(project, materialized_view)
+        new_model = initial_model.replace("enable_refresh=True", "enable_refresh=False")
+        set_model_file(project, materialized_view, new_model)
+
+    @staticmethod
+    def check_state_alter_change_is_applied(project, materialized_view):
+        with get_connection(project.adapter):
+            results = project.adapter.describe_relation(materialized_view)
+        assert isinstance(results, BigQueryMaterializedViewConfig)
+        assert results.options.enable_refresh is False
+        assert results.options.refresh_interval_minutes == 30  # BQ returns it to the default
+
+    @staticmethod
+    def change_config_via_replace(project, materialized_view):
+        initial_model = get_model_file(project, materialized_view)
+        new_model = initial_model.replace('cluster_by=["id", "value"]', 'cluster_by="id"')
+        set_model_file(project, materialized_view, new_model)
+
+    @staticmethod
+    def check_state_replace_change_is_applied(project, materialized_view):
+        with get_connection(project.adapter):
+            results = project.adapter.describe_relation(materialized_view)
+        assert isinstance(results, BigQueryMaterializedViewConfig)
+        assert results.cluster.fields == frozenset({"id"})
+
+
+class TestBigQueryMaterializedViewClusterChangesApply(
+    BigQueryMaterializedViewClusterChanges, MaterializedViewChangesApplyMixin
+):
+    pass
+
+
+class TestBigQueryMaterializedViewClusterChangesContinue(
+    BigQueryMaterializedViewClusterChanges, MaterializedViewChangesContinueMixin
+):
+    pass
+
+
+class TestBigQueryMaterializedViewClusterChangesFail(
+    BigQueryMaterializedViewClusterChanges, MaterializedViewChangesFailMixin
+):
+    pass
diff --git a/tests/functional/adapter/materialized_view_tests/test_materialized_view_partition_changes.py b/tests/functional/adapter/materialized_view_tests/test_materialized_view_partition_changes.py
new file mode 100644
index 000000000..7f396ae1b
--- /dev/null
+++ b/tests/functional/adapter/materialized_view_tests/test_materialized_view_partition_changes.py
@@ -0,0 +1,93 @@
+from dbt.tests.adapter.materialized_view.changes import (
+    MaterializedViewChanges,
+    MaterializedViewChangesApplyMixin,
+    MaterializedViewChangesContinueMixin,
+    MaterializedViewChangesFailMixin,
+)
+from dbt.tests.util import get_connection, get_model_file, set_model_file
+
+from dbt.adapters.bigquery.relation_configs import BigQueryMaterializedViewConfig
+
+from tests.functional.adapter.materialized_view_tests._mixin import BigQueryMaterializedViewMixin
+
+
+class BigQueryMaterializedViewPartitionChanges(
+    BigQueryMaterializedViewMixin, MaterializedViewChanges
+):
+    @staticmethod
+    def check_start_state(project, materialized_view):
+        with get_connection(project.adapter):
+            results = project.adapter.describe_relation(materialized_view)
+        assert isinstance(results, BigQueryMaterializedViewConfig)
+        assert results.options.enable_refresh is True
+        assert results.options.refresh_interval_minutes == 60
+        assert results.partition.field == "record_valid_date"
+        assert results.partition.data_type == "datetime"
+        assert results.partition.granularity == "day"
+
+    @staticmethod
+    def change_config_via_alter(project, materialized_view):
+        initial_model = get_model_file(project, materialized_view)
+        new_model = initial_model.replace("enable_refresh=True", "enable_refresh=False")
+        set_model_file(project, materialized_view, new_model)
+
+    @staticmethod
+    def check_state_alter_change_is_applied(project, materialized_view):
+        with get_connection(project.adapter):
+            results = project.adapter.describe_relation(materialized_view)
+        assert isinstance(results, BigQueryMaterializedViewConfig)
+        # these change when run manually
+        assert results.options.enable_refresh is False
+        assert results.options.refresh_interval_minutes == 30  # BQ returns it to the default
+
+    @staticmethod
+    def change_config_via_replace(project, materialized_view):
+        initial_model = get_model_file(project, materialized_view)
+        # the whitespace to the left on partition matters here
+        old_partition = """
+    partition_by={
+        "field": "record_valid_date",
+        "data_type": "datetime",
+        "granularity": "day"
+    },"""
+        new_partition = """
+    partition_by={
+        "field": "value",
+        "data_type": "int64",
+        "range": {
+            "start": 0,
+            "end": 500,
+            "interval": 50
+        }
+    },"""
+        new_model = initial_model.replace(old_partition, new_partition).replace(
+            "'my_base_table'", "'my_other_base_table'"
+        )
+        set_model_file(project, materialized_view, new_model)
+
+    @staticmethod
+    def check_state_replace_change_is_applied(project, materialized_view):
+        with get_connection(project.adapter):
+            results = project.adapter.describe_relation(materialized_view)
+        assert isinstance(results, BigQueryMaterializedViewConfig)
+        assert results.partition.field == "value"
+        assert results.partition.data_type == "int64"
+        assert results.partition.range == {"start": 0, "end": 500, "interval": 50}
+
+
+class TestBigQueryMaterializedViewPartitionChangesApply(
+    BigQueryMaterializedViewPartitionChanges, MaterializedViewChangesApplyMixin
+):
+    pass
+
+
+class TestBigQueryMaterializedViewPartitionChangesContinue(
+    BigQueryMaterializedViewPartitionChanges, MaterializedViewChangesContinueMixin
+):
+    pass
+
+
+class TestBigQueryMaterializedViewPartitionChangesFail(
+    BigQueryMaterializedViewPartitionChanges, MaterializedViewChangesFailMixin
+):
+    pass
diff --git a/tests/functional/adapter/materialized_view_tests/utils.py b/tests/functional/adapter/materialized_view_tests/utils.py
deleted file mode 100644
index 25e40d337..000000000
--- a/tests/functional/adapter/materialized_view_tests/utils.py
+++ /dev/null
@@ -1,13 +0,0 @@
-from dbt.adapters.bigquery.relation import BigQueryRelation
-
-
-def query_autorefresh(project, relation: BigQueryRelation) -> bool:
-    sql = f"""
-        select
-            case mv.autorefresh when 't' then True when 'f' then False end as autorefresh
-        from stv_mv_info mv
-        where trim(mv.name) ilike '{ relation.identifier }'
-        and trim(mv.schema) ilike '{ relation.schema }'
-        and trim(mv.db_name) ilike '{ relation.database }'
-    """
-    return project.run_sql(sql, fetch="one")[0]
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 4db2ce83d..926547e10 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -8,10 +8,8 @@
 from unittest.mock import patch, MagicMock, create_autospec
 
 import dbt.dataclass_schema
-
-from dbt.adapters.bigquery import PartitionConfig
-from dbt.adapters.bigquery import BigQueryAdapter
-from dbt.adapters.bigquery import BigQueryRelation
+from dbt.adapters.bigquery.relation_configs import PartitionConfig
+from dbt.adapters.bigquery import BigQueryAdapter, BigQueryRelation
 from dbt.adapters.bigquery import Plugin as BigQueryPlugin
 from google.cloud.bigquery.table import Table
 from dbt.adapters.bigquery.connections import _sanitize_label, _VALIDATE_LABEL_LENGTH_LIMIT

From f4ab3f2d27afb783a3cded7b3fa144a8ef5f6360 Mon Sep 17 00:00:00 2001
From: FishtownBuildBot <77737458+FishtownBuildBot@users.noreply.github.com>
Date: Thu, 12 Oct 2023 12:16:10 -0400
Subject: [PATCH 728/860] Cleanup main after cutting new 1.7.latest branch
 (#966)

* Clean up changelog on main

* Bumping version to 1.8.0a1

* Code quality cleanup
---
 .bumpversion.cfg                              |  2 +-
 .changes/1.7.0-b1.md                          | 36 ----------
 .changes/1.7.0-b2.md                          | 28 --------
 .../1.7.0/Dependencies-20230711-002411.yaml   |  6 --
 .../1.7.0/Dependencies-20230717-000639.yaml   |  6 --
 .../1.7.0/Dependencies-20230721-010106.yaml   |  6 --
 .../1.7.0/Dependencies-20230724-004957.yaml   |  6 --
 .../1.7.0/Dependencies-20230731-004314.yaml   |  6 --
 .../1.7.0/Dependencies-20230803-131551.yaml   |  6 --
 .../1.7.0/Dependencies-20230809-005844.yaml   |  6 --
 .../1.7.0/Dependencies-20230809-034524.yaml   |  6 --
 .../1.7.0/Dependencies-20230810-002407.yaml   |  6 --
 .../1.7.0/Dependencies-20230811-010246.yaml   |  6 --
 .../1.7.0/Dependencies-20230814-005415.yaml   |  6 --
 .../1.7.0/Dependencies-20230814-005542.yaml   |  6 --
 .../1.7.0/Dependencies-20230817-003623.yaml   |  6 --
 .../1.7.0/Dependencies-20230817-003629.yaml   |  6 --
 .../1.7.0/Dependencies-20230901-225226.yaml   |  6 --
 .../1.7.0/Dependencies-20230904-011145.yaml   |  6 --
 .../1.7.0/Dependencies-20230911-010957.yaml   |  6 --
 .changes/1.7.0/Features-20230330-165721.yaml  |  6 --
 .changes/1.7.0/Fixes-20230630-092618.yaml     |  6 --
 .changes/1.7.0/Fixes-20230727-213526.yaml     |  6 --
 .changes/1.7.0/Fixes-20230803-102915.yaml     |  6 --
 .changes/1.7.0/Fixes-20230804-005817.yaml     |  6 --
 .changes/1.7.0/Fixes-20230818-214616.yaml     |  6 --
 .changes/1.7.0/Fixes-20230829-162111.yaml     |  6 --
 .changes/1.7.0/Fixes-20230907-161347.yaml     |  6 --
 .changes/1.7.0/Fixes-20230911-005506.yaml     |  7 --
 .../1.7.0/Under the Hood-20230724-164946.yaml |  6 --
 .../1.7.0/Under the Hood-20230829-122918.yaml |  6 --
 .../1.7.0/Under the Hood-20230830-160707.yaml |  6 --
 .../Dependencies-20230919-003620.yaml         |  6 --
 .../unreleased/Features-20230426-152526.yaml  |  7 --
 .../unreleased/Features-20230913-130445.yaml  |  6 --
 .../unreleased/Features-20230921-175106.yaml  |  6 --
 .../unreleased/Fixes-20230721-101041.yaml     |  6 --
 .../unreleased/Fixes-20230817-095527.yaml     |  6 --
 .../unreleased/Fixes-20230906-141819.yaml     |  6 --
 .../unreleased/Fixes-20231005-235950.yaml     |  6 --
 .../Under the Hood-20230921-155645.yaml       |  6 --
 .../Under the Hood-20230922-114217.yaml       |  6 --
 .../Under the Hood-20230922-125327.yaml       |  6 --
 .../Under the Hood-20230925-143628.yaml       |  6 --
 .../Under the Hood-20231005-115800.yaml       |  6 --
 CHANGELOG.md                                  | 67 -------------------
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 48 files changed, 3 insertions(+), 388 deletions(-)
 delete mode 100644 .changes/1.7.0-b1.md
 delete mode 100644 .changes/1.7.0-b2.md
 delete mode 100644 .changes/1.7.0/Dependencies-20230711-002411.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230717-000639.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230721-010106.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230724-004957.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230731-004314.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230803-131551.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230809-005844.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230809-034524.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230810-002407.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230811-010246.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230814-005415.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230814-005542.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230817-003623.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230817-003629.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230901-225226.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230904-011145.yaml
 delete mode 100644 .changes/1.7.0/Dependencies-20230911-010957.yaml
 delete mode 100644 .changes/1.7.0/Features-20230330-165721.yaml
 delete mode 100644 .changes/1.7.0/Fixes-20230630-092618.yaml
 delete mode 100644 .changes/1.7.0/Fixes-20230727-213526.yaml
 delete mode 100644 .changes/1.7.0/Fixes-20230803-102915.yaml
 delete mode 100644 .changes/1.7.0/Fixes-20230804-005817.yaml
 delete mode 100644 .changes/1.7.0/Fixes-20230818-214616.yaml
 delete mode 100644 .changes/1.7.0/Fixes-20230829-162111.yaml
 delete mode 100644 .changes/1.7.0/Fixes-20230907-161347.yaml
 delete mode 100644 .changes/1.7.0/Fixes-20230911-005506.yaml
 delete mode 100644 .changes/1.7.0/Under the Hood-20230724-164946.yaml
 delete mode 100644 .changes/1.7.0/Under the Hood-20230829-122918.yaml
 delete mode 100644 .changes/1.7.0/Under the Hood-20230830-160707.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20230919-003620.yaml
 delete mode 100644 .changes/unreleased/Features-20230426-152526.yaml
 delete mode 100644 .changes/unreleased/Features-20230913-130445.yaml
 delete mode 100644 .changes/unreleased/Features-20230921-175106.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230721-101041.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230817-095527.yaml
 delete mode 100644 .changes/unreleased/Fixes-20230906-141819.yaml
 delete mode 100644 .changes/unreleased/Fixes-20231005-235950.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20230921-155645.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20230922-114217.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20230922-125327.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20230925-143628.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20231005-115800.yaml

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 8693b3160..e74c22bc8 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.7.0b2
+current_version = 1.8.0a1
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.7.0-b1.md b/.changes/1.7.0-b1.md
deleted file mode 100644
index 13dd4982d..000000000
--- a/.changes/1.7.0-b1.md
+++ /dev/null
@@ -1,36 +0,0 @@
-## dbt-bigquery 1.7.0-b1 - August 17, 2023
-
-### Features
-
-- Change batch_id to model override ([#671](https://github.com/dbt-labs/dbt-bigquery/issues/671))
-
-### Fixes
-
-- Avoid creating twice the temp table in dynamic insert overwrite for sql on_schema_change != 'ignore' or python models ([#810](https://github.com/dbt-labs/dbt-bigquery/issues/810))
-- Requiring Python >=3.8 enables walrus operator ([#849](https://github.com/dbt-labs/dbt-bigquery/issues/849))
-- Fix contact validation failure on query data types change ([#861](https://github.com/dbt-labs/dbt-bigquery/issues/861))
-
-### Under the Hood
-
-- Update stale workflow to use centralized version ([#844](https://github.com/dbt-labs/dbt-bigquery/issues/844))
-
-### Dependencies
-
-- Update black requirement from ~=23.3 to ~=23.7 ([#815](https://github.com/dbt-labs/dbt-bigquery/pull/815))
-- Update pip-tools requirement from ~=6.14 to ~=7.0 ([#831](https://github.com/dbt-labs/dbt-bigquery/pull/831))
-- Update ddtrace requirement from ~=1.16 to ~=1.17 ([#839](https://github.com/dbt-labs/dbt-bigquery/pull/839))
-- Update wheel requirement from ~=0.40 to ~=0.41 ([#843](https://github.com/dbt-labs/dbt-bigquery/pull/843))
-- Update flake8 requirement from ~=6.0 to ~=6.1 ([#847](https://github.com/dbt-labs/dbt-bigquery/pull/847))
-- Update pip-tools requirement from ~=7.0 to ~=7.2 ([#851](https://github.com/dbt-labs/dbt-bigquery/pull/851))
-- Update pip-tools requirement from ~=7.2 to ~=7.3 ([#870](https://github.com/dbt-labs/dbt-bigquery/pull/870))
-- Update tox requirement from ~=4.6 to ~=4.7 ([#869](https://github.com/dbt-labs/dbt-bigquery/pull/869))
-- Update types-protobuf requirement from ~=4.23 to ~=4.24 ([#873](https://github.com/dbt-labs/dbt-bigquery/pull/873))
-- Bump mypy from 1.4.1 to 1.5.0 ([#874](https://github.com/dbt-labs/dbt-bigquery/pull/874))
-- Update tox requirement from ~=4.7 to ~=4.8 ([#880](https://github.com/dbt-labs/dbt-bigquery/pull/880))
-- Update ddtrace requirement from ~=1.17 to ~=1.18 ([#881](https://github.com/dbt-labs/dbt-bigquery/pull/881))
-
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#810](https://github.com/dbt-labs/dbt-bigquery/issues/810))
-- [@cemsbr](https://github.com/cemsbr) ([#849](https://github.com/dbt-labs/dbt-bigquery/issues/849))
-- [@dementiev27](https://github.com/dementiev27) ([#861](https://github.com/dbt-labs/dbt-bigquery/issues/861))
-- [@nickozilla](https://github.com/nickozilla) ([#671](https://github.com/dbt-labs/dbt-bigquery/issues/671))
diff --git a/.changes/1.7.0-b2.md b/.changes/1.7.0-b2.md
deleted file mode 100644
index 0dd5d798b..000000000
--- a/.changes/1.7.0-b2.md
+++ /dev/null
@@ -1,28 +0,0 @@
-## dbt-bigquery 1.7.0-b2 - September 15, 2023
-
-### Features
-
-- Add support for materialized views ([#dbt-labs/dbt-core#6911](https://github.com/dbt-labs/dbt-bigquery/issues/dbt-labs/dbt-core#6911))
-
-### Fixes
-
-- case insensitive check on partition matching ([#886](https://github.com/dbt-labs/dbt-bigquery/issues/886))
-- Time out queries if user supplies `job_execution_timeout` ([#231](https://github.com/dbt-labs/dbt-bigquery/issues/231))
-- changes expected value types to AnyInteger to take into account changes in core ([#915](https://github.com/dbt-labs/dbt-bigquery/issues/915))
-- Quote column names in struct data types to fix time ingestion partitioning table creation ([#913](https://github.com/dbt-labs/dbt-bigquery/issues/913))
-
-### Under the Hood
-
-- Re-organize adapters.sql into more granular files inside of macros/relations ([#904](https://github.com/dbt-labs/dbt-bigquery/issues/904))
-- Update BigQueryCredentials to support migration off hologram ([#906](https://github.com/dbt-labs/dbt-bigquery/issues/906))
-
-### Dependencies
-
-- Bump mypy from 1.5.0 to 1.5.1 ([#883](https://github.com/dbt-labs/dbt-bigquery/pull/883))
-- Update tox requirement from ~=4.8 to ~=4.9 ([#884](https://github.com/dbt-labs/dbt-bigquery/pull/884))
-- Update tox requirement from ~=4.9 to ~=4.11 ([#907](https://github.com/dbt-labs/dbt-bigquery/pull/907))
-- Update pre-commit requirement from ~=3.3 to ~=3.4 ([#909](https://github.com/dbt-labs/dbt-bigquery/pull/909))
-- Update black requirement from ~=23.7 to ~=23.9 ([#920](https://github.com/dbt-labs/dbt-bigquery/pull/920))
-
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#886](https://github.com/dbt-labs/dbt-bigquery/issues/886), [#913](https://github.com/dbt-labs/dbt-bigquery/issues/913))
diff --git a/.changes/1.7.0/Dependencies-20230711-002411.yaml b/.changes/1.7.0/Dependencies-20230711-002411.yaml
deleted file mode 100644
index f19b4f8e9..000000000
--- a/.changes/1.7.0/Dependencies-20230711-002411.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update black requirement from ~=23.3 to ~=23.7"
-time: 2023-07-11T00:24:11.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 815
diff --git a/.changes/1.7.0/Dependencies-20230717-000639.yaml b/.changes/1.7.0/Dependencies-20230717-000639.yaml
deleted file mode 100644
index 81bead79c..000000000
--- a/.changes/1.7.0/Dependencies-20230717-000639.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pip-tools requirement from ~=6.14 to ~=7.0"
-time: 2023-07-17T00:06:39.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 831
diff --git a/.changes/1.7.0/Dependencies-20230721-010106.yaml b/.changes/1.7.0/Dependencies-20230721-010106.yaml
deleted file mode 100644
index 266f491c5..000000000
--- a/.changes/1.7.0/Dependencies-20230721-010106.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update ddtrace requirement from ~=1.16 to ~=1.17"
-time: 2023-07-21T01:01:06.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 839
diff --git a/.changes/1.7.0/Dependencies-20230724-004957.yaml b/.changes/1.7.0/Dependencies-20230724-004957.yaml
deleted file mode 100644
index ee65265c5..000000000
--- a/.changes/1.7.0/Dependencies-20230724-004957.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update wheel requirement from ~=0.40 to ~=0.41"
-time: 2023-07-24T00:49:57.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 843
diff --git a/.changes/1.7.0/Dependencies-20230731-004314.yaml b/.changes/1.7.0/Dependencies-20230731-004314.yaml
deleted file mode 100644
index 68e8e9b8d..000000000
--- a/.changes/1.7.0/Dependencies-20230731-004314.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update flake8 requirement from ~=6.0 to ~=6.1"
-time: 2023-07-31T00:43:14.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 847
diff --git a/.changes/1.7.0/Dependencies-20230803-131551.yaml b/.changes/1.7.0/Dependencies-20230803-131551.yaml
deleted file mode 100644
index 0c0245168..000000000
--- a/.changes/1.7.0/Dependencies-20230803-131551.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pip-tools requirement from ~=7.0 to ~=7.2"
-time: 2023-08-03T13:15:51.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 851
diff --git a/.changes/1.7.0/Dependencies-20230809-005844.yaml b/.changes/1.7.0/Dependencies-20230809-005844.yaml
deleted file mode 100644
index bd2edc1cd..000000000
--- a/.changes/1.7.0/Dependencies-20230809-005844.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pip-tools requirement from ~=7.2 to ~=7.3"
-time: 2023-08-09T00:58:44.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 870
diff --git a/.changes/1.7.0/Dependencies-20230809-034524.yaml b/.changes/1.7.0/Dependencies-20230809-034524.yaml
deleted file mode 100644
index 390c39540..000000000
--- a/.changes/1.7.0/Dependencies-20230809-034524.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update tox requirement from ~=4.6 to ~=4.7"
-time: 2023-08-09T03:45:24.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 869
diff --git a/.changes/1.7.0/Dependencies-20230810-002407.yaml b/.changes/1.7.0/Dependencies-20230810-002407.yaml
deleted file mode 100644
index d89aaca4f..000000000
--- a/.changes/1.7.0/Dependencies-20230810-002407.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update types-protobuf requirement from ~=4.23 to ~=4.24"
-time: 2023-08-10T00:24:07.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 873
diff --git a/.changes/1.7.0/Dependencies-20230811-010246.yaml b/.changes/1.7.0/Dependencies-20230811-010246.yaml
deleted file mode 100644
index bac65f14e..000000000
--- a/.changes/1.7.0/Dependencies-20230811-010246.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump mypy from 1.4.1 to 1.5.0"
-time: 2023-08-11T01:02:46.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 874
diff --git a/.changes/1.7.0/Dependencies-20230814-005415.yaml b/.changes/1.7.0/Dependencies-20230814-005415.yaml
deleted file mode 100644
index f1722073f..000000000
--- a/.changes/1.7.0/Dependencies-20230814-005415.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update tox requirement from ~=4.7 to ~=4.8"
-time: 2023-08-14T00:54:15.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 880
diff --git a/.changes/1.7.0/Dependencies-20230814-005542.yaml b/.changes/1.7.0/Dependencies-20230814-005542.yaml
deleted file mode 100644
index 7acf66fca..000000000
--- a/.changes/1.7.0/Dependencies-20230814-005542.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update ddtrace requirement from ~=1.17 to ~=1.18"
-time: 2023-08-14T00:55:42.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 881
diff --git a/.changes/1.7.0/Dependencies-20230817-003623.yaml b/.changes/1.7.0/Dependencies-20230817-003623.yaml
deleted file mode 100644
index 75723f321..000000000
--- a/.changes/1.7.0/Dependencies-20230817-003623.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump mypy from 1.5.0 to 1.5.1"
-time: 2023-08-17T00:36:23.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 883
diff --git a/.changes/1.7.0/Dependencies-20230817-003629.yaml b/.changes/1.7.0/Dependencies-20230817-003629.yaml
deleted file mode 100644
index 5bd27095f..000000000
--- a/.changes/1.7.0/Dependencies-20230817-003629.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update tox requirement from ~=4.8 to ~=4.9"
-time: 2023-08-17T00:36:29.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 884
diff --git a/.changes/1.7.0/Dependencies-20230901-225226.yaml b/.changes/1.7.0/Dependencies-20230901-225226.yaml
deleted file mode 100644
index 794c94ef7..000000000
--- a/.changes/1.7.0/Dependencies-20230901-225226.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update tox requirement from ~=4.9 to ~=4.11"
-time: 2023-09-01T22:52:26.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 907
diff --git a/.changes/1.7.0/Dependencies-20230904-011145.yaml b/.changes/1.7.0/Dependencies-20230904-011145.yaml
deleted file mode 100644
index e94fa0ea7..000000000
--- a/.changes/1.7.0/Dependencies-20230904-011145.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pre-commit requirement from ~=3.3 to ~=3.4"
-time: 2023-09-04T01:11:45.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 909
diff --git a/.changes/1.7.0/Dependencies-20230911-010957.yaml b/.changes/1.7.0/Dependencies-20230911-010957.yaml
deleted file mode 100644
index e07dd91b1..000000000
--- a/.changes/1.7.0/Dependencies-20230911-010957.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update black requirement from ~=23.7 to ~=23.9"
-time: 2023-09-11T01:09:57.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 920
diff --git a/.changes/1.7.0/Features-20230330-165721.yaml b/.changes/1.7.0/Features-20230330-165721.yaml
deleted file mode 100644
index 02ce58447..000000000
--- a/.changes/1.7.0/Features-20230330-165721.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Add support for materialized views
-time: 2023-03-30T16:57:21.858512-04:00
-custom:
-  Author: mikealfare McKnight-42
-  Issue: dbt-labs/dbt-core#6911
diff --git a/.changes/1.7.0/Fixes-20230630-092618.yaml b/.changes/1.7.0/Fixes-20230630-092618.yaml
deleted file mode 100644
index 16f34bec0..000000000
--- a/.changes/1.7.0/Fixes-20230630-092618.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Change batch_id to model override
-time: 2023-06-30T09:26:18.854492+01:00
-custom:
-  Author: nickozilla
-  Issue: "671"
diff --git a/.changes/1.7.0/Fixes-20230727-213526.yaml b/.changes/1.7.0/Fixes-20230727-213526.yaml
deleted file mode 100644
index ced5b1350..000000000
--- a/.changes/1.7.0/Fixes-20230727-213526.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Avoid creating twice the temp table in dynamic insert overwrite for sql on_schema_change != 'ignore' or python models
-time: 2023-07-27T21:35:26.531164+02:00
-custom:
-  Author: Kayrnt
-  Issue: "810"
diff --git a/.changes/1.7.0/Fixes-20230803-102915.yaml b/.changes/1.7.0/Fixes-20230803-102915.yaml
deleted file mode 100644
index 6070fb448..000000000
--- a/.changes/1.7.0/Fixes-20230803-102915.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Requiring Python >=3.8 enables walrus operator
-time: 2023-08-03T10:29:15.371798-06:00
-custom:
-  Author: cemsbr dbeatty10
-  Issue: "849"
diff --git a/.changes/1.7.0/Fixes-20230804-005817.yaml b/.changes/1.7.0/Fixes-20230804-005817.yaml
deleted file mode 100644
index 34e0601e1..000000000
--- a/.changes/1.7.0/Fixes-20230804-005817.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Fix contact validation failure on query data types change
-time: 2023-08-04T00:58:17.97704+03:00
-custom:
-  Author: dementiev27
-  Issue: "861"
diff --git a/.changes/1.7.0/Fixes-20230818-214616.yaml b/.changes/1.7.0/Fixes-20230818-214616.yaml
deleted file mode 100644
index a66059e21..000000000
--- a/.changes/1.7.0/Fixes-20230818-214616.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: case insensitive check on partition matching
-time: 2023-08-18T21:46:16.828488+02:00
-custom:
-  Author: Kayrnt
-  Issue: "886"
diff --git a/.changes/1.7.0/Fixes-20230829-162111.yaml b/.changes/1.7.0/Fixes-20230829-162111.yaml
deleted file mode 100644
index 5d34acd3e..000000000
--- a/.changes/1.7.0/Fixes-20230829-162111.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Time out queries if user supplies `job_execution_timeout`
-time: 2023-08-29T16:21:11.69291-07:00
-custom:
-  Author: colin-rogers-dbt McKnight-42
-  Issue: "231"
diff --git a/.changes/1.7.0/Fixes-20230907-161347.yaml b/.changes/1.7.0/Fixes-20230907-161347.yaml
deleted file mode 100644
index b0309afc2..000000000
--- a/.changes/1.7.0/Fixes-20230907-161347.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: changes expected value types to AnyInteger to take into account changes in core
-time: 2023-09-07T16:13:47.005796-05:00
-custom:
-  Author: McKnight-42
-  Issue: "915"
diff --git a/.changes/1.7.0/Fixes-20230911-005506.yaml b/.changes/1.7.0/Fixes-20230911-005506.yaml
deleted file mode 100644
index 16f15f846..000000000
--- a/.changes/1.7.0/Fixes-20230911-005506.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: Quote column names in struct data types to fix time ingestion partitioning table
-  creation
-time: 2023-09-11T00:55:06.904238+02:00
-custom:
-  Author: Kayrnt
-  Issue: "913"
diff --git a/.changes/1.7.0/Under the Hood-20230724-164946.yaml b/.changes/1.7.0/Under the Hood-20230724-164946.yaml
deleted file mode 100644
index 2b14d4c4c..000000000
--- a/.changes/1.7.0/Under the Hood-20230724-164946.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Update stale workflow to use centralized version
-time: 2023-07-24T16:49:46.771984-04:00
-custom:
-  Author: mikealfare
-  Issue: "844"
diff --git a/.changes/1.7.0/Under the Hood-20230829-122918.yaml b/.changes/1.7.0/Under the Hood-20230829-122918.yaml
deleted file mode 100644
index 6dace0db4..000000000
--- a/.changes/1.7.0/Under the Hood-20230829-122918.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Re-organize adapters.sql into more granular files inside of macros/relations
-time: 2023-08-29T12:29:18.356174-04:00
-custom:
-  Author: mikealfare
-  Issue: "904"
diff --git a/.changes/1.7.0/Under the Hood-20230830-160707.yaml b/.changes/1.7.0/Under the Hood-20230830-160707.yaml
deleted file mode 100644
index 257e3cdbc..000000000
--- a/.changes/1.7.0/Under the Hood-20230830-160707.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Update BigQueryCredentials to support migration off hologram
-time: 2023-08-30T16:07:07.768609-07:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "906"
diff --git a/.changes/unreleased/Dependencies-20230919-003620.yaml b/.changes/unreleased/Dependencies-20230919-003620.yaml
deleted file mode 100644
index 31720679b..000000000
--- a/.changes/unreleased/Dependencies-20230919-003620.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update ddtrace requirement from ~=1.18 to ~=1.19"
-time: 2023-09-19T00:36:20.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 928
diff --git a/.changes/unreleased/Features-20230426-152526.yaml b/.changes/unreleased/Features-20230426-152526.yaml
deleted file mode 100644
index 8481f1f74..000000000
--- a/.changes/unreleased/Features-20230426-152526.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: Support partition_by and cluster_by on python models when supplied in model
-  configurations
-time: 2023-04-26T15:25:26.285021+09:00
-custom:
-  Author: kalanyuz
-  Issue: "680"
diff --git a/.changes/unreleased/Features-20230913-130445.yaml b/.changes/unreleased/Features-20230913-130445.yaml
deleted file mode 100644
index 42311fd92..000000000
--- a/.changes/unreleased/Features-20230913-130445.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: "Support change monitoring for materialized views, including: autorefresh, clustering, partitioning"
-time: 2023-09-13T13:04:45.761294-05:00
-custom:
-  Author: McKnight-42
-  Issue: "924"
diff --git a/.changes/unreleased/Features-20230921-175106.yaml b/.changes/unreleased/Features-20230921-175106.yaml
deleted file mode 100644
index cd833de67..000000000
--- a/.changes/unreleased/Features-20230921-175106.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Support storing test failures as views
-time: 2023-09-21T17:51:06.389529-04:00
-custom:
-  Author: mikealfare
-  Issue: "6914"
diff --git a/.changes/unreleased/Fixes-20230721-101041.yaml b/.changes/unreleased/Fixes-20230721-101041.yaml
deleted file mode 100644
index 6db81cf50..000000000
--- a/.changes/unreleased/Fixes-20230721-101041.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Serverless Spark to Poll with .GetBatch() instead of using operation.result()
-time: 2023-07-21T10:10:41.64843-07:00
-custom:
-  Author: wazi55
-  Issue: "734"
diff --git a/.changes/unreleased/Fixes-20230817-095527.yaml b/.changes/unreleased/Fixes-20230817-095527.yaml
deleted file mode 100644
index 34f1bf908..000000000
--- a/.changes/unreleased/Fixes-20230817-095527.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Avoid setting lifetime on impersonation tokens.
-time: 2023-08-17T09:55:27.333914673-04:00
-custom:
-  Author: cmc333333
-  Issue: "769"
diff --git a/.changes/unreleased/Fixes-20230906-141819.yaml b/.changes/unreleased/Fixes-20230906-141819.yaml
deleted file mode 100644
index f40944670..000000000
--- a/.changes/unreleased/Fixes-20230906-141819.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Fix bigquery copy materialization
-time: 2023-09-06T14:18:19.445262+02:00
-custom:
-  Author: m-sche
-  Issue: "820"
diff --git a/.changes/unreleased/Fixes-20231005-235950.yaml b/.changes/unreleased/Fixes-20231005-235950.yaml
deleted file mode 100644
index bf0bf6fa6..000000000
--- a/.changes/unreleased/Fixes-20231005-235950.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Fix issue where job labels are not rendered when using macro for query comment
-time: 2023-10-05T23:59:50.077842+02:00
-custom:
-  Author: kodaho mikealfare
-  Issue: "863"
diff --git a/.changes/unreleased/Under the Hood-20230921-155645.yaml b/.changes/unreleased/Under the Hood-20230921-155645.yaml
deleted file mode 100644
index 12cd663f8..000000000
--- a/.changes/unreleased/Under the Hood-20230921-155645.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Address type annotation issues and remove protected method ref from impl
-time: 2023-09-21T15:56:45.329798-07:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "933"
diff --git a/.changes/unreleased/Under the Hood-20230922-114217.yaml b/.changes/unreleased/Under the Hood-20230922-114217.yaml
deleted file mode 100644
index 78fee33c4..000000000
--- a/.changes/unreleased/Under the Hood-20230922-114217.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: update SQLQuery to include node_info
-time: 2023-09-22T11:42:17.770033-07:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "936"
diff --git a/.changes/unreleased/Under the Hood-20230922-125327.yaml b/.changes/unreleased/Under the Hood-20230922-125327.yaml
deleted file mode 100644
index 9ce871321..000000000
--- a/.changes/unreleased/Under the Hood-20230922-125327.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Fixed a mypy failure by reworking BigQueryAdapter constructor.
-time: 2023-09-22T12:53:27.339599-04:00
-custom:
-  Author: peterallenwebb
-  Issue: "934"
diff --git a/.changes/unreleased/Under the Hood-20230925-143628.yaml b/.changes/unreleased/Under the Hood-20230925-143628.yaml
deleted file mode 100644
index fb925ae40..000000000
--- a/.changes/unreleased/Under the Hood-20230925-143628.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Add tests for inlined limit + sql-header in dbt show query
-time: 2023-09-25T14:36:28.335466+01:00
-custom:
-  Author: michelleark
-  Issue: "940"
diff --git a/.changes/unreleased/Under the Hood-20231005-115800.yaml b/.changes/unreleased/Under the Hood-20231005-115800.yaml
deleted file mode 100644
index a5b56ed72..000000000
--- a/.changes/unreleased/Under the Hood-20231005-115800.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Support for use of type aliases in contract column data_type
-time: 2023-10-05T11:58:00.719136-04:00
-custom:
-  Author: gshank
-  Issue: "953"
diff --git a/CHANGELOG.md b/CHANGELOG.md
index d7450ff07..ade60b8f6 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,73 +5,6 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
-## dbt-bigquery 1.7.0-b2 - September 15, 2023
-
-### Features
-
-- Add support for materialized views ([#dbt-labs/dbt-core#6911](https://github.com/dbt-labs/dbt-bigquery/issues/dbt-labs/dbt-core#6911))
-
-### Fixes
-
-- case insensitive check on partition matching ([#886](https://github.com/dbt-labs/dbt-bigquery/issues/886))
-- Time out queries if user supplies `job_execution_timeout` ([#231](https://github.com/dbt-labs/dbt-bigquery/issues/231))
-- changes expected value types to AnyInteger to take into account changes in core ([#915](https://github.com/dbt-labs/dbt-bigquery/issues/915))
-- Quote column names in struct data types to fix time ingestion partitioning table creation ([#913](https://github.com/dbt-labs/dbt-bigquery/issues/913))
-
-### Under the Hood
-
-- Re-organize adapters.sql into more granular files inside of macros/relations ([#904](https://github.com/dbt-labs/dbt-bigquery/issues/904))
-- Update BigQueryCredentials to support migration off hologram ([#906](https://github.com/dbt-labs/dbt-bigquery/issues/906))
-
-### Dependencies
-
-- Bump mypy from 1.5.0 to 1.5.1 ([#883](https://github.com/dbt-labs/dbt-bigquery/pull/883))
-- Update tox requirement from ~=4.8 to ~=4.9 ([#884](https://github.com/dbt-labs/dbt-bigquery/pull/884))
-- Update tox requirement from ~=4.9 to ~=4.11 ([#907](https://github.com/dbt-labs/dbt-bigquery/pull/907))
-- Update pre-commit requirement from ~=3.3 to ~=3.4 ([#909](https://github.com/dbt-labs/dbt-bigquery/pull/909))
-- Update black requirement from ~=23.7 to ~=23.9 ([#920](https://github.com/dbt-labs/dbt-bigquery/pull/920))
-
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#886](https://github.com/dbt-labs/dbt-bigquery/issues/886), [#913](https://github.com/dbt-labs/dbt-bigquery/issues/913))
-
-
-## dbt-bigquery 1.7.0-b1 - August 17, 2023
-
-### Features
-
-- Change batch_id to model override ([#671](https://github.com/dbt-labs/dbt-bigquery/issues/671))
-
-### Fixes
-
-- Avoid creating twice the temp table in dynamic insert overwrite for sql on_schema_change != 'ignore' or python models ([#810](https://github.com/dbt-labs/dbt-bigquery/issues/810))
-- Requiring Python >=3.8 enables walrus operator ([#849](https://github.com/dbt-labs/dbt-bigquery/issues/849))
-- Fix contact validation failure on query data types change ([#861](https://github.com/dbt-labs/dbt-bigquery/issues/861))
-
-### Under the Hood
-
-- Update stale workflow to use centralized version ([#844](https://github.com/dbt-labs/dbt-bigquery/issues/844))
-
-### Dependencies
-
-- Update black requirement from ~=23.3 to ~=23.7 ([#815](https://github.com/dbt-labs/dbt-bigquery/pull/815))
-- Update pip-tools requirement from ~=6.14 to ~=7.0 ([#831](https://github.com/dbt-labs/dbt-bigquery/pull/831))
-- Update ddtrace requirement from ~=1.16 to ~=1.17 ([#839](https://github.com/dbt-labs/dbt-bigquery/pull/839))
-- Update wheel requirement from ~=0.40 to ~=0.41 ([#843](https://github.com/dbt-labs/dbt-bigquery/pull/843))
-- Update flake8 requirement from ~=6.0 to ~=6.1 ([#847](https://github.com/dbt-labs/dbt-bigquery/pull/847))
-- Update pip-tools requirement from ~=7.0 to ~=7.2 ([#851](https://github.com/dbt-labs/dbt-bigquery/pull/851))
-- Update pip-tools requirement from ~=7.2 to ~=7.3 ([#870](https://github.com/dbt-labs/dbt-bigquery/pull/870))
-- Update tox requirement from ~=4.6 to ~=4.7 ([#869](https://github.com/dbt-labs/dbt-bigquery/pull/869))
-- Update types-protobuf requirement from ~=4.23 to ~=4.24 ([#873](https://github.com/dbt-labs/dbt-bigquery/pull/873))
-- Bump mypy from 1.4.1 to 1.5.0 ([#874](https://github.com/dbt-labs/dbt-bigquery/pull/874))
-- Update tox requirement from ~=4.7 to ~=4.8 ([#880](https://github.com/dbt-labs/dbt-bigquery/pull/880))
-- Update ddtrace requirement from ~=1.17 to ~=1.18 ([#881](https://github.com/dbt-labs/dbt-bigquery/pull/881))
-
-### Contributors
-- [@Kayrnt](https://github.com/Kayrnt) ([#810](https://github.com/dbt-labs/dbt-bigquery/issues/810))
-- [@cemsbr](https://github.com/cemsbr) ([#849](https://github.com/dbt-labs/dbt-bigquery/issues/849))
-- [@dementiev27](https://github.com/dementiev27) ([#861](https://github.com/dbt-labs/dbt-bigquery/issues/861))
-- [@nickozilla](https://github.com/nickozilla) ([#671](https://github.com/dbt-labs/dbt-bigquery/issues/671))
-
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 - [1.6](https://github.com/dbt-labs/dbt-bigquery/blob/1.6.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 3f5d3c0b7..f15b401d1 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.7.0b2"
+version = "1.8.0a1"
diff --git a/setup.py b/setup.py
index 494cd286e..8920b862c 100644
--- a/setup.py
+++ b/setup.py
@@ -58,7 +58,7 @@ def _dbt_core_version(plugin_version: str) -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.7.0b2"
+package_version = "1.8.0a1"
 dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 

From bfc5dc4ddb61f8e43d03ebe317e306db01cc366d Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Fri, 13 Oct 2023 15:39:15 -0500
Subject: [PATCH 729/860] add docs-issue workflow to dbt-bigquery (#968)

---
 .github/workflows/docs-issue.yml | 43 ++++++++++++++++++++++++++++++++
 1 file changed, 43 insertions(+)
 create mode 100644 .github/workflows/docs-issue.yml

diff --git a/.github/workflows/docs-issue.yml b/.github/workflows/docs-issue.yml
new file mode 100644
index 000000000..00a098df8
--- /dev/null
+++ b/.github/workflows/docs-issue.yml
@@ -0,0 +1,43 @@
+# **what?**
+# Open an issue in docs.getdbt.com when a PR is labeled `user docs`
+
+# **why?**
+# To reduce barriers for keeping docs up to date
+
+# **when?**
+# When a PR is labeled `user docs` and is merged.  Runs on pull_request_target to run off the workflow already merged,
+# not the workflow that existed on the PR branch.  This allows old PRs to get comments.
+
+
+name: Open issues in docs.getdbt.com repo when a PR is labeled
+run-name: "Open an issue in docs.getdbt.com for PR #${{ github.event.pull_request.number }}"
+
+on:
+  pull_request_target:
+    types: [labeled, closed]
+
+defaults:
+  run:
+    shell: bash
+
+permissions:
+    issues: write # opens new issues
+    pull-requests: write # comments on PRs
+
+
+jobs:
+  open_issues:
+    # we only want to run this when the PR has been merged or the label in the labeled event is `user docs`.  Otherwise it runs the
+    # risk of duplicaton of issues being created due to merge and label both triggering this workflow to run and neither having
+    # generating the comment before the other runs.  This lives here instead of the shared workflow because this is where we
+    # decide if it should run or not.
+    if: |
+      (github.event.pull_request.merged == true) &&
+      ((github.event.action == 'closed' && contains( github.event.pull_request.labels.*.name, 'user docs')) ||
+      (github.event.action == 'labeled' && github.event.label.name == 'user docs'))
+    uses: dbt-labs/actions/.github/workflows/open-issue-in-repo.yml@main
+    with:
+        issue_repository: "dbt-labs/docs.getdbt.com"
+        issue_title: "Docs Changes Needed from ${{ github.event.repository.name }} PR #${{ github.event.pull_request.number }}"
+        issue_body: "At a minimum, update body to include a link to the page on docs.getdbt.com requiring updates and what part(s) of the page you would like to see updated."
+    secrets: inherit

From 6a3f45897f4e7ccd41de584d64605e7342dc75e0 Mon Sep 17 00:00:00 2001
From: matt-winkler <75497565+matt-winkler@users.noreply.github.com>
Date: Mon, 23 Oct 2023 11:35:27 -0600
Subject: [PATCH 730/860] Patch for `dbt show` for a model with a `struct`
 column containing a `json` datatype (#974)

* patch for --show with json

* update matts handle in changelog

* Update dbt/adapters/bigquery/connections.py

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>

* add json struct functional test

* remove old change log

* add comment to test

---------

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Co-authored-by: Colin <colin.rogers@dbtlabs.com>
---
 .../unreleased/Fixes-20231023-082312.yaml     |  6 +++
 dbt/adapters/bigquery/connections.py          | 12 +++++-
 .../adapter/dbt_show/test_dbt_show.py         | 37 +++++++++++++++++++
 3 files changed, 54 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20231023-082312.yaml

diff --git a/.changes/unreleased/Fixes-20231023-082312.yaml b/.changes/unreleased/Fixes-20231023-082312.yaml
new file mode 100644
index 000000000..368c58e95
--- /dev/null
+++ b/.changes/unreleased/Fixes-20231023-082312.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Patch for json inline --show
+time: 2023-10-23T08:23:12.245223-06:00
+custom:
+  Author: matt-winkler
+  Issue: "972"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 132854748..de84e4bf8 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -15,7 +15,7 @@
 
 import google.auth
 import google.auth.exceptions
-import google.cloud.bigquery
+import google.cloud.bigquery as bigquery
 import google.cloud.exceptions
 from google.api_core import retry, client_info
 from google.auth import impersonated_credentials
@@ -63,6 +63,16 @@
 )
 
 
+# Override broken json deserializer for dbt show --inline
+# can remove once this is fixed: https://github.com/googleapis/python-bigquery/issues/1500
+def _json_from_json(value, _):
+    """NOOP string -> string coercion"""
+    return json.loads(value)
+
+
+bigquery._helpers._CELLDATA_FROM_JSON["JSON"] = _json_from_json
+
+
 @lru_cache()
 def get_bigquery_defaults(scopes=None) -> Tuple[Any, Optional[str]]:
     """
diff --git a/tests/functional/adapter/dbt_show/test_dbt_show.py b/tests/functional/adapter/dbt_show/test_dbt_show.py
index c60a26aec..203d7031b 100644
--- a/tests/functional/adapter/dbt_show/test_dbt_show.py
+++ b/tests/functional/adapter/dbt_show/test_dbt_show.py
@@ -1,5 +1,29 @@
+import pytest
 from dbt.tests.adapter.dbt_show.test_dbt_show import BaseShowSqlHeader, BaseShowLimit
 
+from dbt.tests.util import run_dbt
+
+model_with_json_struct = """
+    select *
+    from (
+        select
+  struct<
+    k array<
+        struct<c1 int64, c2 json>
+      >
+  >(
+    [
+      struct(
+        1 as c1,
+        to_json(struct(1 as a)) as c2
+      )
+    ]
+  )
+  as v
+    ) as model_limit_subq
+    limit 5
+    """
+
 
 class TestBigQueryShowLimit(BaseShowLimit):
     pass
@@ -7,3 +31,16 @@ class TestBigQueryShowLimit(BaseShowLimit):
 
 class TestBigQueryShowSqlHeader(BaseShowSqlHeader):
     pass
+
+
+# Added to check if dbt show works with JSON struct
+# Addresses: https://github.com/dbt-labs/dbt-bigquery/issues/972
+class TestBigQueryShowSqlWorksWithJSONStruct:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "json_struct_model.sql": model_with_json_struct,
+        }
+
+    def test_sql_header(self, project):
+        run_dbt(["show", "--select", "json_struct_model"])

From f0f6d1451a1f04da5a7f4718685572c8e7a1316c Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Wed, 25 Oct 2023 16:23:37 -0600
Subject: [PATCH 731/860] Lower bound of `2.11.0` for `google-api-core` (#980)

* Lower bound of `2.11.0` for `google-api-core`

* Changelog entry
---
 .changes/unreleased/Fixes-20231025-131907.yaml | 6 ++++++
 setup.py                                       | 3 +++
 2 files changed, 9 insertions(+)
 create mode 100644 .changes/unreleased/Fixes-20231025-131907.yaml

diff --git a/.changes/unreleased/Fixes-20231025-131907.yaml b/.changes/unreleased/Fixes-20231025-131907.yaml
new file mode 100644
index 000000000..9a3b8d8a8
--- /dev/null
+++ b/.changes/unreleased/Fixes-20231025-131907.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Lower bound of `2.11.0` for `google-api-core`
+time: 2023-10-25T13:19:07.580826-06:00
+custom:
+  Author: gmyrianthous dbeatty10
+  Issue: "979"
diff --git a/setup.py b/setup.py
index 8920b862c..56c271fec 100644
--- a/setup.py
+++ b/setup.py
@@ -78,6 +78,9 @@ def _dbt_core_version(plugin_version: str) -> str:
         "google-cloud-bigquery~=3.0",
         "google-cloud-storage~=2.4",
         "google-cloud-dataproc~=5.0",
+        # ----
+        # Expect compatibility with all new versions of these packages, so lower bounds only.
+        "google-api-core>=2.11.0",
     ],
     zip_safe=False,
     classifiers=[

From 0fcb2c886852fd1d95119b58fe78885d6f0adabd Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 27 Oct 2023 03:23:28 +0000
Subject: [PATCH 732/860] Update black requirement from ~=23.9 to ~=23.10
 (#973)

* Update black requirement from ~=23.9 to ~=23.10

Updates the requirements on [black](https://github.com/psf/black) to permit the latest version.
- [Release notes](https://github.com/psf/black/releases)
- [Changelog](https://github.com/psf/black/blob/main/CHANGES.md)
- [Commits](https://github.com/psf/black/compare/23.9.0...23.10.0)

---
updated-dependencies:
- dependency-name: black
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231018-010429.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231018-010429.yaml

diff --git a/.changes/unreleased/Dependencies-20231018-010429.yaml b/.changes/unreleased/Dependencies-20231018-010429.yaml
new file mode 100644
index 000000000..fec345104
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231018-010429.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update black requirement from ~=23.9 to ~=23.10"
+time: 2023-10-18T01:04:29.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 973
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 496aa5538..36f65c7f1 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -5,7 +5,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor
-black~=23.9
+black~=23.10
 bumpversion~=0.6.0
 click~=8.1
 ddtrace~=1.19

From 18fceefc9e0c5eae7625cf03655b06e1270d19b6 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 27 Oct 2023 09:26:14 -0400
Subject: [PATCH 733/860] Bump mypy from 1.5.1 to 1.6.0 (#963)

* Bump mypy from 1.5.1 to 1.6.0

Bumps [mypy](https://github.com/python/mypy) from 1.5.1 to 1.6.0.
- [Commits](https://github.com/python/mypy/compare/v1.5.1...v1.6.0)

---
updated-dependencies:
- dependency-name: mypy
  dependency-type: direct:development
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231011-002031.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231011-002031.yaml

diff --git a/.changes/unreleased/Dependencies-20231011-002031.yaml b/.changes/unreleased/Dependencies-20231011-002031.yaml
new file mode 100644
index 000000000..5cc3c36a1
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231011-002031.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump mypy from 1.5.1 to 1.6.0"
+time: 2023-10-11T00:20:31.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 963
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 36f65c7f1..752885a58 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -13,7 +13,7 @@ flake8~=6.1
 flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
-mypy==1.5.1  # patch updates have historically introduced breaking changes
+mypy==1.6.0  # patch updates have historically introduced breaking changes
 pip-tools~=7.3
 pre-commit~=3.4
 pre-commit-hooks~=4.4

From a3b1c958a59f682f117fec7196b088a339ca6c95 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 27 Oct 2023 10:30:39 -0400
Subject: [PATCH 734/860] Update pre-commit requirement from ~=3.4 to ~=3.5
 (#969)

* Update pre-commit requirement from ~=3.4 to ~=3.5

Updates the requirements on [pre-commit](https://github.com/pre-commit/pre-commit) to permit the latest version.
- [Release notes](https://github.com/pre-commit/pre-commit/releases)
- [Changelog](https://github.com/pre-commit/pre-commit/blob/main/CHANGELOG.md)
- [Commits](https://github.com/pre-commit/pre-commit/compare/v3.4.0...v3.5.0)

---
updated-dependencies:
- dependency-name: pre-commit
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231016-002928.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231016-002928.yaml

diff --git a/.changes/unreleased/Dependencies-20231016-002928.yaml b/.changes/unreleased/Dependencies-20231016-002928.yaml
new file mode 100644
index 000000000..ac9470de7
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231016-002928.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pre-commit requirement from ~=3.4 to ~=3.5"
+time: 2023-10-16T00:29:28.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 969
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 752885a58..03e8b92a7 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -15,7 +15,7 @@ freezegun~=1.2
 ipdb~=0.13.13
 mypy==1.6.0  # patch updates have historically introduced breaking changes
 pip-tools~=7.3
-pre-commit~=3.4
+pre-commit~=3.5
 pre-commit-hooks~=4.4
 pytest~=7.4
 pytest-csv~=3.0

From da4a79aef4058a637f5017dde249bfc00c55a275 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 27 Oct 2023 13:10:48 -0400
Subject: [PATCH 735/860] Bump mypy from 1.6.0 to 1.6.1 (#985)

* Bump mypy from 1.6.0 to 1.6.1

Bumps [mypy](https://github.com/python/mypy) from 1.6.0 to 1.6.1.
- [Changelog](https://github.com/python/mypy/blob/master/CHANGELOG.md)
- [Commits](https://github.com/python/mypy/compare/v1.6.0...v1.6.1)

---
updated-dependencies:
- dependency-name: mypy
  dependency-type: direct:development
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231027-132742.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231027-132742.yaml

diff --git a/.changes/unreleased/Dependencies-20231027-132742.yaml b/.changes/unreleased/Dependencies-20231027-132742.yaml
new file mode 100644
index 000000000..d72ac124b
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231027-132742.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump mypy from 1.6.0 to 1.6.1"
+time: 2023-10-27T13:27:42.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 985
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 03e8b92a7..387ce2a49 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -13,7 +13,7 @@ flake8~=6.1
 flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
-mypy==1.6.0  # patch updates have historically introduced breaking changes
+mypy==1.6.1  # patch updates have historically introduced breaking changes
 pip-tools~=7.3
 pre-commit~=3.5
 pre-commit-hooks~=4.4

From 377fc936d0cb1df9f6973ecedafa9789d259ff28 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 27 Oct 2023 15:10:48 -0400
Subject: [PATCH 736/860] Update pre-commit-hooks requirement from ~=4.4 to
 ~=4.5 (#960)

* Update pre-commit-hooks requirement from ~=4.4 to ~=4.5

Updates the requirements on [pre-commit-hooks](https://github.com/pre-commit/pre-commit-hooks) to permit the latest version.
- [Release notes](https://github.com/pre-commit/pre-commit-hooks/releases)
- [Changelog](https://github.com/pre-commit/pre-commit-hooks/blob/main/CHANGELOG.md)
- [Commits](https://github.com/pre-commit/pre-commit-hooks/compare/v4.4.0...v4.5.0)

---
updated-dependencies:
- dependency-name: pre-commit-hooks
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231009-005842.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231009-005842.yaml

diff --git a/.changes/unreleased/Dependencies-20231009-005842.yaml b/.changes/unreleased/Dependencies-20231009-005842.yaml
new file mode 100644
index 000000000..acedd8d52
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231009-005842.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pre-commit-hooks requirement from ~=4.4 to ~=4.5"
+time: 2023-10-09T00:58:42.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 960
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 387ce2a49..3113133e6 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -16,7 +16,7 @@ ipdb~=0.13.13
 mypy==1.6.1  # patch updates have historically introduced breaking changes
 pip-tools~=7.3
 pre-commit~=3.5
-pre-commit-hooks~=4.4
+pre-commit-hooks~=4.5
 pytest~=7.4
 pytest-csv~=3.0
 pytest-dotenv~=0.5.2

From b8af6a6bba53ee46c3190b6d7484dc91653df267 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 27 Oct 2023 16:15:52 -0400
Subject: [PATCH 737/860] Update ddtrace requirement from ~=1.19 to ~=1.20
 (#948)

* Update ddtrace requirement from ~=1.19 to ~=1.20

Updates the requirements on [ddtrace](https://github.com/DataDog/dd-trace-py) to permit the latest version.
- [Release notes](https://github.com/DataDog/dd-trace-py/releases)
- [Changelog](https://github.com/DataDog/dd-trace-py/blob/2.x/CHANGELOG.md)
- [Commits](https://github.com/DataDog/dd-trace-py/compare/v1.19.0...v1.20.2)

---
updated-dependencies:
- dependency-name: ddtrace
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231002-164012.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231002-164012.yaml

diff --git a/.changes/unreleased/Dependencies-20231002-164012.yaml b/.changes/unreleased/Dependencies-20231002-164012.yaml
new file mode 100644
index 000000000..344aeb148
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231002-164012.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update ddtrace requirement from ~=1.19 to ~=1.20"
+time: 2023-10-02T16:40:12.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 948
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 3113133e6..5935fe821 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -8,7 +8,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 black~=23.10
 bumpversion~=0.6.0
 click~=8.1
-ddtrace~=1.19
+ddtrace~=1.20
 flake8~=6.1
 flaky~=3.7
 freezegun~=1.2

From f139e95cb3fd05651bcfda15b72cb7113545d1f6 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Fri, 27 Oct 2023 17:01:44 -0400
Subject: [PATCH 738/860] Fix refresh syntax, config comparison with empty
 labels (#983)

* map an empty labels dict to none for comparison

* remove drop statement that isn't used

* fix refresh statement

* add drop materialized view back, it gets called by replace

* changelog
---
 .changes/unreleased/Fixes-20231025-223003.yaml              | 6 ++++++
 dbt/adapters/bigquery/relation_configs/_options.py          | 6 +++++-
 .../bigquery/macros/relations/materialized_view/refresh.sql | 2 +-
 3 files changed, 12 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20231025-223003.yaml

diff --git a/.changes/unreleased/Fixes-20231025-223003.yaml b/.changes/unreleased/Fixes-20231025-223003.yaml
new file mode 100644
index 000000000..ebec94a30
--- /dev/null
+++ b/.changes/unreleased/Fixes-20231025-223003.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fix refresh syntax, config comparison with empty labels
+time: 2023-10-25T22:30:03.0034-04:00
+custom:
+  Author: mikealfare
+  Issue: "983"
diff --git a/dbt/adapters/bigquery/relation_configs/_options.py b/dbt/adapters/bigquery/relation_configs/_options.py
index 51774e3fb..72f9d73e6 100644
--- a/dbt/adapters/bigquery/relation_configs/_options.py
+++ b/dbt/adapters/bigquery/relation_configs/_options.py
@@ -138,8 +138,12 @@ def parse_bq_table(cls, table: BigQueryTable) -> Dict[str, Any]:
             "expiration_timestamp": table.expires,
             "max_staleness": None,
             "description": table.description,
-            "labels": table.labels,
         }
+
+        # map the empty dict to None
+        if labels := table.labels:
+            config_dict.update({"labels": labels})
+
         if encryption_configuration := table.encryption_configuration:
             config_dict.update({"kms_key_name": encryption_configuration.kms_key_name})
         return config_dict
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql b/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql
index 82bf819cd..40ad59f7b 100644
--- a/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql
+++ b/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql
@@ -1,3 +1,3 @@
 {% macro bigquery__refresh_materialized_view(relation) %}
-    call bq.refresh_materialized_view('{{ relation }}')
+    call bq.refresh_materialized_view('{{ relation.database }}.{{ relation.schema }}.{{ relation.identifier }}')
 {% endmacro %}

From 2367be79640ae2e1c040852b50530c53e9e4273a Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 27 Oct 2023 17:54:04 -0400
Subject: [PATCH 739/860] Update ddtrace requirement from ~=1.20 to ~=2.1
 (#989)

* Update ddtrace requirement from ~=1.20 to ~=2.1

Updates the requirements on [ddtrace](https://github.com/DataDog/dd-trace-py) to permit the latest version.
- [Release notes](https://github.com/DataDog/dd-trace-py/releases)
- [Changelog](https://github.com/DataDog/dd-trace-py/blob/2.x/CHANGELOG.md)
- [Commits](https://github.com/DataDog/dd-trace-py/compare/v1.20.0...v2.1.3)

---
updated-dependencies:
- dependency-name: ddtrace
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231027-201709.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231027-201709.yaml

diff --git a/.changes/unreleased/Dependencies-20231027-201709.yaml b/.changes/unreleased/Dependencies-20231027-201709.yaml
new file mode 100644
index 000000000..e9d2a1adb
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231027-201709.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update ddtrace requirement from ~=1.20 to ~=2.1"
+time: 2023-10-27T20:17:09.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 989
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 5935fe821..48598298b 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -8,7 +8,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 black~=23.10
 bumpversion~=0.6.0
 click~=8.1
-ddtrace~=1.20
+ddtrace~=2.1
 flake8~=6.1
 flaky~=3.7
 freezegun~=1.2

From f6e26d0073fbf3830457532f57b3b59e60af0535 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Tue, 31 Oct 2023 00:41:17 -0400
Subject: [PATCH 740/860] ADAP-972: Fix issue where materialized views were
 being mapped as tables in catalog queries (#996)

* changelog
* add test demonstrating issue
* update catalog query to correctly identify materialized views
---
 .../unreleased/Fixes-20231030-222134.yaml     |  6 +++
 dbt/include/bigquery/macros/catalog.sql       | 40 +++++++++++------
 .../functional/adapter/catalog_tests/files.py | 33 ++++++++++++++
 .../catalog_tests/test_relation_types.py      | 44 +++++++++++++++++++
 4 files changed, 109 insertions(+), 14 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20231030-222134.yaml
 create mode 100644 tests/functional/adapter/catalog_tests/files.py
 create mode 100644 tests/functional/adapter/catalog_tests/test_relation_types.py

diff --git a/.changes/unreleased/Fixes-20231030-222134.yaml b/.changes/unreleased/Fixes-20231030-222134.yaml
new file mode 100644
index 000000000..62bfc5f27
--- /dev/null
+++ b/.changes/unreleased/Fixes-20231030-222134.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Assign the correct relation type to materialized views in catalog queries
+time: 2023-10-30T22:21:34.401675-04:00
+custom:
+  Author: mikealfare
+  Issue: "995"
diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql
index d430b80f8..25166c7b4 100644
--- a/dbt/include/bigquery/macros/catalog.sql
+++ b/dbt/include/bigquery/macros/catalog.sql
@@ -7,30 +7,42 @@
   {%- else -%}
 
   {%- set query -%}
-    with tables as (
+    with materialized_views as (
         select
-            project_id as table_database,
-            dataset_id as table_schema,
-            table_id as original_table_name,
+            table_catalog as project_id,
+            table_schema as dataset_id,
+            table_name as table_id
+        from {{ information_schema.replace(information_schema_view='MATERIALIZED_VIEWS') }}
+    ),
+    tables as (
+        select
+            tables.project_id as table_database,
+            tables.dataset_id as table_schema,
+            tables.table_id as original_table_name,
 
-            concat(project_id, '.', dataset_id, '.', table_id) as relation_id,
+            concat(tables.project_id, '.', tables.dataset_id, '.', tables.table_id) as relation_id,
 
-            row_count,
-            size_bytes as size_bytes,
+            tables.row_count,
+            tables.size_bytes as size_bytes,
             case
-                when type = 1 then 'table'
-                when type = 2 then 'view'
+                when materialized_views.table_id is not null then 'materialized view'
+                when tables.type = 1 then 'table'
+                when tables.type = 2 then 'view'
                 else 'external'
             end as table_type,
 
-            REGEXP_CONTAINS(table_id, '^.+[0-9]{8}$') and coalesce(type, 0) = 1 as is_date_shard,
-            REGEXP_EXTRACT(table_id, '^(.+)[0-9]{8}$') as shard_base_name,
-            REGEXP_EXTRACT(table_id, '^.+([0-9]{8})$') as shard_name
+            REGEXP_CONTAINS(tables.table_id, '^.+[0-9]{8}$') and coalesce(type, 0) = 1 as is_date_shard,
+            REGEXP_EXTRACT(tables.table_id, '^(.+)[0-9]{8}$') as shard_base_name,
+            REGEXP_EXTRACT(tables.table_id, '^.+([0-9]{8})$') as shard_name
 
-        from {{ information_schema.replace(information_schema_view='__TABLES__') }}
+        from {{ information_schema.replace(information_schema_view='__TABLES__') }} tables
+        left join materialized_views
+            on materialized_views.project_id = tables.project_id
+            and materialized_views.dataset_id = tables.dataset_id
+            and materialized_views.table_id = tables.table_id
         where (
           {%- for schema in schemas -%}
-            upper(dataset_id) = upper('{{ schema }}'){%- if not loop.last %} or {% endif -%}
+            upper(tables.dataset_id) = upper('{{ schema }}'){%- if not loop.last %} or {% endif -%}
           {%- endfor -%}
         )
     ),
diff --git a/tests/functional/adapter/catalog_tests/files.py b/tests/functional/adapter/catalog_tests/files.py
new file mode 100644
index 000000000..595517bf2
--- /dev/null
+++ b/tests/functional/adapter/catalog_tests/files.py
@@ -0,0 +1,33 @@
+MY_SEED = """
+id,value,record_valid_date
+1,100,2023-01-01 00:00:00
+2,200,2023-01-02 00:00:00
+3,300,2023-01-02 00:00:00
+""".strip()
+
+
+MY_TABLE = """
+{{ config(
+    materialized='table',
+) }}
+select *
+from {{ ref('my_seed') }}
+"""
+
+
+MY_VIEW = """
+{{ config(
+    materialized='view',
+) }}
+select *
+from {{ ref('my_seed') }}
+"""
+
+
+MY_MATERIALIZED_VIEW = """
+{{ config(
+    materialized='materialized_view',
+) }}
+select *
+from {{ ref('my_table') }}
+"""
diff --git a/tests/functional/adapter/catalog_tests/test_relation_types.py b/tests/functional/adapter/catalog_tests/test_relation_types.py
new file mode 100644
index 000000000..96beb69c9
--- /dev/null
+++ b/tests/functional/adapter/catalog_tests/test_relation_types.py
@@ -0,0 +1,44 @@
+from dbt.contracts.results import CatalogArtifact
+from dbt.tests.util import run_dbt
+import pytest
+
+from tests.functional.adapter.catalog_tests import files
+
+
+class TestCatalogRelationTypes:
+    @pytest.fixture(scope="class", autouse=True)
+    def seeds(self):
+        return {"my_seed.csv": files.MY_SEED}
+
+    @pytest.fixture(scope="class", autouse=True)
+    def models(self):
+        yield {
+            "my_table.sql": files.MY_TABLE,
+            "my_view.sql": files.MY_VIEW,
+            "my_materialized_view.sql": files.MY_MATERIALIZED_VIEW,
+        }
+
+    @pytest.fixture(scope="class", autouse=True)
+    def docs(self, project):
+        run_dbt(["seed"])
+        run_dbt(["run"])
+        yield run_dbt(["docs", "generate"])
+
+    @pytest.mark.parametrize(
+        "node_name,relation_type",
+        [
+            ("seed.test.my_seed", "table"),
+            ("model.test.my_table", "table"),
+            ("model.test.my_view", "view"),
+            ("model.test.my_materialized_view", "materialized view"),
+        ],
+    )
+    def test_relation_types_populate_correctly(
+        self, docs: CatalogArtifact, node_name: str, relation_type: str
+    ):
+        """
+        This test addresses: https://github.com/dbt-labs/dbt-bigquery/issues/995
+        """
+        assert node_name in docs.nodes
+        node = docs.nodes[node_name]
+        assert node.metadata.type == relation_type

From 548d532052d356a6f4600b01ba2af5b132eb1560 Mon Sep 17 00:00:00 2001
From: Gerda Shank <gerda@dbtlabs.com>
Date: Tue, 7 Nov 2023 14:11:43 -0500
Subject: [PATCH 741/860] Support agate Integer type, test with empty seed
 (#1004)

---
 .changes/unreleased/Fixes-20231107-100905.yaml | 6 ++++++
 dbt/adapters/bigquery/impl.py                  | 4 ++++
 tests/functional/adapter/test_simple_seed.py   | 5 +++++
 tests/unit/mock_adapter.py                     | 3 +++
 4 files changed, 18 insertions(+)
 create mode 100644 .changes/unreleased/Fixes-20231107-100905.yaml

diff --git a/.changes/unreleased/Fixes-20231107-100905.yaml b/.changes/unreleased/Fixes-20231107-100905.yaml
new file mode 100644
index 000000000..942298ed9
--- /dev/null
+++ b/.changes/unreleased/Fixes-20231107-100905.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Support agate Integer type, test with empty seed
+time: 2023-11-07T10:09:05.723451-05:00
+custom:
+  Author: gshank
+  Issue: "1003"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 50ce21f11..7d9b003b8 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -324,6 +324,10 @@ def convert_number_type(cls, agate_table: agate.Table, col_idx: int) -> str:
         decimals = agate_table.aggregate(agate.MaxPrecision(col_idx))  # type: ignore[attr-defined]
         return "float64" if decimals else "int64"
 
+    @classmethod
+    def convert_integer_type(cls, agate_table: agate.Table, col_idx: int) -> str:
+        return "int64"
+
     @classmethod
     def convert_boolean_type(cls, agate_table: agate.Table, col_idx: int) -> str:
         return "bool"
diff --git a/tests/functional/adapter/test_simple_seed.py b/tests/functional/adapter/test_simple_seed.py
index c8b178467..b01f99346 100644
--- a/tests/functional/adapter/test_simple_seed.py
+++ b/tests/functional/adapter/test_simple_seed.py
@@ -2,6 +2,7 @@
 from dbt.tests.adapter.simple_seed.fixtures import macros__schema_test
 from dbt.tests.adapter.simple_seed.seeds import seeds__enabled_in_config_csv, seeds__tricky_csv
 from dbt.tests.adapter.simple_seed.test_seed import SeedConfigBase
+from dbt.tests.adapter.simple_seed.test_seed import BaseTestEmptySeed
 from dbt.tests.adapter.utils.base_utils import run_dbt
 
 
@@ -151,3 +152,7 @@ def test__bigquery_seed_table_with_labels_config_bigquery(self, project):
             assert bq_table.labels
             assert bq_table.labels == self.table_labels()
             assert bq_table.expires
+
+
+class TestBigQueryEmptySeed(BaseTestEmptySeed):
+    pass
diff --git a/tests/unit/mock_adapter.py b/tests/unit/mock_adapter.py
index d3bdf87b2..885854261 100644
--- a/tests/unit/mock_adapter.py
+++ b/tests/unit/mock_adapter.py
@@ -55,6 +55,9 @@ def convert_text_type(self, *args, **kwargs):
         def convert_number_type(self, *args, **kwargs):
             return self.responder.convert_number_type(*args, **kwargs)
 
+        def convert_integer_type(self, *args, **kwargs):
+            return self.responder.convert_integer_type(*args, **kwargs)
+
         def convert_boolean_type(self, *args, **kwargs):
             return self.responder.convert_boolean_type(*args, **kwargs)
 

From 64e042a2349f17342f7af96d559e4e3a5138a49e Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Tue, 7 Nov 2023 20:47:20 -0500
Subject: [PATCH 742/860] ADAP-1017: Fix configuration change monitoring for
 scenarios with no changes (#1009)

* changie
* add a test demonstrating the issue
* check that the change collection has changes instead of checking that it exists
* remove the partition config value from the config change since it is not hashable
---
 .../unreleased/Fixes-20231107-174352.yaml     |  7 ++++++
 dbt/adapters/bigquery/relation.py             |  5 ++--
 .../bigquery/relation_configs/_partition.py   |  2 +-
 .../adapter/materialized_view_tests/_files.py | 11 +++++++++
 .../test_materialized_view.py                 | 24 +++++++++++++++++++
 5 files changed, 46 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20231107-174352.yaml

diff --git a/.changes/unreleased/Fixes-20231107-174352.yaml b/.changes/unreleased/Fixes-20231107-174352.yaml
new file mode 100644
index 000000000..80592758d
--- /dev/null
+++ b/.changes/unreleased/Fixes-20231107-174352.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: Fixed issue where materialized views were failing on re-run with minimal config
+  parameters
+time: 2023-11-07T17:43:52.972135-05:00
+custom:
+  Author: "mikealfare"
+  Issue: "1007"
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index a689e76fc..c14dba238 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -84,9 +84,10 @@ def materialized_view_config_changeset(
             )
 
         if new_materialized_view.partition != existing_materialized_view.partition:
+            # the existing PartitionConfig is not hashable, but since we need to do
+            # a full refresh either way, we don't need to provide a context
             config_change_collection.partition = BigQueryPartitionConfigChange(
                 action=RelationConfigChangeAction.alter,
-                context=new_materialized_view.partition,
             )
 
         if new_materialized_view.cluster != existing_materialized_view.cluster:
@@ -95,7 +96,7 @@ def materialized_view_config_changeset(
                 context=new_materialized_view.cluster,
             )
 
-        if config_change_collection:
+        if config_change_collection.has_changes:
             return config_change_collection
         return None
 
diff --git a/dbt/adapters/bigquery/relation_configs/_partition.py b/dbt/adapters/bigquery/relation_configs/_partition.py
index 094e4f1c4..0d0ee23a1 100644
--- a/dbt/adapters/bigquery/relation_configs/_partition.py
+++ b/dbt/adapters/bigquery/relation_configs/_partition.py
@@ -152,7 +152,7 @@ def parse_bq_table(cls, table: BigQueryTable) -> Dict[str, Any]:
 
 @dataclass(frozen=True, eq=True, unsafe_hash=True)
 class BigQueryPartitionConfigChange(RelationConfigChange):
-    context: Optional[PartitionConfig]
+    context: Optional[Any] = None
 
     @property
     def requires_full_refresh(self) -> bool:
diff --git a/tests/functional/adapter/materialized_view_tests/_files.py b/tests/functional/adapter/materialized_view_tests/_files.py
index 1ee64269d..86714036a 100644
--- a/tests/functional/adapter/materialized_view_tests/_files.py
+++ b/tests/functional/adapter/materialized_view_tests/_files.py
@@ -67,3 +67,14 @@
     record_valid_date
 from {{ ref('my_seed') }}
 """
+
+
+MY_MINIMAL_MATERIALIZED_VIEW = """
+{{
+  config(
+    materialized = 'materialized_view',
+    )
+}}
+
+select * from {{ ref('my_seed') }}
+"""
diff --git a/tests/functional/adapter/materialized_view_tests/test_materialized_view.py b/tests/functional/adapter/materialized_view_tests/test_materialized_view.py
index 7ca90983e..4e980c2e4 100644
--- a/tests/functional/adapter/materialized_view_tests/test_materialized_view.py
+++ b/tests/functional/adapter/materialized_view_tests/test_materialized_view.py
@@ -4,6 +4,7 @@
 from dbt.tests.adapter.materialized_view.basic import MaterializedViewBasic
 
 from tests.functional.adapter.materialized_view_tests._mixin import BigQueryMaterializedViewMixin
+from tests.functional.adapter.materialized_view_tests import _files
 
 
 class TestBigqueryMaterializedViewsBasic(BigQueryMaterializedViewMixin, MaterializedViewBasic):
@@ -27,3 +28,26 @@ def test_materialized_view_only_updates_after_refresh(
         self, project, my_materialized_view, my_seed
     ):
         pass
+
+
+class TestMaterializedViewRerun:
+    """
+    This addresses: https://github.com/dbt-labs/dbt-bigquery/issues/1007
+
+    This effectively tests that defaults get properly set so that the run is idempotent.
+    If the defaults are not properly set, changes could appear when there are no changes
+    and cause unexpected scenarios.
+    """
+
+    @pytest.fixture(scope="class", autouse=True)
+    def models(self):
+        return {"my_minimal_materialized_view.sql": _files.MY_MINIMAL_MATERIALIZED_VIEW}
+
+    @pytest.fixture(scope="class", autouse=True)
+    def seeds(self):
+        return {"my_seed.csv": _files.MY_SEED}
+
+    def test_minimal_materialized_view_is_idempotent(self, project):
+        run_dbt(["seed"])
+        run_dbt(["run"])
+        run_dbt(["run"])

From bf30b66bab0e96dd4d100f45c2818daa5dde456d Mon Sep 17 00:00:00 2001
From: tnk-ysk <61967017+tnk-ysk@users.noreply.github.com>
Date: Thu, 9 Nov 2023 01:26:14 +0900
Subject: [PATCH 743/860] Fix inline comments (--) on the last line of an
 incremental model (partition replacement) (#991)

* Fix inline comments on the last line
* Add changes
* Add test for comment

---------

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20231105-125740.yaml              | 6 ++++++
 .../incremental_strategy/insert_overwrite.sql               | 2 +-
 .../adapter/incremental/incremental_strategy_fixtures.py    | 1 +
 3 files changed, 8 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20231105-125740.yaml

diff --git a/.changes/unreleased/Fixes-20231105-125740.yaml b/.changes/unreleased/Fixes-20231105-125740.yaml
new file mode 100644
index 000000000..928fbb302
--- /dev/null
+++ b/.changes/unreleased/Fixes-20231105-125740.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fix inline comments (--) on the last line of an incremental model
+time: 2023-11-05T12:57:40.289399+09:00
+custom:
+  Author: tnk-ysk
+  Issue: "896"
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
index 4c22fd376..3ba67931e 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
@@ -70,7 +70,7 @@
             {{ wrap_with_time_ingestion_partitioning_sql(partition_by, sql, True) }}
           {%- else -%}
             {{sql}}
-          {%- endif -%}
+          {%- endif %}
 
         )
       {%- endset -%}
diff --git a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
index 8dd470ffb..a8f0004c5 100644
--- a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
+++ b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
@@ -306,6 +306,7 @@
 {% if is_incremental() %}
 where date_day in ({{ config.get("partitions") | join(",") }})
 {% endif %}
+-- Test comment to prevent recurrence of https://github.com/dbt-labs/dbt-bigquery/issues/896
 """.lstrip()
 
 overwrite_range_sql = """

From 9e39acfc1207f3e8495b81b675b7be99d0668587 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 9 Nov 2023 09:52:14 -0800
Subject: [PATCH 744/860] Fix Broken Python Models (#1014)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* fix partitioninb

* skip TestPythonBatchIdModels

* add changie

* run python integration tests separately

* run python integration tests separately

* cleanup _get_batch_id

* add space to pipe

* fix integration.yml

* add extra changie
---
 .../unreleased/Fixes-20231108-171128.yaml     |  6 ++
 .../Under the Hood-20231109-095012.yaml       |  6 ++
 .github/workflows/integration.yml             | 78 ++++++++++++++-----
 dbt/adapters/bigquery/dataproc/batch.py       |  2 +-
 dbt/adapters/bigquery/python_submissions.py   |  9 ++-
 .../macros/materializations/table.sql         |  6 +-
 tests/functional/adapter/test_python_model.py |  1 +
 7 files changed, 84 insertions(+), 24 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20231108-171128.yaml
 create mode 100644 .changes/unreleased/Under the Hood-20231109-095012.yaml

diff --git a/.changes/unreleased/Fixes-20231108-171128.yaml b/.changes/unreleased/Fixes-20231108-171128.yaml
new file mode 100644
index 000000000..116ff00d2
--- /dev/null
+++ b/.changes/unreleased/Fixes-20231108-171128.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fix broken partition config granularity and batch_id being set to None
+time: 2023-11-08T17:11:28.819877-08:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "1006"
diff --git a/.changes/unreleased/Under the Hood-20231109-095012.yaml b/.changes/unreleased/Under the Hood-20231109-095012.yaml
new file mode 100644
index 000000000..a93215e8f
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20231109-095012.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Upgrade spark-bigquery Java deps for serverless to 2.13-0.34.0
+time: 2023-11-09T09:50:12.252774-08:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "1006"
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index bb0211b35..7df6973a8 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -97,10 +97,6 @@ jobs:
               - 'dbt/**'
               - 'tests/**'
               - 'dev-requirements.txt'
-            bigquery-python:
-              - 'dbt/adapters/bigquery/dataproc/**'
-              - 'dbt/adapters/bigquery/python_submissions.py'
-              - 'dbt/include/bigquery/python_model/**'
 
       - name: Generate integration test matrix
         id: generate-matrix
@@ -192,21 +188,6 @@ jobs:
           GCS_BUCKET: dbt-ci
         run: tox -- --ddtrace
 
-      # python models tests are slow so we only want to run them if we're changing them
-      - name: Run tox (python models)
-        if: needs.test-metadata.outputs.run-python-tests == 'true'
-        env:
-          BIGQUERY_TEST_SERVICE_ACCOUNT_JSON: ${{ secrets.BIGQUERY_TEST_SERVICE_ACCOUNT_JSON }}
-          BIGQUERY_TEST_ALT_DATABASE: ${{ secrets.BIGQUERY_TEST_ALT_DATABASE }}
-          BIGQUERY_TEST_NO_ACCESS_DATABASE: ${{ secrets.BIGQUERY_TEST_NO_ACCESS_DATABASE }}
-          DBT_TEST_USER_1: group:buildbot@dbtlabs.com
-          DBT_TEST_USER_2: group:engineering-core-team@dbtlabs.com
-          DBT_TEST_USER_3: serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserviceaccount.com
-          DATAPROC_REGION: us-central1
-          DATAPROC_CLUSTER_NAME: dbt-test-1
-          GCS_BUCKET: dbt-ci
-        run: tox -e python-tests -- --ddtrace
-
       - uses: actions/upload-artifact@v3
         if: always()
         with:
@@ -225,10 +206,67 @@ jobs:
           name: integration_results_${{ matrix.python-version }}_${{ matrix.os }}_${{ matrix.adapter }}-${{ steps.date.outputs.date }}.csv
           path: integration_results.csv
 
+  # python integration tests are slow so we only run them seperately and for a single OS / python version
+  test-python:
+    name: "test-python"
+    needs: test-metadata
+    runs-on: ubuntu-latest
+    if: >-
+      needs.test-metadata.outputs.matrix &&
+      fromJSON( needs.test-metadata.outputs.matrix ).include[0] &&
+      (
+        github.event_name != 'pull_request_target' ||
+        github.event.pull_request.head.repo.full_name == github.repository ||
+        contains(github.event.pull_request.labels.*.name, 'ok to test')
+      )
+
+    steps:
+      - name: Check out the repository
+        if: github.event_name != 'pull_request_target'
+        uses: actions/checkout@v3
+        with:
+          persist-credentials: false
+
+      # explicitly checkout the branch for the PR,
+      # this is necessary for the `pull_request_target` event
+      - name: Check out the repository (PR)
+        if: github.event_name == 'pull_request_target'
+        uses: actions/checkout@v3
+        with:
+          persist-credentials: false
+          ref: ${{ github.event.pull_request.head.sha }}
+
+      - name: Set up Python 3.8
+        uses: actions/setup-python@v4
+        with:
+          python-version: "3.8"
+
+      - name: Install python dependencies
+        run: |
+          python -m pip install --user --upgrade pip
+          python -m pip install tox
+          python -m pip --version
+          tox --version
+
+      - name: Run tox (python models)
+        env:
+          BIGQUERY_TEST_SERVICE_ACCOUNT_JSON: ${{ secrets.BIGQUERY_TEST_SERVICE_ACCOUNT_JSON }}
+          BIGQUERY_TEST_ALT_DATABASE: ${{ secrets.BIGQUERY_TEST_ALT_DATABASE }}
+          BIGQUERY_TEST_NO_ACCESS_DATABASE: ${{ secrets.BIGQUERY_TEST_NO_ACCESS_DATABASE }}
+          DBT_TEST_USER_1: group:buildbot@dbtlabs.com
+          DBT_TEST_USER_2: group:engineering-core-team@dbtlabs.com
+          DBT_TEST_USER_3: serviceAccount:dbt-integration-test-user@dbt-test-env.iam.gserviceaccount.com
+          DATAPROC_REGION: us-central1
+          DATAPROC_CLUSTER_NAME: dbt-test-1
+          GCS_BUCKET: dbt-ci
+        run: tox -e python-tests -- --ddtrace
+
   require-label-comment:
     runs-on: ubuntu-latest
 
-    needs: test
+    needs:
+      - test
+      - test-python
 
     permissions:
       pull-requests: write
diff --git a/dbt/adapters/bigquery/dataproc/batch.py b/dbt/adapters/bigquery/dataproc/batch.py
index 0dc54aa78..61dc3c18b 100644
--- a/dbt/adapters/bigquery/dataproc/batch.py
+++ b/dbt/adapters/bigquery/dataproc/batch.py
@@ -13,7 +13,7 @@
 from dbt.adapters.bigquery.connections import DataprocBatchConfig
 
 _BATCH_RUNNING_STATES = [Batch.State.PENDING, Batch.State.RUNNING]
-DEFAULT_JAR_FILE_URI = "gs://spark-lib/bigquery/spark-bigquery-with-dependencies_2.12-0.21.1.jar"
+DEFAULT_JAR_FILE_URI = "gs://spark-lib/bigquery/spark-bigquery-with-dependencies_2.13-0.34.0.jar"
 
 
 def create_batch_request(
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 8fd354eb5..114ebf979 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -1,5 +1,7 @@
 from typing import Dict, Union
 
+from dbt.events import AdapterLogger
+
 from dbt.adapters.base import PythonJobHelper
 from google.api_core.future.polling import POLLING_PREDICATE
 
@@ -17,6 +19,7 @@
 )
 
 OPERATION_RETRY_TIME = 10
+logger = AdapterLogger("BigQuery")
 
 
 class BaseDataProcHelper(PythonJobHelper):
@@ -122,10 +125,14 @@ def _get_job_client(self) -> dataproc_v1.BatchControllerClient:
         )
 
     def _get_batch_id(self) -> str:
-        return self.parsed_model["config"].get("batch_id")
+        model = self.parsed_model
+        default_batch_id = model["unique_id"].replace(".", "-").replace("_", "-")
+        default_batch_id += str(int(model["created_at"]))
+        return model["config"].get("batch_id", default_batch_id)
 
     def _submit_dataproc_job(self) -> Batch:
         batch_id = self._get_batch_id()
+        logger.info(f"Submitting batch job with id: {batch_id}")
         request = create_batch_request(
             batch=self._configure_batch(),
             batch_id=batch_id,
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index 16b454351..68117b06a 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -110,14 +110,16 @@ df.write \
   .mode("overwrite") \
   .format("bigquery") \
   .option("writeMethod", "indirect").option("writeDisposition", 'WRITE_TRUNCATE') \
+  {%- if partition_config is not none %}
   {%- if partition_config.data_type | lower in ('date','timestamp','datetime') %}
   .option("partitionField", "{{- partition_config.field -}}") \
   {%- if partition_config.granularity is not none %}
-  .option("partitionType", "{{- partition_config.granularity -}}") \
+  .option("partitionType", "{{- partition_config.granularity| upper -}}") \
+  {%- endif %}
   {%- endif %}
   {%- endif %}
   {%- if raw_cluster_by is not none %}
-  .option("clusteredFields", "{{- raw_cluster_by|join(',') -}}") \
+  .option("clusteredFields", "{{- raw_cluster_by | join(',') -}}") \
   {%- endif %}
   .save("{{target_relation}}")
 {% endmacro %}
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
index b67384667..1e1c2775c 100644
--- a/tests/functional/adapter/test_python_model.py
+++ b/tests/functional/adapter/test_python_model.py
@@ -216,6 +216,7 @@ def model(dbt, spark):
 """
 
 
+@pytest.mark.skip(reason="Currently failing as run_started_at is the same across dbt runs")
 class TestPythonBatchIdModels:
     @pytest.fixture(scope="class")
     def models(self):

From 0c5422c6df8f542611ee04e0c59c64ca0e0335a1 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Sun, 12 Nov 2023 09:59:06 -0800
Subject: [PATCH 745/860] fix 409 duplicate batch issue (#1020)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* fix partitioninb

* skip TestPythonBatchIdModels

* add changie

* run python integration tests separately

* run python integration tests separately

* cleanup _get_batch_id

* add space to pipe

* fix integration.yml

* add extra changie

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* replace deterministic default batch_id with uuid

* add changie
---
 .changes/unreleased/Fixes-20231111-150959.yaml | 6 ++++++
 dbt/adapters/bigquery/python_submissions.py    | 4 ++--
 tests/functional/adapter/test_python_model.py  | 3 ---
 3 files changed, 8 insertions(+), 5 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20231111-150959.yaml

diff --git a/.changes/unreleased/Fixes-20231111-150959.yaml b/.changes/unreleased/Fixes-20231111-150959.yaml
new file mode 100644
index 000000000..3d9f245a6
--- /dev/null
+++ b/.changes/unreleased/Fixes-20231111-150959.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: replace deterministic batch_id with uuid
+time: 2023-11-11T15:09:59.243797-08:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "1006"
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 114ebf979..6ea5a5564 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -1,3 +1,4 @@
+import uuid
 from typing import Dict, Union
 
 from dbt.events import AdapterLogger
@@ -126,8 +127,7 @@ def _get_job_client(self) -> dataproc_v1.BatchControllerClient:
 
     def _get_batch_id(self) -> str:
         model = self.parsed_model
-        default_batch_id = model["unique_id"].replace(".", "-").replace("_", "-")
-        default_batch_id += str(int(model["created_at"]))
+        default_batch_id = str(uuid.uuid4())
         return model["config"].get("batch_id", default_batch_id)
 
     def _submit_dataproc_job(self) -> Batch:
diff --git a/tests/functional/adapter/test_python_model.py b/tests/functional/adapter/test_python_model.py
index 1e1c2775c..bc5d8c91c 100644
--- a/tests/functional/adapter/test_python_model.py
+++ b/tests/functional/adapter/test_python_model.py
@@ -187,8 +187,6 @@ def model(dbt, spark):
 models:
   - name: python_array_batch_id
     description: A random table with a calculated column defined in python.
-    config:
-      batch_id: '{{ run_started_at.strftime("%Y-%m-%d-%H-%M-%S") }}-python-array'
     columns:
       - name: A
         description: Column A
@@ -216,7 +214,6 @@ def model(dbt, spark):
 """
 
 
-@pytest.mark.skip(reason="Currently failing as run_started_at is the same across dbt runs")
 class TestPythonBatchIdModels:
     @pytest.fixture(scope="class")
     def models(self):

From f7ec534377b59475a685f92d788027e60da4012e Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 27 Nov 2023 13:25:25 -0500
Subject: [PATCH 746/860] Update wheel requirement from ~=0.41 to ~=0.42
 (#1033)

* Update wheel requirement from ~=0.41 to ~=0.42

Updates the requirements on [wheel](https://github.com/pypa/wheel) to permit the latest version.
- [Release notes](https://github.com/pypa/wheel/releases)
- [Changelog](https://github.com/pypa/wheel/blob/main/docs/news.rst)
- [Commits](https://github.com/pypa/wheel/compare/0.41.0...0.42.0)

---
updated-dependencies:
- dependency-name: wheel
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
---
 .changes/unreleased/Dependencies-20231127-004827.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231127-004827.yaml

diff --git a/.changes/unreleased/Dependencies-20231127-004827.yaml b/.changes/unreleased/Dependencies-20231127-004827.yaml
new file mode 100644
index 000000000..a78708328
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231127-004827.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update wheel requirement from ~=0.41 to ~=0.42"
+time: 2023-11-27T00:48:27.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1033
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 48598298b..2c894e46c 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -28,4 +28,4 @@ types-pytz~=2023.3
 types-protobuf~=4.24
 types-requests~=2.31
 twine~=4.0
-wheel~=0.41
+wheel~=0.42

From 44ec2d327015f6d4f99e2288aaf28c364ac904a6 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 27 Nov 2023 14:55:59 -0500
Subject: [PATCH 747/860] Update ddtrace requirement from ~=2.1 to ~=2.2
 (#1028)

* Update ddtrace requirement from ~=2.1 to ~=2.2

Updates the requirements on [ddtrace](https://github.com/DataDog/dd-trace-py) to permit the latest version.
- [Release notes](https://github.com/DataDog/dd-trace-py/releases)
- [Changelog](https://github.com/DataDog/dd-trace-py/blob/2.x/CHANGELOG.md)
- [Commits](https://github.com/DataDog/dd-trace-py/compare/v2.1.0...v2.2.0)

---
updated-dependencies:
- dependency-name: ddtrace
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231116-001342.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231116-001342.yaml

diff --git a/.changes/unreleased/Dependencies-20231116-001342.yaml b/.changes/unreleased/Dependencies-20231116-001342.yaml
new file mode 100644
index 000000000..6af15169f
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231116-001342.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update ddtrace requirement from ~=2.1 to ~=2.2"
+time: 2023-11-16T00:13:42.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1028
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 2c894e46c..bfac809aa 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -8,7 +8,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 black~=23.10
 bumpversion~=0.6.0
 click~=8.1
-ddtrace~=2.1
+ddtrace~=2.2
 flake8~=6.1
 flaky~=3.7
 freezegun~=1.2

From 890f416d000f4328560619d705b1b0753cf94cfa Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 27 Nov 2023 16:35:35 -0500
Subject: [PATCH 748/860] Bump mypy from 1.6.1 to 1.7.0 (#1023)

* Bump mypy from 1.6.1 to 1.7.0

Bumps [mypy](https://github.com/python/mypy) from 1.6.1 to 1.7.0.
- [Changelog](https://github.com/python/mypy/blob/master/CHANGELOG.md)
- [Commits](https://github.com/python/mypy/compare/v1.6.1...v1.7.0)

---
updated-dependencies:
- dependency-name: mypy
  dependency-type: direct:development
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231113-002621.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231113-002621.yaml

diff --git a/.changes/unreleased/Dependencies-20231113-002621.yaml b/.changes/unreleased/Dependencies-20231113-002621.yaml
new file mode 100644
index 000000000..d889da51e
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231113-002621.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump mypy from 1.6.1 to 1.7.0"
+time: 2023-11-13T00:26:21.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1023
diff --git a/dev-requirements.txt b/dev-requirements.txt
index bfac809aa..045de326e 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -13,7 +13,7 @@ flake8~=6.1
 flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
-mypy==1.6.1  # patch updates have historically introduced breaking changes
+mypy==1.7.0  # patch updates have historically introduced breaking changes
 pip-tools~=7.3
 pre-commit~=3.5
 pre-commit-hooks~=4.5

From 631467a62cdefa6f8434317a1b3045000dd9d6c3 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Mon, 27 Nov 2023 18:16:57 -0500
Subject: [PATCH 749/860] Update black requirement from ~=23.10 to ~=23.11
 (#1013)

* Update black requirement from ~=23.10 to ~=23.11

Updates the requirements on [black](https://github.com/psf/black) to permit the latest version.
- [Release notes](https://github.com/psf/black/releases)
- [Changelog](https://github.com/psf/black/blob/main/CHANGES.md)
- [Commits](https://github.com/psf/black/compare/23.10.0...23.11.0)

---
updated-dependencies:
- dependency-name: black
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231109-005623.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231109-005623.yaml

diff --git a/.changes/unreleased/Dependencies-20231109-005623.yaml b/.changes/unreleased/Dependencies-20231109-005623.yaml
new file mode 100644
index 000000000..bc4ca2342
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231109-005623.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update black requirement from ~=23.10 to ~=23.11"
+time: 2023-11-09T00:56:23.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1013
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 045de326e..806d94c5d 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -5,7 +5,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor
-black~=23.10
+black~=23.11
 bumpversion~=0.6.0
 click~=8.1
 ddtrace~=2.2

From d29a230c2d35f3520396cc28c366a313e9cbe72a Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 28 Nov 2023 14:09:18 -0500
Subject: [PATCH 750/860] Update pytest-xdist requirement from ~=3.3 to ~=3.4
 (#1022)

* Update pytest-xdist requirement from ~=3.3 to ~=3.4

Updates the requirements on [pytest-xdist](https://github.com/pytest-dev/pytest-xdist) to permit the latest version.
- [Changelog](https://github.com/pytest-dev/pytest-xdist/blob/master/CHANGELOG.rst)
- [Commits](https://github.com/pytest-dev/pytest-xdist/compare/v3.3.0...v3.4.0)

---
updated-dependencies:
- dependency-name: pytest-xdist
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231113-002529.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231113-002529.yaml

diff --git a/.changes/unreleased/Dependencies-20231113-002529.yaml b/.changes/unreleased/Dependencies-20231113-002529.yaml
new file mode 100644
index 000000000..ad7272882
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231113-002529.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pytest-xdist requirement from ~=3.3 to ~=3.4"
+time: 2023-11-13T00:25:29.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1022
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 806d94c5d..6307f84c6 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -21,7 +21,7 @@ pytest~=7.4
 pytest-csv~=3.0
 pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
-pytest-xdist~=3.3
+pytest-xdist~=3.4
 pytz~=2023.3
 tox~=4.11
 types-pytz~=2023.3

From 4420e727e678996f029489420bac6ac33f10bf8b Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 28 Nov 2023 19:56:56 -0500
Subject: [PATCH 751/860] Update ddtrace requirement from ~=2.2 to ~=2.3
 (#1035)

* Update ddtrace requirement from ~=2.2 to ~=2.3

Updates the requirements on [ddtrace](https://github.com/DataDog/dd-trace-py) to permit the latest version.
- [Release notes](https://github.com/DataDog/dd-trace-py/releases)
- [Changelog](https://github.com/DataDog/dd-trace-py/blob/2.x/CHANGELOG.md)
- [Commits](https://github.com/DataDog/dd-trace-py/compare/v2.2.0...v2.3.1)

---
updated-dependencies:
- dependency-name: ddtrace
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231128-005103.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231128-005103.yaml

diff --git a/.changes/unreleased/Dependencies-20231128-005103.yaml b/.changes/unreleased/Dependencies-20231128-005103.yaml
new file mode 100644
index 000000000..205ecd1d8
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231128-005103.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update ddtrace requirement from ~=2.2 to ~=2.3"
+time: 2023-11-28T00:51:03.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1035
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 6307f84c6..a67869e86 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -8,7 +8,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 black~=23.11
 bumpversion~=0.6.0
 click~=8.1
-ddtrace~=2.2
+ddtrace~=2.3
 flake8~=6.1
 flaky~=3.7
 freezegun~=1.2

From c1c41c7ad5316a6dd7729d8d0cc2b28b0ecb7476 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Wed, 29 Nov 2023 00:47:05 -0500
Subject: [PATCH 752/860] Update pytest-xdist requirement from ~=3.4 to ~=3.5
 (#1037)

* Update pytest-xdist requirement from ~=3.4 to ~=3.5

Updates the requirements on [pytest-xdist](https://github.com/pytest-dev/pytest-xdist) to permit the latest version.
- [Release notes](https://github.com/pytest-dev/pytest-xdist/releases)
- [Changelog](https://github.com/pytest-dev/pytest-xdist/blob/master/CHANGELOG.rst)
- [Commits](https://github.com/pytest-dev/pytest-xdist/compare/v3.4.0...v3.5.0)

---
updated-dependencies:
- dependency-name: pytest-xdist
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231129-001523.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231129-001523.yaml

diff --git a/.changes/unreleased/Dependencies-20231129-001523.yaml b/.changes/unreleased/Dependencies-20231129-001523.yaml
new file mode 100644
index 000000000..e1c145ced
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231129-001523.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pytest-xdist requirement from ~=3.4 to ~=3.5"
+time: 2023-11-29T00:15:23.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1037
diff --git a/dev-requirements.txt b/dev-requirements.txt
index a67869e86..c4b577d2c 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -21,7 +21,7 @@ pytest~=7.4
 pytest-csv~=3.0
 pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
-pytest-xdist~=3.4
+pytest-xdist~=3.5
 pytz~=2023.3
 tox~=4.11
 types-pytz~=2023.3

From 67feec4741970d8934d78edd953e71481902fc48 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Wed, 29 Nov 2023 12:05:41 -0500
Subject: [PATCH 753/860] Bump mypy from 1.7.0 to 1.7.1 (#1034)

* Bump mypy from 1.7.0 to 1.7.1

Bumps [mypy](https://github.com/python/mypy) from 1.7.0 to 1.7.1.
- [Changelog](https://github.com/python/mypy/blob/master/CHANGELOG.md)
- [Commits](https://github.com/python/mypy/compare/v1.7.0...v1.7.1)

---
updated-dependencies:
- dependency-name: mypy
  dependency-type: direct:development
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231128-005012.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231128-005012.yaml

diff --git a/.changes/unreleased/Dependencies-20231128-005012.yaml b/.changes/unreleased/Dependencies-20231128-005012.yaml
new file mode 100644
index 000000000..236ca1e52
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231128-005012.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump mypy from 1.7.0 to 1.7.1"
+time: 2023-11-28T00:50:12.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1034
diff --git a/dev-requirements.txt b/dev-requirements.txt
index c4b577d2c..5b1aa1ad0 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -13,7 +13,7 @@ flake8~=6.1
 flaky~=3.7
 freezegun~=1.2
 ipdb~=0.13.13
-mypy==1.7.0  # patch updates have historically introduced breaking changes
+mypy==1.7.1  # patch updates have historically introduced breaking changes
 pip-tools~=7.3
 pre-commit~=3.5
 pre-commit-hooks~=4.5

From c4893326ce6b628d27d8205fcad459c09dd3daa2 Mon Sep 17 00:00:00 2001
From: Michelle Ark <MichelleArk@users.noreply.github.com>
Date: Fri, 1 Dec 2023 17:49:38 -0500
Subject: [PATCH 754/860] add tests for --empty flag (#1030)

* add tests for --empty flag

* restore dev-requirements
---
 .changes/unreleased/Under the Hood-20231117-121214.yaml | 6 ++++++
 tests/functional/adapter/empty/test_empty.py            | 5 +++++
 2 files changed, 11 insertions(+)
 create mode 100644 .changes/unreleased/Under the Hood-20231117-121214.yaml
 create mode 100644 tests/functional/adapter/empty/test_empty.py

diff --git a/.changes/unreleased/Under the Hood-20231117-121214.yaml b/.changes/unreleased/Under the Hood-20231117-121214.yaml
new file mode 100644
index 000000000..61b0617ad
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20231117-121214.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Add tests for --empty flag
+time: 2023-11-17T12:12:14.643365-05:00
+custom:
+  Author: michelleark
+  Issue: "1029"
diff --git a/tests/functional/adapter/empty/test_empty.py b/tests/functional/adapter/empty/test_empty.py
new file mode 100644
index 000000000..c224c51df
--- /dev/null
+++ b/tests/functional/adapter/empty/test_empty.py
@@ -0,0 +1,5 @@
+from dbt.tests.adapter.empty.test_empty import BaseTestEmpty
+
+
+class TestBigQueryEmpty(BaseTestEmpty):
+    pass

From ef1fee52601c1b91c41a7dc4071211dac6366372 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 12 Dec 2023 13:50:37 -0800
Subject: [PATCH 755/860] Update freezegun requirement from ~=1.2 to ~=1.3
 (#1040)

* Update freezegun requirement from ~=1.2 to ~=1.3

Updates the requirements on [freezegun](https://github.com/spulec/freezegun) to permit the latest version.
- [Release notes](https://github.com/spulec/freezegun/releases)
- [Changelog](https://github.com/spulec/freezegun/blob/master/CHANGELOG)
- [Commits](https://github.com/spulec/freezegun/compare/1.2.0...1.3.0)

---
updated-dependencies:
- dependency-name: freezegun
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
---
 .changes/unreleased/Dependencies-20231204-003807.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231204-003807.yaml

diff --git a/.changes/unreleased/Dependencies-20231204-003807.yaml b/.changes/unreleased/Dependencies-20231204-003807.yaml
new file mode 100644
index 000000000..18b9bb618
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231204-003807.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update freezegun requirement from ~=1.2 to ~=1.3"
+time: 2023-12-04T00:38:07.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1040
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 5b1aa1ad0..083341006 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -11,7 +11,7 @@ click~=8.1
 ddtrace~=2.3
 flake8~=6.1
 flaky~=3.7
-freezegun~=1.2
+freezegun~=1.3
 ipdb~=0.13.13
 mypy==1.7.1  # patch updates have historically introduced breaking changes
 pip-tools~=7.3

From 06851679f75d18ece98c95d4eb2a0ddd16544f4d Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 12 Dec 2023 21:25:40 -0800
Subject: [PATCH 756/860] Update black requirement from ~=23.11 to ~=23.12
 (#1056)

* Update black requirement from ~=23.11 to ~=23.12

Updates the requirements on [black](https://github.com/psf/black) to permit the latest version.
- [Release notes](https://github.com/psf/black/releases)
- [Changelog](https://github.com/psf/black/blob/main/CHANGES.md)
- [Commits](https://github.com/psf/black/compare/23.11.0...23.12.0)

---
updated-dependencies:
- dependency-name: black
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
---
 .changes/unreleased/Dependencies-20231213-003845.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231213-003845.yaml

diff --git a/.changes/unreleased/Dependencies-20231213-003845.yaml b/.changes/unreleased/Dependencies-20231213-003845.yaml
new file mode 100644
index 000000000..93618b5e0
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231213-003845.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update black requirement from ~=23.11 to ~=23.12"
+time: 2023-12-13T00:38:45.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1056
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 083341006..a101e8b3e 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -5,7 +5,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=
 
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor
-black~=23.11
+black~=23.12
 bumpversion~=0.6.0
 click~=8.1
 ddtrace~=2.3

From f2804c0f3184ebe36cada9113802051c75c347d0 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Wed, 10 Jan 2024 14:44:41 -0800
Subject: [PATCH 757/860] feature/decouple adapters from core (#1026)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* init

* init

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* update bq feature branch to work against core feature

* add changie

* remove  --force-reinstall from make

* update dev-requirements.txt to point to dbt-core main

* fix imports in _materialized_view.py
---
 .../unreleased/Features-20240102-152030.yaml  |  6 ++
 dbt/adapters/bigquery/connections.py          | 35 ++++----
 dbt/adapters/bigquery/dataset.py              |  2 +-
 dbt/adapters/bigquery/gcloud.py               |  8 +-
 dbt/adapters/bigquery/impl.py                 | 63 +++++++-------
 dbt/adapters/bigquery/python_submissions.py   |  2 +-
 dbt/adapters/bigquery/relation.py             | 18 ++--
 .../bigquery/relation_configs/_base.py        | 18 ++--
 .../bigquery/relation_configs/_cluster.py     | 12 +--
 .../relation_configs/_materialized_view.py    | 26 +++---
 .../bigquery/relation_configs/_options.py     | 17 ++--
 .../bigquery/relation_configs/_partition.py   | 16 ++--
 dbt/adapters/bigquery/utility.py              |  4 +-
 .../relations/materialized_view/alter.sql     |  2 +-
 .../relations/materialized_view/create.sql    |  2 +-
 .../relations/materialized_view/replace.sql   |  2 +-
 dev-requirements.txt                          |  1 -
 .../test_describe_relation.py                 |  2 +-
 .../adapter/materialized_view_tests/_mixin.py |  2 +-
 tests/unit/test_bigquery_adapter.py           | 84 ++++++++++++-------
 .../unit/test_bigquery_connection_manager.py  |  8 +-
 tests/unit/utils.py                           | 32 ++++---
 tox.ini                                       |  2 +-
 23 files changed, 202 insertions(+), 162 deletions(-)
 create mode 100644 .changes/unreleased/Features-20240102-152030.yaml

diff --git a/.changes/unreleased/Features-20240102-152030.yaml b/.changes/unreleased/Features-20240102-152030.yaml
new file mode 100644
index 000000000..81c683de0
--- /dev/null
+++ b/.changes/unreleased/Features-20240102-152030.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Update base adapter references as part of decoupling migration
+time: 2024-01-02T15:20:30.038221-08:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "1067"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index de84e4bf8..cb933baed 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -5,7 +5,9 @@
 from contextlib import contextmanager
 from dataclasses import dataclass, field
 
-from dbt.events.contextvars import get_node_info
+from dbt.common.invocation import get_invocation_id
+
+from dbt.common.events.contextvars import get_node_info
 from mashumaro.helper import pass_through
 
 from functools import lru_cache
@@ -25,23 +27,21 @@
 )
 
 from dbt.adapters.bigquery import gcloud
-from dbt.clients import agate_helper
-from dbt.config.profile import INVALID_PROFILE_MESSAGE
-from dbt.tracking import active_user
-from dbt.contracts.connection import ConnectionState, AdapterResponse
-from dbt.exceptions import (
-    FailedToConnectError,
+from dbt.common.clients import agate_helper
+from dbt.adapters.contracts.connection import ConnectionState, AdapterResponse
+from dbt.common.exceptions import (
     DbtRuntimeError,
-    DbtDatabaseError,
-    DbtProfileError,
+    DbtConfigError,
 )
+from dbt.common.exceptions import DbtDatabaseError
+from dbt.adapters.exceptions.connection import FailedToConnectError
 from dbt.adapters.base import BaseConnectionManager, Credentials
-from dbt.events import AdapterLogger
-from dbt.events.functions import fire_event
-from dbt.events.types import SQLQuery
-from dbt.version import __version__ as dbt_version
+from dbt.adapters.events.logging import AdapterLogger
+from dbt.adapters.events.types import SQLQuery
+from dbt.common.events.functions import fire_event
+from dbt.adapters.bigquery import __version__ as dbt_version
 
-from dbt.dataclass_schema import ExtensibleDbtClassMixin, StrEnum
+from dbt.common.dataclass_schema import ExtensibleDbtClassMixin, StrEnum
 
 logger = AdapterLogger("BigQuery")
 
@@ -85,7 +85,7 @@ def get_bigquery_defaults(scopes=None) -> Tuple[Any, Optional[str]]:
         credentials, _ = google.auth.default(scopes=scopes)
         return credentials, _
     except google.auth.exceptions.DefaultCredentialsError as e:
-        raise DbtProfileError(INVALID_PROFILE_MESSAGE.format(error_string=e))
+        raise DbtConfigError(f"Failed to authenticate with supplied credentials\nerror:\n{e}")
 
 
 class Priority(StrEnum):
@@ -382,7 +382,7 @@ def get_bigquery_client(cls, profile_credentials):
         execution_project = profile_credentials.execution_project
         location = getattr(profile_credentials, "location", None)
 
-        info = client_info.ClientInfo(user_agent=f"dbt-{dbt_version}")
+        info = client_info.ClientInfo(user_agent=f"dbt-bigquery-{dbt_version.version}")
         return google.cloud.bigquery.Client(
             execution_project,
             creds,
@@ -470,8 +470,7 @@ def raw_execute(
 
         labels = self.get_labels_from_query_comment()
 
-        if active_user:
-            labels["dbt_invocation_id"] = active_user.invocation_id
+        labels["dbt_invocation_id"] = get_invocation_id()
 
         job_params = {
             "use_legacy_sql": use_legacy_sql,
diff --git a/dbt/adapters/bigquery/dataset.py b/dbt/adapters/bigquery/dataset.py
index c886637d7..4ecd6daa5 100644
--- a/dbt/adapters/bigquery/dataset.py
+++ b/dbt/adapters/bigquery/dataset.py
@@ -1,7 +1,7 @@
 from typing import List
 from google.cloud.bigquery import Dataset, AccessEntry
 
-from dbt.events import AdapterLogger
+from dbt.adapters.events.logging import AdapterLogger
 
 logger = AdapterLogger("BigQuery")
 
diff --git a/dbt/adapters/bigquery/gcloud.py b/dbt/adapters/bigquery/gcloud.py
index c303097bc..0a08f734d 100644
--- a/dbt/adapters/bigquery/gcloud.py
+++ b/dbt/adapters/bigquery/gcloud.py
@@ -1,6 +1,6 @@
-from dbt.events import AdapterLogger
-import dbt.exceptions
-from dbt.clients.system import run_cmd
+from dbt.adapters.events.logging import AdapterLogger
+import dbt.common.exceptions
+from dbt.common.clients.system import run_cmd
 
 NOT_INSTALLED_MSG = """
 dbt requires the gcloud SDK to be installed to authenticate with BigQuery.
@@ -25,4 +25,4 @@ def setup_default_credentials():
     if gcloud_installed():
         run_cmd(".", ["gcloud", "auth", "application-default", "login"])
     else:
-        raise dbt.exceptions.DbtRuntimeError(NOT_INSTALLED_MSG)
+        raise dbt.common.exceptions.DbtRuntimeError(NOT_INSTALLED_MSG)
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 7d9b003b8..03cfd3561 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -1,11 +1,15 @@
 from dataclasses import dataclass
 import json
 import threading
+from multiprocessing.context import SpawnContext
+
 import time
-from typing import Any, Dict, List, Optional, Type, Set, Union
+from typing import Any, Dict, List, Optional, Type, Set, Union, FrozenSet, Tuple, Iterable
 
 import agate
-from dbt import ui  # type: ignore
+from dbt.adapters.contracts.relation import RelationConfig
+
+import dbt.common.exceptions.base
 from dbt.adapters.base import (  # type: ignore
     AdapterConfig,
     BaseAdapter,
@@ -17,17 +21,15 @@
     available,
 )
 from dbt.adapters.cache import _make_ref_key_dict  # type: ignore
-import dbt.clients.agate_helper
-from dbt.contracts.connection import AdapterResponse
-from dbt.contracts.graph.manifest import Manifest
-from dbt.contracts.graph.nodes import ColumnLevelConstraint, ConstraintType, ModelLevelConstraint  # type: ignore
-from dbt.dataclass_schema import dbtClassMixin
-import dbt.deprecations
-from dbt.events import AdapterLogger
-from dbt.events.functions import fire_event
-from dbt.events.types import SchemaCreation, SchemaDrop
-import dbt.exceptions
-from dbt.utils import filter_null_values
+import dbt.common.clients.agate_helper
+from dbt.adapters.contracts.connection import AdapterResponse
+from dbt.common.contracts.constraints import ColumnLevelConstraint, ConstraintType, ModelLevelConstraint  # type: ignore
+from dbt.common.dataclass_schema import dbtClassMixin
+from dbt.adapters.events.logging import AdapterLogger
+from dbt.common.events.functions import fire_event
+from dbt.adapters.events.types import SchemaCreation, SchemaDrop
+import dbt.common.exceptions
+from dbt.common.utils import filter_null_values
 import google.api_core
 import google.auth
 import google.oauth2
@@ -116,8 +118,8 @@ class BigQueryAdapter(BaseAdapter):
         ConstraintType.foreign_key: ConstraintSupport.ENFORCED,
     }
 
-    def __init__(self, config) -> None:
-        super().__init__(config)
+    def __init__(self, config, mp_context: SpawnContext) -> None:
+        super().__init__(config, mp_context)
         self.connections: BigQueryConnectionManager = self.connections
 
     ###
@@ -145,7 +147,9 @@ def drop_relation(self, relation: BigQueryRelation) -> None:
         conn.handle.delete_table(table_ref, not_found_ok=True)
 
     def truncate_relation(self, relation: BigQueryRelation) -> None:
-        raise dbt.exceptions.NotImplementedError("`truncate` is not implemented for this adapter!")
+        raise dbt.common.exceptions.base.NotImplementedError(
+            "`truncate` is not implemented for this adapter!"
+        )
 
     def rename_relation(
         self, from_relation: BigQueryRelation, to_relation: BigQueryRelation
@@ -160,7 +164,7 @@ def rename_relation(
             or from_relation.type == RelationType.View
             or to_relation.type == RelationType.View
         ):
-            raise dbt.exceptions.DbtRuntimeError(
+            raise dbt.common.exceptions.DbtRuntimeError(
                 "Renaming of views is not currently supported in BigQuery"
             )
 
@@ -386,7 +390,7 @@ def copy_table(self, source, destination, materialization):
         elif materialization == "table":
             write_disposition = WRITE_TRUNCATE
         else:
-            raise dbt.exceptions.CompilationError(
+            raise dbt.common.exceptions.CompilationError(
                 'Copy table materialization must be "copy" or "table", but '
                 f"config.get('copy_materialization', 'table') was "
                 f"{materialization}"
@@ -433,11 +437,11 @@ def poll_until_job_completes(cls, job, timeout):
             job.reload()
 
         if job.state != "DONE":
-            raise dbt.exceptions.DbtRuntimeError("BigQuery Timeout Exceeded")
+            raise dbt.common.exceptions.DbtRuntimeError("BigQuery Timeout Exceeded")
 
         elif job.error_result:
             message = "\n".join(error["message"].strip() for error in job.errors)
-            raise dbt.exceptions.DbtRuntimeError(message)
+            raise dbt.common.exceptions.DbtRuntimeError(message)
 
     def _bq_table_to_relation(self, bq_table) -> Union[BigQueryRelation, None]:
         if bq_table is None:
@@ -454,15 +458,14 @@ def _bq_table_to_relation(self, bq_table) -> Union[BigQueryRelation, None]:
     @classmethod
     def warning_on_hooks(cls, hook_type):
         msg = "{} is not supported in bigquery and will be ignored"
-        warn_msg = dbt.ui.color(msg, ui.COLOR_FG_YELLOW)
-        logger.info(warn_msg)
+        logger.info(msg)
 
     @available
     def add_query(self, sql, auto_begin=True, bindings=None, abridge_sql_log=False):
         if self.nice_connection_name() in ["on-run-start", "on-run-end"]:
             self.warning_on_hooks(self.nice_connection_name())
         else:
-            raise dbt.exceptions.NotImplementedError(
+            raise dbt.common.exceptions.base.NotImplementedError(
                 "`add_query` is not implemented for this adapter!"
             )
 
@@ -679,14 +682,16 @@ def upload_file(
             self.poll_until_job_completes(job, timeout)
 
     @classmethod
-    def _catalog_filter_table(cls, table: agate.Table, manifest: Manifest) -> agate.Table:
+    def _catalog_filter_table(
+        cls, table: agate.Table, used_schemas: FrozenSet[Tuple[str, str]]
+    ) -> agate.Table:
         table = table.rename(
             column_names={col.name: col.name.replace("__", ":") for col in table.columns}
         )
-        return super()._catalog_filter_table(table, manifest)
+        return super()._catalog_filter_table(table, used_schemas)
 
-    def _get_catalog_schemas(self, manifest: Manifest) -> SchemaSearchMap:
-        candidates = super()._get_catalog_schemas(manifest)
+    def _get_catalog_schemas(self, relation_config: Iterable[RelationConfig]) -> SchemaSearchMap:
+        candidates = super()._get_catalog_schemas(relation_config)
         db_schemas: Dict[str, Set[str]] = {}
         result = SchemaSearchMap()
 
@@ -772,7 +777,7 @@ def describe_relation(
             bq_table = self.get_bq_table(relation)
             parser = BigQueryMaterializedViewConfig
         else:
-            raise dbt.exceptions.DbtRuntimeError(
+            raise dbt.common.exceptions.DbtRuntimeError(
                 f"The method `BigQueryAdapter.describe_relation` is not implemented "
                 f"for the relation type: {relation.type}"
             )
@@ -838,7 +843,7 @@ def string_add_sql(
         elif location == "prepend":
             return f"concat('{value}', {add_to})"
         else:
-            raise dbt.exceptions.DbtRuntimeError(
+            raise dbt.common.exceptions.DbtRuntimeError(
                 f'Got an unexpected location value of "{location}"'
             )
 
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 6ea5a5564..065c65d8b 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -1,7 +1,7 @@
 import uuid
 from typing import Dict, Union
 
-from dbt.events import AdapterLogger
+from dbt.adapters.events.logging import AdapterLogger
 
 from dbt.adapters.base import PythonJobHelper
 from google.api_core.future.polling import POLLING_PREDICATE
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index c14dba238..c25ef0a67 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -2,7 +2,6 @@
 from typing import FrozenSet, Optional, TypeVar
 
 from itertools import chain, islice
-from dbt.context.providers import RuntimeConfigObject
 from dbt.adapters.base.relation import BaseRelation, ComponentName, InformationSchema
 from dbt.adapters.relation_configs import RelationConfigChangeAction
 from dbt.adapters.bigquery.relation_configs import (
@@ -12,10 +11,9 @@
     BigQueryOptionsConfigChange,
     BigQueryPartitionConfigChange,
 )
-from dbt.contracts.graph.nodes import ModelNode
-from dbt.contracts.relation import RelationType
-from dbt.exceptions import CompilationError
-from dbt.utils import filter_null_values
+from dbt.adapters.contracts.relation import RelationType, RelationConfig
+from dbt.common.exceptions import CompilationError
+from dbt.common.utils.dict import filter_null_values
 
 
 Self = TypeVar("Self", bound="BigQueryRelation")
@@ -63,19 +61,19 @@ def dataset(self):
         return self.schema
 
     @classmethod
-    def materialized_view_from_model_node(
-        cls, model_node: ModelNode
+    def materialized_view_from_relation_config(
+        cls, relation_config: RelationConfig
     ) -> BigQueryMaterializedViewConfig:
-        return BigQueryMaterializedViewConfig.from_model_node(model_node)  # type: ignore
+        return BigQueryMaterializedViewConfig.from_relation_config(relation_config)  # type: ignore
 
     @classmethod
     def materialized_view_config_changeset(
         cls,
         existing_materialized_view: BigQueryMaterializedViewConfig,
-        runtime_config: RuntimeConfigObject,
+        relation_config: RelationConfig,
     ) -> Optional[BigQueryMaterializedViewConfigChangeset]:
         config_change_collection = BigQueryMaterializedViewConfigChangeset()
-        new_materialized_view = cls.materialized_view_from_model_node(runtime_config.model)
+        new_materialized_view = cls.materialized_view_from_relation_config(relation_config)
 
         if new_materialized_view.options != existing_materialized_view.options:
             config_change_collection.options = BigQueryOptionsConfigChange(
diff --git a/dbt/adapters/bigquery/relation_configs/_base.py b/dbt/adapters/bigquery/relation_configs/_base.py
index c92f7c01a..be34a08ac 100644
--- a/dbt/adapters/bigquery/relation_configs/_base.py
+++ b/dbt/adapters/bigquery/relation_configs/_base.py
@@ -1,17 +1,17 @@
 from dataclasses import dataclass
-from typing import Optional
+from typing import Optional, Dict
 
 import agate
 from dbt.adapters.base.relation import Policy
 from dbt.adapters.relation_configs import RelationConfigBase
 from google.cloud.bigquery import Table as BigQueryTable
+from typing_extensions import Self
 
 from dbt.adapters.bigquery.relation_configs._policies import (
     BigQueryIncludePolicy,
     BigQueryQuotePolicy,
 )
-from dbt.contracts.graph.nodes import ModelNode
-from dbt.contracts.relation import ComponentName
+from dbt.adapters.contracts.relation import ComponentName, RelationConfig
 
 
 @dataclass(frozen=True, eq=True, unsafe_hash=True)
@@ -25,25 +25,25 @@ def quote_policy(cls) -> Policy:
         return BigQueryQuotePolicy()
 
     @classmethod
-    def from_model_node(cls, model_node: ModelNode) -> "BigQueryBaseRelationConfig":
-        relation_config = cls.parse_model_node(model_node)
-        relation = cls.from_dict(relation_config)
+    def from_relation_config(cls, relation_config: RelationConfig) -> Self:
+        relation_config_dict = cls.parse_relation_config(relation_config)
+        relation = cls.from_dict(relation_config_dict)
         return relation  # type: ignore
 
     @classmethod
-    def parse_model_node(cls, model_node: ModelNode) -> dict:
+    def parse_relation_config(cls, relation_config: RelationConfig) -> Dict:
         raise NotImplementedError(
             "`parse_model_node()` needs to be implemented on this RelationConfigBase instance"
         )
 
     @classmethod
-    def from_bq_table(cls, table: BigQueryTable) -> "BigQueryBaseRelationConfig":
+    def from_bq_table(cls, table: BigQueryTable) -> Self:
         relation_config = cls.parse_bq_table(table)
         relation = cls.from_dict(relation_config)
         return relation  # type: ignore
 
     @classmethod
-    def parse_bq_table(cls, table: BigQueryTable) -> dict:
+    def parse_bq_table(cls, table: BigQueryTable) -> Dict:
         raise NotImplementedError("`parse_bq_table()` is not implemented for this relation type")
 
     @classmethod
diff --git a/dbt/adapters/bigquery/relation_configs/_cluster.py b/dbt/adapters/bigquery/relation_configs/_cluster.py
index c7161a96a..53092cb35 100644
--- a/dbt/adapters/bigquery/relation_configs/_cluster.py
+++ b/dbt/adapters/bigquery/relation_configs/_cluster.py
@@ -2,8 +2,9 @@
 from typing import Any, Dict, FrozenSet, Optional
 
 from dbt.adapters.relation_configs import RelationConfigChange
-from dbt.contracts.graph.nodes import ModelNode
+from dbt.adapters.contracts.relation import RelationConfig
 from google.cloud.bigquery import Table as BigQueryTable
+from typing_extensions import Self
 
 from dbt.adapters.bigquery.relation_configs._base import BigQueryBaseRelationConfig
 
@@ -22,16 +23,15 @@ class BigQueryClusterConfig(BigQueryBaseRelationConfig):
     fields: FrozenSet[str]
 
     @classmethod
-    def from_dict(cls, config_dict: Dict[str, Any]) -> "BigQueryClusterConfig":
+    def from_dict(cls, config_dict: Dict[str, Any]) -> Self:
         kwargs_dict = {"fields": config_dict.get("fields")}
-        cluster: "BigQueryClusterConfig" = super().from_dict(kwargs_dict)  # type: ignore
-        return cluster
+        return super().from_dict(kwargs_dict)  # type: ignore
 
     @classmethod
-    def parse_model_node(cls, model_node: ModelNode) -> Dict[str, Any]:
+    def parse_relation_config(cls, relation_config: RelationConfig) -> Dict[str, Any]:
         config_dict = {}
 
-        if cluster_by := model_node.config.extra.get("cluster_by"):
+        if cluster_by := relation_config.config.extra.get("cluster_by"):  # type: ignore
             # users may input a single field as a string
             if isinstance(cluster_by, str):
                 cluster_by = [cluster_by]
diff --git a/dbt/adapters/bigquery/relation_configs/_materialized_view.py b/dbt/adapters/bigquery/relation_configs/_materialized_view.py
index a9baa3ab0..fd0c191c3 100644
--- a/dbt/adapters/bigquery/relation_configs/_materialized_view.py
+++ b/dbt/adapters/bigquery/relation_configs/_materialized_view.py
@@ -1,8 +1,10 @@
 from dataclasses import dataclass
 from typing import Any, Dict, Optional
 
-from dbt.contracts.graph.nodes import ModelNode
-from dbt.contracts.relation import ComponentName
+from dbt.adapters.contracts.relation import (
+    RelationConfig,
+    ComponentName,
+)
 from google.cloud.bigquery import Table as BigQueryTable
 
 from dbt.adapters.bigquery.relation_configs._base import BigQueryBaseRelationConfig
@@ -63,21 +65,23 @@ def from_dict(cls, config_dict: Dict[str, Any]) -> "BigQueryMaterializedViewConf
         return materialized_view
 
     @classmethod
-    def parse_model_node(cls, model_node: ModelNode) -> Dict[str, Any]:
+    def parse_relation_config(cls, relation_config: RelationConfig) -> Dict[str, Any]:
         config_dict = {
-            "table_id": model_node.identifier,
-            "dataset_id": model_node.schema,
-            "project_id": model_node.database,
+            "table_id": relation_config.identifier,
+            "dataset_id": relation_config.schema,
+            "project_id": relation_config.database,
             # despite this being a foreign object, there will always be options because of defaults
-            "options": BigQueryOptionsConfig.parse_model_node(model_node),
+            "options": BigQueryOptionsConfig.parse_relation_config(relation_config),
         }
 
         # optional
-        if "partition_by" in model_node.config:
-            config_dict.update({"partition": PartitionConfig.parse_model_node(model_node)})
+        if "partition_by" in relation_config.config:
+            config_dict.update({"partition": PartitionConfig.parse_model_node(relation_config)})
 
-        if "cluster_by" in model_node.config:
-            config_dict.update({"cluster": BigQueryClusterConfig.parse_model_node(model_node)})
+        if "cluster_by" in relation_config.config:
+            config_dict.update(
+                {"cluster": BigQueryClusterConfig.parse_relation_config(relation_config)}
+            )
 
         return config_dict
 
diff --git a/dbt/adapters/bigquery/relation_configs/_options.py b/dbt/adapters/bigquery/relation_configs/_options.py
index 72f9d73e6..f0272df08 100644
--- a/dbt/adapters/bigquery/relation_configs/_options.py
+++ b/dbt/adapters/bigquery/relation_configs/_options.py
@@ -3,8 +3,9 @@
 from typing import Any, Dict, Optional
 
 from dbt.adapters.relation_configs import RelationConfigChange
-from dbt.contracts.graph.nodes import ModelNode
+from dbt.adapters.contracts.relation import RelationConfig
 from google.cloud.bigquery import Table as BigQueryTable
+from typing_extensions import Self
 
 from dbt.adapters.bigquery.relation_configs._base import BigQueryBaseRelationConfig
 from dbt.adapters.bigquery.utility import bool_setting, float_setting, sql_escape
@@ -78,7 +79,7 @@ def formatted_option(name: str) -> Optional[Any]:
         return options
 
     @classmethod
-    def from_dict(cls, config_dict: Dict[str, Any]) -> "BigQueryOptionsConfig":
+    def from_dict(cls, config_dict: Dict[str, Any]) -> Self:
         setting_formatters = {
             "enable_refresh": bool_setting,
             "refresh_interval_minutes": float_setting,
@@ -102,13 +103,13 @@ def formatted_setting(name: str) -> Any:
         if kwargs_dict["enable_refresh"] is False:
             kwargs_dict.update({"refresh_interval_minutes": None, "max_staleness": None})
 
-        options: "BigQueryOptionsConfig" = super().from_dict(kwargs_dict)  # type: ignore
+        options: Self = super().from_dict(kwargs_dict)  # type: ignore
         return options
 
     @classmethod
-    def parse_model_node(cls, model_node: ModelNode) -> Dict[str, Any]:
+    def parse_relation_config(cls, relation_config: RelationConfig) -> Dict[str, Any]:
         config_dict = {
-            option: model_node.config.extra.get(option)
+            option: relation_config.config.extra.get(option)  # type: ignore
             for option in [
                 "enable_refresh",
                 "refresh_interval_minutes",
@@ -121,11 +122,13 @@ def parse_model_node(cls, model_node: ModelNode) -> Dict[str, Any]:
         }
 
         # update dbt-specific versions of these settings
-        if hours_to_expiration := model_node.config.extra.get("hours_to_expiration"):
+        if hours_to_expiration := relation_config.config.extra.get(  # type: ignore
+            "hours_to_expiration"
+        ):  # type: ignore
             config_dict.update(
                 {"expiration_timestamp": datetime.now() + timedelta(hours=hours_to_expiration)}
             )
-        if not model_node.config.persist_docs:
+        if not relation_config.config.persist_docs:  # type: ignore
             del config_dict["description"]
 
         return config_dict
diff --git a/dbt/adapters/bigquery/relation_configs/_partition.py b/dbt/adapters/bigquery/relation_configs/_partition.py
index 0d0ee23a1..0fe816359 100644
--- a/dbt/adapters/bigquery/relation_configs/_partition.py
+++ b/dbt/adapters/bigquery/relation_configs/_partition.py
@@ -1,10 +1,10 @@
 from dataclasses import dataclass
 from typing import Any, Dict, List, Optional
 
+import dbt.common.exceptions
 from dbt.adapters.relation_configs import RelationConfigChange
-from dbt.contracts.graph.nodes import ModelNode
-from dbt.dataclass_schema import dbtClassMixin, ValidationError
-import dbt.exceptions
+from dbt.adapters.contracts.relation import RelationConfig
+from dbt.common.dataclass_schema import dbtClassMixin, ValidationError
 from google.cloud.bigquery.table import Table as BigQueryTable
 
 
@@ -92,16 +92,18 @@ def parse(cls, raw_partition_by) -> Optional["PartitionConfig"]:
                 }
             )
         except ValidationError as exc:
-            raise dbt.exceptions.DbtValidationError("Could not parse partition config") from exc
+            raise dbt.common.exceptions.base.DbtValidationError(
+                "Could not parse partition config"
+            ) from exc
         except TypeError:
-            raise dbt.exceptions.CompilationError(
+            raise dbt.common.exceptions.CompilationError(
                 f"Invalid partition_by config:\n"
                 f"  Got: {raw_partition_by}\n"
                 f'  Expected a dictionary with "field" and "data_type" keys'
             )
 
     @classmethod
-    def parse_model_node(cls, model_node: ModelNode) -> Dict[str, Any]:
+    def parse_model_node(cls, relation_config: RelationConfig) -> Dict[str, Any]:
         """
         Parse model node into a raw config for `PartitionConfig.parse`
 
@@ -109,7 +111,7 @@ def parse_model_node(cls, model_node: ModelNode) -> Dict[str, Any]:
             This doesn't currently collect `time_ingestion_partitioning` and `copy_partitions`
             because this was built for materialized views, which do not support those settings.
         """
-        config_dict = model_node.config.extra.get("partition_by")
+        config_dict = relation_config.config.extra.get("partition_by")  # type: ignore
         if "time_ingestion_partitioning" in config_dict:
             del config_dict["time_ingestion_partitioning"]
         if "copy_partitions" in config_dict:
diff --git a/dbt/adapters/bigquery/utility.py b/dbt/adapters/bigquery/utility.py
index c66b2d97d..5d9c3de12 100644
--- a/dbt/adapters/bigquery/utility.py
+++ b/dbt/adapters/bigquery/utility.py
@@ -1,7 +1,7 @@
 import json
 from typing import Any, Optional
 
-import dbt.exceptions
+import dbt.common.exceptions
 
 
 def bool_setting(value: Optional[Any] = None) -> Optional[bool]:
@@ -41,5 +41,5 @@ def float_setting(value: Optional[Any] = None) -> Optional[float]:
 
 def sql_escape(string):
     if not isinstance(string, str):
-        raise dbt.exceptions.CompilationError(f"cannot escape a non-string: {string}")
+        raise dbt.common.exceptions.CompilationError(f"cannot escape a non-string: {string}")
     return json.dumps(string)[1:-1]
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/alter.sql b/dbt/include/bigquery/macros/relations/materialized_view/alter.sql
index 7320addde..e71f869ae 100644
--- a/dbt/include/bigquery/macros/relations/materialized_view/alter.sql
+++ b/dbt/include/bigquery/macros/relations/materialized_view/alter.sql
@@ -20,6 +20,6 @@
 
 {% macro bigquery__get_materialized_view_configuration_changes(existing_relation, new_config) %}
     {% set _existing_materialized_view = adapter.describe_relation(existing_relation) %}
-    {% set _configuration_changes = existing_relation.materialized_view_config_changeset(_existing_materialized_view, new_config) %}
+    {% set _configuration_changes = existing_relation.materialized_view_config_changeset(_existing_materialized_view, new_config.model) %}
     {% do return(_configuration_changes) %}
 {% endmacro %}
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/create.sql b/dbt/include/bigquery/macros/relations/materialized_view/create.sql
index d2b6c5be5..d3e8c7685 100644
--- a/dbt/include/bigquery/macros/relations/materialized_view/create.sql
+++ b/dbt/include/bigquery/macros/relations/materialized_view/create.sql
@@ -1,6 +1,6 @@
 {% macro bigquery__get_create_materialized_view_as_sql(relation, sql) %}
 
-    {%- set materialized_view = adapter.Relation.materialized_view_from_model_node(config.model) -%}
+    {%- set materialized_view = adapter.Relation.materialized_view_from_relation_config(config.model) -%}
 
     create materialized view if not exists {{ relation }}
     {% if materialized_view.partition %}{{ partition_by(materialized_view.partition) }}{% endif %}
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/replace.sql b/dbt/include/bigquery/macros/relations/materialized_view/replace.sql
index 4fdfd21f6..2e4a0b69f 100644
--- a/dbt/include/bigquery/macros/relations/materialized_view/replace.sql
+++ b/dbt/include/bigquery/macros/relations/materialized_view/replace.sql
@@ -1,6 +1,6 @@
 {% macro bigquery__get_replace_materialized_view_as_sql(relation, sql) %}
 
-    {%- set materialized_view = adapter.Relation.materialized_view_from_model_node(config.model) -%}
+    {%- set materialized_view = adapter.Relation.materialized_view_from_relation_config(config.model) -%}
 
     create or replace materialized view if not exists {{ relation }}
     {% if materialized_view.partition %}{{ partition_by(materialized_view.partition) }}{% endif %}
diff --git a/dev-requirements.txt b/dev-requirements.txt
index a101e8b3e..bd45c99a1 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -2,7 +2,6 @@
 # TODO: how to automate switching from develop to version branches?
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
-
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor
 black~=23.12
diff --git a/tests/functional/adapter/describe_relation/test_describe_relation.py b/tests/functional/adapter/describe_relation/test_describe_relation.py
index adccd5126..5ec995d11 100644
--- a/tests/functional/adapter/describe_relation/test_describe_relation.py
+++ b/tests/functional/adapter/describe_relation/test_describe_relation.py
@@ -1,7 +1,7 @@
 import pytest
 
 from dbt.adapters.base.relation import BaseRelation
-from dbt.contracts.relation import RelationType
+from dbt.adapters.contracts.relation import RelationType
 from dbt.tests.util import get_connection, run_dbt
 
 from dbt.adapters.bigquery.relation_configs import BigQueryMaterializedViewConfig
diff --git a/tests/functional/adapter/materialized_view_tests/_mixin.py b/tests/functional/adapter/materialized_view_tests/_mixin.py
index 5f75c7c04..4eb98e047 100644
--- a/tests/functional/adapter/materialized_view_tests/_mixin.py
+++ b/tests/functional/adapter/materialized_view_tests/_mixin.py
@@ -3,7 +3,7 @@
 import pytest
 
 from dbt.adapters.base.relation import BaseRelation
-from dbt.contracts.relation import RelationType
+from dbt.adapters.contracts.relation import RelationType
 from dbt.tests.adapter.materialized_view.files import MY_TABLE, MY_VIEW
 from dbt.tests.util import (
     get_connection,
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 926547e10..a324c431a 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -1,3 +1,6 @@
+from multiprocessing import get_context
+from unittest import mock
+
 import agate
 import decimal
 import string
@@ -7,21 +10,28 @@
 import unittest
 from unittest.mock import patch, MagicMock, create_autospec
 
-import dbt.dataclass_schema
+import dbt.common.dataclass_schema
+import dbt.common.exceptions.base
 from dbt.adapters.bigquery.relation_configs import PartitionConfig
 from dbt.adapters.bigquery import BigQueryAdapter, BigQueryRelation
-from dbt.adapters.bigquery import Plugin as BigQueryPlugin
 from google.cloud.bigquery.table import Table
 from dbt.adapters.bigquery.connections import _sanitize_label, _VALIDATE_LABEL_LENGTH_LIMIT
-from dbt.adapters.base.query_headers import MacroQueryStringSetter
-from dbt.clients import agate_helper
-import dbt.exceptions
+from dbt.common.clients import agate_helper
+import dbt.common.exceptions
+from dbt.context.manifest import generate_query_header_context
+from dbt.contracts.files import FileHash
+from dbt.contracts.graph.manifest import ManifestStateCheck
 from dbt.logger import GLOBAL_LOGGER as logger  # noqa
-from dbt.context.providers import RuntimeConfigObject
+from dbt.context.providers import RuntimeConfigObject, generate_runtime_macro_context
 
 from google.cloud.bigquery import AccessEntry
 
-from .utils import config_from_parts_or_dicts, inject_adapter, TestAdapterConversions
+from .utils import (
+    config_from_parts_or_dicts,
+    inject_adapter,
+    TestAdapterConversions,
+    load_internal_manifest_macros,
+)
 
 
 def _bq_conn():
@@ -146,6 +156,21 @@ def setUp(self):
         }
         self.qh_patch = None
 
+        @mock.patch("dbt.parser.manifest.ManifestLoader.build_manifest_state_check")
+        def _mock_state_check(self):
+            all_projects = self.all_projects
+            return ManifestStateCheck(
+                vars_hash=FileHash.from_contents("vars"),
+                project_hashes={name: FileHash.from_contents(name) for name in all_projects},
+                profile_hash=FileHash.from_contents("profile"),
+            )
+
+        self.load_state_check = mock.patch(
+            "dbt.parser.manifest.ManifestLoader.build_manifest_state_check"
+        )
+        self.mock_state_check = self.load_state_check.start()
+        self.mock_state_check.side_effect = _mock_state_check
+
     def tearDown(self):
         if self.qh_patch:
             self.qh_patch.stop()
@@ -155,20 +180,22 @@ def get_adapter(self, target) -> BigQueryAdapter:
         project = self.project_cfg.copy()
         profile = self.raw_profile.copy()
         profile["target"] = target
-
         config = config_from_parts_or_dicts(
             project=project,
             profile=profile,
         )
-        adapter = BigQueryAdapter(config)
-
-        adapter.connections.query_header = MacroQueryStringSetter(config, MagicMock(macros={}))
+        adapter = BigQueryAdapter(config, get_context("spawn"))
+        adapter.set_macro_resolver(load_internal_manifest_macros(config))
+        adapter.set_macro_context_generator(generate_runtime_macro_context)
+        adapter.connections.set_query_header(
+            generate_query_header_context(config, adapter.get_macro_resolver())
+        )
 
         self.qh_patch = patch.object(adapter.connections.query_header, "add")
         self.mock_query_header_add = self.qh_patch.start()
         self.mock_query_header_add.side_effect = lambda q: "/* dbt */\n{}".format(q)
 
-        inject_adapter(adapter, BigQueryPlugin)
+        inject_adapter(adapter)
         return adapter
 
 
@@ -187,7 +214,7 @@ def test_acquire_connection_oauth_no_project_validations(
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.exceptions.DbtValidationError as e:
+        except dbt.common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
@@ -204,7 +231,7 @@ def test_acquire_connection_oauth_validations(self, mock_open_connection):
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.exceptions.DbtValidationError as e:
+        except dbt.common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
@@ -228,7 +255,7 @@ def test_acquire_connection_dataproc_serverless(
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.exceptions.ValidationException as e:
+        except dbt.common.exceptions.ValidationException as e:
             self.fail("got ValidationException: {}".format(str(e)))
 
         except BaseException:
@@ -245,7 +272,7 @@ def test_acquire_connection_service_account_validations(self, mock_open_connecti
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.exceptions.DbtValidationError as e:
+        except dbt.common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
@@ -262,7 +289,7 @@ def test_acquire_connection_oauth_token_validations(self, mock_open_connection):
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.exceptions.DbtValidationError as e:
+        except dbt.common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
@@ -279,7 +306,7 @@ def test_acquire_connection_oauth_credentials_validations(self, mock_open_connec
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.exceptions.DbtValidationError as e:
+        except dbt.common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
@@ -298,7 +325,7 @@ def test_acquire_connection_impersonated_service_account_validations(
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.exceptions.DbtValidationError as e:
+        except dbt.common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
@@ -316,7 +343,7 @@ def test_acquire_connection_priority(self, mock_open_connection):
             self.assertEqual(connection.type, "bigquery")
             self.assertEqual(connection.credentials.priority, "batch")
 
-        except dbt.exceptions.DbtValidationError as e:
+        except dbt.common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         mock_open_connection.assert_not_called()
@@ -331,7 +358,7 @@ def test_acquire_connection_maximum_bytes_billed(self, mock_open_connection):
             self.assertEqual(connection.type, "bigquery")
             self.assertEqual(connection.credentials.maximum_bytes_billed, 0)
 
-        except dbt.exceptions.DbtValidationError as e:
+        except dbt.common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         mock_open_connection.assert_not_called()
@@ -379,7 +406,7 @@ def test_location_user_agent(self, mock_bq, mock_auth_default):
 
 
 class HasUserAgent:
-    PAT = re.compile(r"dbt-\d+\.\d+\.\d+((a|b|rc)\d+)?")
+    PAT = re.compile(r"dbt-bigquery-\d+\.\d+\.\d+((a|b|rc)\d+)?")
 
     def __eq__(self, other):
         compare = getattr(other, "user_agent", "")
@@ -482,7 +509,7 @@ def test_invalid_relation(self):
             },
             "quote_policy": {"identifier": False, "schema": True},
         }
-        with self.assertRaises(dbt.dataclass_schema.ValidationError):
+        with self.assertRaises(dbt.common.dataclass_schema.ValidationError):
             BigQueryRelation.validate(kwargs)
 
 
@@ -554,10 +581,10 @@ def test_copy_table_materialization_incremental(self):
     def test_parse_partition_by(self):
         adapter = self.get_adapter("oauth")
 
-        with self.assertRaises(dbt.exceptions.DbtValidationError):
+        with self.assertRaises(dbt.common.exceptions.base.DbtValidationError):
             adapter.parse_partition_by("date(ts)")
 
-        with self.assertRaises(dbt.exceptions.DbtValidationError):
+        with self.assertRaises(dbt.common.exceptions.base.DbtValidationError):
             adapter.parse_partition_by("ts")
 
         self.assertEqual(
@@ -709,7 +736,7 @@ def test_parse_partition_by(self):
         )
 
         # Invalid, should raise an error
-        with self.assertRaises(dbt.exceptions.DbtValidationError):
+        with self.assertRaises(dbt.common.exceptions.base.DbtValidationError):
             adapter.parse_partition_by({})
 
         # passthrough
@@ -778,8 +805,7 @@ def test_view_kms_key_name(self):
 
 class TestBigQueryFilterCatalog(unittest.TestCase):
     def test__catalog_filter_table(self):
-        manifest = MagicMock()
-        manifest.get_used_schemas.return_value = [["a", "B"], ["a", "1234"]]
+        used_schemas = [["a", "B"], ["a", "1234"]]
         column_names = ["table_name", "table_database", "table_schema", "something"]
         rows = [
             ["foo", "a", "b", "1234"],  # include
@@ -789,7 +815,7 @@ def test__catalog_filter_table(self):
         ]
         table = agate.Table(rows, column_names, agate_helper.DEFAULT_TYPE_TESTER)
 
-        result = BigQueryAdapter._catalog_filter_table(table, manifest)
+        result = BigQueryAdapter._catalog_filter_table(table, used_schemas)
         assert len(result) == 3
         for row in result.rows:
             assert isinstance(row["table_schema"], str)
diff --git a/tests/unit/test_bigquery_connection_manager.py b/tests/unit/test_bigquery_connection_manager.py
index d6c3f64fc..04e6d1352 100644
--- a/tests/unit/test_bigquery_connection_manager.py
+++ b/tests/unit/test_bigquery_connection_manager.py
@@ -6,12 +6,12 @@
 from requests.exceptions import ConnectionError
 from unittest.mock import patch, MagicMock, Mock, ANY
 
-import dbt.dataclass_schema
+import dbt.common.dataclass_schema
 
 from dbt.adapters.bigquery import BigQueryCredentials
 from dbt.adapters.bigquery import BigQueryRelation
 from dbt.adapters.bigquery.connections import BigQueryConnectionManager
-import dbt.exceptions
+import dbt.common.exceptions
 from dbt.logger import GLOBAL_LOGGER as logger  # noqa
 
 
@@ -19,7 +19,7 @@ class TestBigQueryConnectionManager(unittest.TestCase):
     def setUp(self):
         credentials = Mock(BigQueryCredentials)
         profile = Mock(query_comment=None, credentials=credentials)
-        self.connections = BigQueryConnectionManager(profile=profile)
+        self.connections = BigQueryConnectionManager(profile=profile, mp_context=Mock())
 
         self.mock_client = Mock(dbt.adapters.bigquery.impl.google.cloud.bigquery.Client)
         self.mock_connection = MagicMock()
@@ -127,7 +127,7 @@ def test_query_and_results_timeout(self, mock_bq):
         self.mock_client.query = Mock(
             return_value=Mock(result=lambda *args, **kwargs: time.sleep(4))
         )
-        with pytest.raises(dbt.exceptions.DbtRuntimeError) as exc:
+        with pytest.raises(dbt.common.exceptions.DbtRuntimeError) as exc:
             self.connections._query_and_results(
                 self.mock_client,
                 "sql",
diff --git a/tests/unit/utils.py b/tests/unit/utils.py
index 7cd2bb9ab..6d21828b3 100644
--- a/tests/unit/utils.py
+++ b/tests/unit/utils.py
@@ -9,7 +9,8 @@
 
 import agate
 import pytest
-from dbt.dataclass_schema import ValidationError
+
+from dbt.common.dataclass_schema import ValidationError
 from dbt.config.project import PartialProject
 
 
@@ -123,7 +124,6 @@ def inject_plugin(plugin):
 
 
 def inject_plugin_for(config):
-    # from dbt.adapters.postgres import Plugin, PostgresAdapter
     from dbt.adapters.factory import FACTORY
 
     FACTORY.load_plugin(config.credentials.type)
@@ -131,11 +131,10 @@ def inject_plugin_for(config):
     return adapter
 
 
-def inject_adapter(value, plugin):
+def inject_adapter(value):
     """Inject the given adapter into the adapter factory, so your hand-crafted
     artisanal adapter will be available from get_adapter() as if dbt loaded it.
     """
-    inject_plugin(plugin)
     from dbt.adapters.factory import FACTORY
 
     key = value.type()
@@ -229,7 +228,7 @@ def assert_fails_validation(dct, cls):
 
 
 def generate_name_macros(package):
-    from dbt.contracts.graph.parsed import ParsedMacro
+    from dbt.contracts.graph.nodes import Macro
     from dbt.node_types import NodeType
 
     name_sql = {}
@@ -243,13 +242,12 @@ def generate_name_macros(package):
         name_sql[name] = sql
 
     for name, sql in name_sql.items():
-        pm = ParsedMacro(
+        pm = Macro(
             name=name,
             resource_type=NodeType.Macro,
             unique_id=f"macro.{package}.{name}",
             package_name=package,
             original_file_path=normalize("macros/macro.sql"),
-            root_path="./dbt_packages/root",
             path=normalize("macros/macro.sql"),
             macro_sql=sql,
         )
@@ -258,7 +256,7 @@ def generate_name_macros(package):
 
 class TestAdapterConversions(TestCase):
     def _get_tester_for(self, column_type):
-        from dbt.clients import agate_helper
+        from dbt.common.clients import agate_helper
 
         if column_type is agate.TimeDelta:  # dbt never makes this!
             return agate.TimeDelta()
@@ -280,7 +278,7 @@ def _make_table_of(self, rows, column_types):
 
 
 def MockMacro(package, name="my_macro", **kwargs):
-    from dbt.contracts.graph.parsed import ParsedMacro
+    from dbt.contracts.graph.nodes import Macro
     from dbt.node_types import NodeType
 
     mock_kwargs = dict(
@@ -292,7 +290,7 @@ def MockMacro(package, name="my_macro", **kwargs):
 
     mock_kwargs.update(kwargs)
 
-    macro = mock.MagicMock(spec=ParsedMacro, **mock_kwargs)
+    macro = mock.MagicMock(spec=Macro, **mock_kwargs)
     macro.name = name
     return macro
 
@@ -311,10 +309,10 @@ def MockGenerateMacro(package, component="some_component", **kwargs):
 
 def MockSource(package, source_name, name, **kwargs):
     from dbt.node_types import NodeType
-    from dbt.contracts.graph.parsed import ParsedSourceDefinition
+    from dbt.contracts.graph.nodes import SourceDefinition
 
     src = mock.MagicMock(
-        __class__=ParsedSourceDefinition,
+        __class__=SourceDefinition,
         resource_type=NodeType.Source,
         source_name=source_name,
         package_name=package,
@@ -328,14 +326,14 @@ def MockSource(package, source_name, name, **kwargs):
 
 def MockNode(package, name, resource_type=None, **kwargs):
     from dbt.node_types import NodeType
-    from dbt.contracts.graph.parsed import ParsedModelNode, ParsedSeedNode
+    from dbt.contracts.graph.nodes import ModelNode, SeedNode
 
     if resource_type is None:
         resource_type = NodeType.Model
     if resource_type == NodeType.Model:
-        cls = ParsedModelNode
+        cls = ModelNode
     elif resource_type == NodeType.Seed:
-        cls = ParsedSeedNode
+        cls = SeedNode
     else:
         raise ValueError(f"I do not know how to handle {resource_type}")
     node = mock.MagicMock(
@@ -352,10 +350,10 @@ def MockNode(package, name, resource_type=None, **kwargs):
 
 def MockDocumentation(package, name, **kwargs):
     from dbt.node_types import NodeType
-    from dbt.contracts.graph.parsed import ParsedDocumentation
+    from dbt.contracts.graph.nodes import Documentation
 
     doc = mock.MagicMock(
-        __class__=ParsedDocumentation,
+        __class__=Documentation,
         resource_type=NodeType.Documentation,
         package_name=package,
         search_name=name,
diff --git a/tox.ini b/tox.ini
index 1d5413dcf..30e3c5816 100644
--- a/tox.ini
+++ b/tox.ini
@@ -28,7 +28,7 @@ passenv =
     DD_ENV
     DD_SERVICE
 commands =
-  bigquery: {envpython} -m pytest {posargs} -vv tests/functional -k "not TestPython" --profile service_account
+  bigquery: {envpython} -m pytest -n auto {posargs} -vv tests/functional -k "not TestPython" --profile service_account
 deps =
   -rdev-requirements.txt
   -e.

From e86609a1e15a766eb764a535f277100ca10ee67b Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Mon, 22 Jan 2024 13:59:04 -0800
Subject: [PATCH 758/860] Migrate to dbt-adapter and common (#1071)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* replace dbt.common with dbt_common

* add dbt-adapters

* delete dbt/adapters

* fix Credentials import and test fixtures

* remove global exceptions import
---
 .../Under the Hood-20240116-154305.yaml       |  6 ++++
 dbt/adapters/bigquery/connections.py          | 18 +++++-----
 dbt/adapters/bigquery/gcloud.py               |  7 ++--
 dbt/adapters/bigquery/impl.py                 | 30 ++++++++--------
 dbt/adapters/bigquery/relation.py             |  4 +--
 .../bigquery/relation_configs/_partition.py   |  8 ++---
 dbt/adapters/bigquery/utility.py              |  4 +--
 setup.py                                      |  3 +-
 .../adapter/column_types/fixtures.py          |  4 +--
 tests/functional/adapter/test_aliases.py      |  4 +--
 tests/unit/test_bigquery_adapter.py           | 36 ++++++++++---------
 .../unit/test_bigquery_connection_manager.py  |  7 ++--
 tests/unit/utils.py                           |  4 +--
 13 files changed, 73 insertions(+), 62 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20240116-154305.yaml

diff --git a/.changes/unreleased/Under the Hood-20240116-154305.yaml b/.changes/unreleased/Under the Hood-20240116-154305.yaml
new file mode 100644
index 000000000..bb115abd6
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20240116-154305.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Migrate to dbt-common and dbt-adapters package
+time: 2024-01-16T15:43:05.046735-08:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "1071"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index cb933baed..c74effcdc 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -5,9 +5,9 @@
 from contextlib import contextmanager
 from dataclasses import dataclass, field
 
-from dbt.common.invocation import get_invocation_id
+from dbt_common.invocation import get_invocation_id
 
-from dbt.common.events.contextvars import get_node_info
+from dbt_common.events.contextvars import get_node_info
 from mashumaro.helper import pass_through
 
 from functools import lru_cache
@@ -27,21 +27,21 @@
 )
 
 from dbt.adapters.bigquery import gcloud
-from dbt.common.clients import agate_helper
-from dbt.adapters.contracts.connection import ConnectionState, AdapterResponse
-from dbt.common.exceptions import (
+from dbt_common.clients import agate_helper
+from dbt.adapters.contracts.connection import ConnectionState, AdapterResponse, Credentials
+from dbt_common.exceptions import (
     DbtRuntimeError,
     DbtConfigError,
 )
-from dbt.common.exceptions import DbtDatabaseError
+from dbt_common.exceptions import DbtDatabaseError
 from dbt.adapters.exceptions.connection import FailedToConnectError
-from dbt.adapters.base import BaseConnectionManager, Credentials
+from dbt.adapters.base import BaseConnectionManager
 from dbt.adapters.events.logging import AdapterLogger
 from dbt.adapters.events.types import SQLQuery
-from dbt.common.events.functions import fire_event
+from dbt_common.events.functions import fire_event
 from dbt.adapters.bigquery import __version__ as dbt_version
 
-from dbt.common.dataclass_schema import ExtensibleDbtClassMixin, StrEnum
+from dbt_common.dataclass_schema import ExtensibleDbtClassMixin, StrEnum
 
 logger = AdapterLogger("BigQuery")
 
diff --git a/dbt/adapters/bigquery/gcloud.py b/dbt/adapters/bigquery/gcloud.py
index 0a08f734d..ea1f644ba 100644
--- a/dbt/adapters/bigquery/gcloud.py
+++ b/dbt/adapters/bigquery/gcloud.py
@@ -1,6 +1,7 @@
+from dbt_common.exceptions import DbtRuntimeError
+
 from dbt.adapters.events.logging import AdapterLogger
-import dbt.common.exceptions
-from dbt.common.clients.system import run_cmd
+from dbt_common.clients.system import run_cmd
 
 NOT_INSTALLED_MSG = """
 dbt requires the gcloud SDK to be installed to authenticate with BigQuery.
@@ -25,4 +26,4 @@ def setup_default_credentials():
     if gcloud_installed():
         run_cmd(".", ["gcloud", "auth", "application-default", "login"])
     else:
-        raise dbt.common.exceptions.DbtRuntimeError(NOT_INSTALLED_MSG)
+        raise DbtRuntimeError(NOT_INSTALLED_MSG)
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 03cfd3561..3ca7435c4 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -9,7 +9,7 @@
 import agate
 from dbt.adapters.contracts.relation import RelationConfig
 
-import dbt.common.exceptions.base
+import dbt_common.exceptions.base
 from dbt.adapters.base import (  # type: ignore
     AdapterConfig,
     BaseAdapter,
@@ -21,15 +21,15 @@
     available,
 )
 from dbt.adapters.cache import _make_ref_key_dict  # type: ignore
-import dbt.common.clients.agate_helper
+import dbt_common.clients.agate_helper
 from dbt.adapters.contracts.connection import AdapterResponse
-from dbt.common.contracts.constraints import ColumnLevelConstraint, ConstraintType, ModelLevelConstraint  # type: ignore
-from dbt.common.dataclass_schema import dbtClassMixin
+from dbt_common.contracts.constraints import ColumnLevelConstraint, ConstraintType, ModelLevelConstraint  # type: ignore
+from dbt_common.dataclass_schema import dbtClassMixin
 from dbt.adapters.events.logging import AdapterLogger
-from dbt.common.events.functions import fire_event
+from dbt_common.events.functions import fire_event
 from dbt.adapters.events.types import SchemaCreation, SchemaDrop
-import dbt.common.exceptions
-from dbt.common.utils import filter_null_values
+import dbt_common.exceptions
+from dbt_common.utils import filter_null_values
 import google.api_core
 import google.auth
 import google.oauth2
@@ -147,7 +147,7 @@ def drop_relation(self, relation: BigQueryRelation) -> None:
         conn.handle.delete_table(table_ref, not_found_ok=True)
 
     def truncate_relation(self, relation: BigQueryRelation) -> None:
-        raise dbt.common.exceptions.base.NotImplementedError(
+        raise dbt_common.exceptions.base.NotImplementedError(
             "`truncate` is not implemented for this adapter!"
         )
 
@@ -164,7 +164,7 @@ def rename_relation(
             or from_relation.type == RelationType.View
             or to_relation.type == RelationType.View
         ):
-            raise dbt.common.exceptions.DbtRuntimeError(
+            raise dbt_common.exceptions.DbtRuntimeError(
                 "Renaming of views is not currently supported in BigQuery"
             )
 
@@ -390,7 +390,7 @@ def copy_table(self, source, destination, materialization):
         elif materialization == "table":
             write_disposition = WRITE_TRUNCATE
         else:
-            raise dbt.common.exceptions.CompilationError(
+            raise dbt_common.exceptions.CompilationError(
                 'Copy table materialization must be "copy" or "table", but '
                 f"config.get('copy_materialization', 'table') was "
                 f"{materialization}"
@@ -437,11 +437,11 @@ def poll_until_job_completes(cls, job, timeout):
             job.reload()
 
         if job.state != "DONE":
-            raise dbt.common.exceptions.DbtRuntimeError("BigQuery Timeout Exceeded")
+            raise dbt_common.exceptions.DbtRuntimeError("BigQuery Timeout Exceeded")
 
         elif job.error_result:
             message = "\n".join(error["message"].strip() for error in job.errors)
-            raise dbt.common.exceptions.DbtRuntimeError(message)
+            raise dbt_common.exceptions.DbtRuntimeError(message)
 
     def _bq_table_to_relation(self, bq_table) -> Union[BigQueryRelation, None]:
         if bq_table is None:
@@ -465,7 +465,7 @@ def add_query(self, sql, auto_begin=True, bindings=None, abridge_sql_log=False):
         if self.nice_connection_name() in ["on-run-start", "on-run-end"]:
             self.warning_on_hooks(self.nice_connection_name())
         else:
-            raise dbt.common.exceptions.base.NotImplementedError(
+            raise dbt_common.exceptions.base.NotImplementedError(
                 "`add_query` is not implemented for this adapter!"
             )
 
@@ -777,7 +777,7 @@ def describe_relation(
             bq_table = self.get_bq_table(relation)
             parser = BigQueryMaterializedViewConfig
         else:
-            raise dbt.common.exceptions.DbtRuntimeError(
+            raise dbt_common.exceptions.DbtRuntimeError(
                 f"The method `BigQueryAdapter.describe_relation` is not implemented "
                 f"for the relation type: {relation.type}"
             )
@@ -843,7 +843,7 @@ def string_add_sql(
         elif location == "prepend":
             return f"concat('{value}', {add_to})"
         else:
-            raise dbt.common.exceptions.DbtRuntimeError(
+            raise dbt_common.exceptions.DbtRuntimeError(
                 f'Got an unexpected location value of "{location}"'
             )
 
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index c25ef0a67..8abda577b 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -12,8 +12,8 @@
     BigQueryPartitionConfigChange,
 )
 from dbt.adapters.contracts.relation import RelationType, RelationConfig
-from dbt.common.exceptions import CompilationError
-from dbt.common.utils.dict import filter_null_values
+from dbt_common.exceptions import CompilationError
+from dbt_common.utils.dict import filter_null_values
 
 
 Self = TypeVar("Self", bound="BigQueryRelation")
diff --git a/dbt/adapters/bigquery/relation_configs/_partition.py b/dbt/adapters/bigquery/relation_configs/_partition.py
index 0fe816359..8fe8bf5d6 100644
--- a/dbt/adapters/bigquery/relation_configs/_partition.py
+++ b/dbt/adapters/bigquery/relation_configs/_partition.py
@@ -1,10 +1,10 @@
 from dataclasses import dataclass
 from typing import Any, Dict, List, Optional
 
-import dbt.common.exceptions
+import dbt_common.exceptions
 from dbt.adapters.relation_configs import RelationConfigChange
 from dbt.adapters.contracts.relation import RelationConfig
-from dbt.common.dataclass_schema import dbtClassMixin, ValidationError
+from dbt_common.dataclass_schema import dbtClassMixin, ValidationError
 from google.cloud.bigquery.table import Table as BigQueryTable
 
 
@@ -92,11 +92,11 @@ def parse(cls, raw_partition_by) -> Optional["PartitionConfig"]:
                 }
             )
         except ValidationError as exc:
-            raise dbt.common.exceptions.base.DbtValidationError(
+            raise dbt_common.exceptions.base.DbtValidationError(
                 "Could not parse partition config"
             ) from exc
         except TypeError:
-            raise dbt.common.exceptions.CompilationError(
+            raise dbt_common.exceptions.CompilationError(
                 f"Invalid partition_by config:\n"
                 f"  Got: {raw_partition_by}\n"
                 f'  Expected a dictionary with "field" and "data_type" keys'
diff --git a/dbt/adapters/bigquery/utility.py b/dbt/adapters/bigquery/utility.py
index 5d9c3de12..5914280a3 100644
--- a/dbt/adapters/bigquery/utility.py
+++ b/dbt/adapters/bigquery/utility.py
@@ -1,7 +1,7 @@
 import json
 from typing import Any, Optional
 
-import dbt.common.exceptions
+import dbt_common.exceptions
 
 
 def bool_setting(value: Optional[Any] = None) -> Optional[bool]:
@@ -41,5 +41,5 @@ def float_setting(value: Optional[Any] = None) -> Optional[float]:
 
 def sql_escape(string):
     if not isinstance(string, str):
-        raise dbt.common.exceptions.CompilationError(f"cannot escape a non-string: {string}")
+        raise dbt_common.exceptions.CompilationError(f"cannot escape a non-string: {string}")
     return json.dumps(string)[1:-1]
diff --git a/setup.py b/setup.py
index 56c271fec..b3a1065cb 100644
--- a/setup.py
+++ b/setup.py
@@ -74,7 +74,8 @@ def _dbt_core_version(plugin_version: str) -> str:
     packages=find_namespace_packages(include=["dbt", "dbt.*"]),
     include_package_data=True,
     install_requires=[
-        f"dbt-core~={_dbt_core_version(_dbt_bigquery_version())}",
+        "dbt-common<1.0",
+        "dbt-adapters~=0.1.0a1",
         "google-cloud-bigquery~=3.0",
         "google-cloud-storage~=2.4",
         "google-cloud-dataproc~=5.0",
diff --git a/tests/functional/adapter/column_types/fixtures.py b/tests/functional/adapter/column_types/fixtures.py
index b7be1e646..88175a88b 100644
--- a/tests/functional/adapter/column_types/fixtures.py
+++ b/tests/functional/adapter/column_types/fixtures.py
@@ -26,7 +26,7 @@
 version: 2
 models:
   - name: model
-    tests:
+    data_tests:
       - is_type:
           column_map:
             int64_col: ['integer', 'number']
@@ -39,7 +39,7 @@
 version: 2
 models:
   - name: model
-    tests:
+    data_tests:
       - is_type:
           column_map:
             int64_col: ['string', 'not number']
diff --git a/tests/functional/adapter/test_aliases.py b/tests/functional/adapter/test_aliases.py
index fa28ce5d9..5ce13c8bc 100644
--- a/tests/functional/adapter/test_aliases.py
+++ b/tests/functional/adapter/test_aliases.py
@@ -32,12 +32,12 @@
 version: 2
 models:
 - name: model_a
-  tests:
+  data_tests:
   - expect_value:
       field: tablename
       value: duped_alias
 - name: model_b
-  tests:
+  data_tests:
   - expect_value:
       field: tablename
       value: duped_alias
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index a324c431a..34abd0caf 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -10,14 +10,16 @@
 import unittest
 from unittest.mock import patch, MagicMock, create_autospec
 
-import dbt.common.dataclass_schema
-import dbt.common.exceptions.base
+import dbt_common.dataclass_schema
+import dbt_common.exceptions.base
+
+import dbt.adapters
 from dbt.adapters.bigquery.relation_configs import PartitionConfig
 from dbt.adapters.bigquery import BigQueryAdapter, BigQueryRelation
 from google.cloud.bigquery.table import Table
 from dbt.adapters.bigquery.connections import _sanitize_label, _VALIDATE_LABEL_LENGTH_LIMIT
-from dbt.common.clients import agate_helper
-import dbt.common.exceptions
+from dbt_common.clients import agate_helper
+import dbt_common.exceptions
 from dbt.context.manifest import generate_query_header_context
 from dbt.contracts.files import FileHash
 from dbt.contracts.graph.manifest import ManifestStateCheck
@@ -214,7 +216,7 @@ def test_acquire_connection_oauth_no_project_validations(
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.common.exceptions.base.DbtValidationError as e:
+        except dbt_common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
@@ -231,7 +233,7 @@ def test_acquire_connection_oauth_validations(self, mock_open_connection):
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.common.exceptions.base.DbtValidationError as e:
+        except dbt_common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
@@ -255,7 +257,7 @@ def test_acquire_connection_dataproc_serverless(
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.common.exceptions.ValidationException as e:
+        except dbt_common.exceptions.ValidationException as e:
             self.fail("got ValidationException: {}".format(str(e)))
 
         except BaseException:
@@ -272,7 +274,7 @@ def test_acquire_connection_service_account_validations(self, mock_open_connecti
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.common.exceptions.base.DbtValidationError as e:
+        except dbt_common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
@@ -289,7 +291,7 @@ def test_acquire_connection_oauth_token_validations(self, mock_open_connection):
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.common.exceptions.base.DbtValidationError as e:
+        except dbt_common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
@@ -306,7 +308,7 @@ def test_acquire_connection_oauth_credentials_validations(self, mock_open_connec
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.common.exceptions.base.DbtValidationError as e:
+        except dbt_common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
@@ -325,7 +327,7 @@ def test_acquire_connection_impersonated_service_account_validations(
             connection = adapter.acquire_connection("dummy")
             self.assertEqual(connection.type, "bigquery")
 
-        except dbt.common.exceptions.base.DbtValidationError as e:
+        except dbt_common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         except BaseException:
@@ -343,7 +345,7 @@ def test_acquire_connection_priority(self, mock_open_connection):
             self.assertEqual(connection.type, "bigquery")
             self.assertEqual(connection.credentials.priority, "batch")
 
-        except dbt.common.exceptions.base.DbtValidationError as e:
+        except dbt_common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         mock_open_connection.assert_not_called()
@@ -358,7 +360,7 @@ def test_acquire_connection_maximum_bytes_billed(self, mock_open_connection):
             self.assertEqual(connection.type, "bigquery")
             self.assertEqual(connection.credentials.maximum_bytes_billed, 0)
 
-        except dbt.common.exceptions.base.DbtValidationError as e:
+        except dbt_common.exceptions.base.DbtValidationError as e:
             self.fail("got DbtValidationError: {}".format(str(e)))
 
         mock_open_connection.assert_not_called()
@@ -509,7 +511,7 @@ def test_invalid_relation(self):
             },
             "quote_policy": {"identifier": False, "schema": True},
         }
-        with self.assertRaises(dbt.common.dataclass_schema.ValidationError):
+        with self.assertRaises(dbt_common.dataclass_schema.ValidationError):
             BigQueryRelation.validate(kwargs)
 
 
@@ -581,10 +583,10 @@ def test_copy_table_materialization_incremental(self):
     def test_parse_partition_by(self):
         adapter = self.get_adapter("oauth")
 
-        with self.assertRaises(dbt.common.exceptions.base.DbtValidationError):
+        with self.assertRaises(dbt_common.exceptions.base.DbtValidationError):
             adapter.parse_partition_by("date(ts)")
 
-        with self.assertRaises(dbt.common.exceptions.base.DbtValidationError):
+        with self.assertRaises(dbt_common.exceptions.base.DbtValidationError):
             adapter.parse_partition_by("ts")
 
         self.assertEqual(
@@ -736,7 +738,7 @@ def test_parse_partition_by(self):
         )
 
         # Invalid, should raise an error
-        with self.assertRaises(dbt.common.exceptions.base.DbtValidationError):
+        with self.assertRaises(dbt_common.exceptions.base.DbtValidationError):
             adapter.parse_partition_by({})
 
         # passthrough
diff --git a/tests/unit/test_bigquery_connection_manager.py b/tests/unit/test_bigquery_connection_manager.py
index 04e6d1352..6bb89ed36 100644
--- a/tests/unit/test_bigquery_connection_manager.py
+++ b/tests/unit/test_bigquery_connection_manager.py
@@ -6,12 +6,13 @@
 from requests.exceptions import ConnectionError
 from unittest.mock import patch, MagicMock, Mock, ANY
 
-import dbt.common.dataclass_schema
+import dbt.adapters
+import dbt_common.dataclass_schema
 
 from dbt.adapters.bigquery import BigQueryCredentials
 from dbt.adapters.bigquery import BigQueryRelation
 from dbt.adapters.bigquery.connections import BigQueryConnectionManager
-import dbt.common.exceptions
+import dbt_common.exceptions
 from dbt.logger import GLOBAL_LOGGER as logger  # noqa
 
 
@@ -127,7 +128,7 @@ def test_query_and_results_timeout(self, mock_bq):
         self.mock_client.query = Mock(
             return_value=Mock(result=lambda *args, **kwargs: time.sleep(4))
         )
-        with pytest.raises(dbt.common.exceptions.DbtRuntimeError) as exc:
+        with pytest.raises(dbt_common.exceptions.DbtRuntimeError) as exc:
             self.connections._query_and_results(
                 self.mock_client,
                 "sql",
diff --git a/tests/unit/utils.py b/tests/unit/utils.py
index 6d21828b3..88b09ce60 100644
--- a/tests/unit/utils.py
+++ b/tests/unit/utils.py
@@ -10,7 +10,7 @@
 import agate
 import pytest
 
-from dbt.common.dataclass_schema import ValidationError
+from dbt_common.dataclass_schema import ValidationError
 from dbt.config.project import PartialProject
 
 
@@ -256,7 +256,7 @@ def generate_name_macros(package):
 
 class TestAdapterConversions(TestCase):
     def _get_tester_for(self, column_type):
-        from dbt.common.clients import agate_helper
+        from dbt_common.clients import agate_helper
 
         if column_type is agate.TimeDelta:  # dbt never makes this!
             return agate.TimeDelta()

From 70b7445ce802ca03bcb99a44a1ee261b7ef98ca0 Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Tue, 23 Jan 2024 12:19:36 -0700
Subject: [PATCH 759/860] Primary and foreign key constraints are not enforced
 in BigQuery (#1019)

* primary and foreign key constraints are not enforced in BigQuery

* Changelog entry

---------

Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
---
 .changes/unreleased/Under the Hood-20231116-062142.yaml | 6 ++++++
 dbt/adapters/bigquery/impl.py                           | 4 ++--
 2 files changed, 8 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20231116-062142.yaml

diff --git a/.changes/unreleased/Under the Hood-20231116-062142.yaml b/.changes/unreleased/Under the Hood-20231116-062142.yaml
new file mode 100644
index 000000000..c28270898
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20231116-062142.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Primary and foreign key constraints are not enforced in BigQuery
+time: 2023-11-16T06:21:42.935367-08:00
+custom:
+  Author: dbeatty10
+  Issue: "1018"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 3ca7435c4..3e394ad26 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -114,8 +114,8 @@ class BigQueryAdapter(BaseAdapter):
         ConstraintType.check: ConstraintSupport.NOT_SUPPORTED,
         ConstraintType.not_null: ConstraintSupport.ENFORCED,
         ConstraintType.unique: ConstraintSupport.NOT_SUPPORTED,
-        ConstraintType.primary_key: ConstraintSupport.ENFORCED,
-        ConstraintType.foreign_key: ConstraintSupport.ENFORCED,
+        ConstraintType.primary_key: ConstraintSupport.NOT_ENFORCED,
+        ConstraintType.foreign_key: ConstraintSupport.NOT_ENFORCED,
     }
 
     def __init__(self, config, mp_context: SpawnContext) -> None:

From 8fc40e6e7163df08a14d286d4913ed1485079cc3 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Fri, 26 Jan 2024 08:55:37 -0800
Subject: [PATCH 760/860] update dev-requirements.txt to use dbt-adapters for
 dbt-tests-adapter (#1077)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* replace dbt.common with dbt_common

* add dbt-adapters

* delete dbt/adapters

* fix Credentials import and test fixtures

* remove global exceptions import

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* update dev-requirements.txt to use dbt-adapters for dbt-tests-adapter

* remove unneeded install of dbt-core from dev-requirements.txt

* add changie

* remove dbt-core version checking
---
 .../Dependencies-20240124-120321.yaml         |  6 +++++
 dev-requirements.txt                          |  3 +--
 setup.py                                      | 23 -------------------
 3 files changed, 7 insertions(+), 25 deletions(-)
 create mode 100644 .changes/unreleased/Dependencies-20240124-120321.yaml

diff --git a/.changes/unreleased/Dependencies-20240124-120321.yaml b/.changes/unreleased/Dependencies-20240124-120321.yaml
new file mode 100644
index 000000000..ef725de67
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240124-120321.yaml
@@ -0,0 +1,6 @@
+kind: Dependencies
+body: get dbt-tests-adapters from dbt-adapters repo
+time: 2024-01-24T12:03:21.523295-08:00
+custom:
+  Author: colin-rogers-dbt
+  PR: "1077"
diff --git a/dev-requirements.txt b/dev-requirements.txt
index bd45c99a1..3e0373253 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -1,7 +1,6 @@
 # install latest changes in dbt-core
 # TODO: how to automate switching from develop to version branches?
-git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
-git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-tests-adapter&subdirectory=tests/adapter
+git+https://github.com/dbt-labs/dbt-adapters.git#subdirectory=dbt-tests-adapter
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor
 black~=23.12
diff --git a/setup.py b/setup.py
index b3a1065cb..2e969e246 100644
--- a/setup.py
+++ b/setup.py
@@ -35,31 +35,8 @@ def _dbt_bigquery_version() -> str:
     return attributes["version"]
 
 
-# require a compatible minor version (~=) and prerelease if this is a prerelease
-def _dbt_core_version(plugin_version: str) -> str:
-    """
-    Determine the compatible version of dbt-core using this package's version
-    """
-    try:
-        # *_ may indicate a dev release which won't affect the core version needed
-        major, minor, plugin_patch, *_ = plugin_version.split(".", maxsplit=3)
-    except ValueError:
-        raise ValueError(f"Invalid version: {plugin_version}")
-
-    pre_release_phase = "".join([i for i in plugin_patch if not i.isdigit()])
-    if pre_release_phase:
-        if pre_release_phase not in ["a", "b", "rc"]:
-            raise ValueError(f"Invalid version: {plugin_version}")
-        core_patch = f"0{pre_release_phase}1"
-    else:
-        core_patch = "0"
-
-    return f"{major}.{minor}.{core_patch}"
-
-
 package_name = "dbt-bigquery"
 package_version = "1.8.0a1"
-dbt_core_version = _dbt_core_version(_dbt_bigquery_version())
 description = """The BigQuery adapter plugin for dbt"""
 
 setup(

From 77a9f140c95cc23b5120248af3fe02323ac8604c Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 1 Feb 2024 16:22:21 -0800
Subject: [PATCH 761/860] Add dbt-core install back to dev-requirements.txt
 (#1082)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* add dbt-core back to dev-requirements.txt

* install dev dbt-adapters
---
 dev-requirements.txt | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index 3e0373253..d630d5175 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -1,5 +1,7 @@
 # install latest changes in dbt-core
 # TODO: how to automate switching from develop to version branches?
+git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
+git+https://github.com/dbt-labs/dbt-adapters.git
 git+https://github.com/dbt-labs/dbt-adapters.git#subdirectory=dbt-tests-adapter
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor

From 9a03bd818f29c1b104f7f7b1fc791c5f4d6d55c5 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 7 Feb 2024 18:11:05 -0500
Subject: [PATCH 762/860] remove actions that sync github and jira (#1095)

---
 .github/workflows/jira-creation.yml   | 28 --------------------------
 .github/workflows/jira-label.yml      | 28 --------------------------
 .github/workflows/jira-transition.yml | 29 ---------------------------
 3 files changed, 85 deletions(-)
 delete mode 100644 .github/workflows/jira-creation.yml
 delete mode 100644 .github/workflows/jira-label.yml
 delete mode 100644 .github/workflows/jira-transition.yml

diff --git a/.github/workflows/jira-creation.yml b/.github/workflows/jira-creation.yml
deleted file mode 100644
index 2611a8bdd..000000000
--- a/.github/workflows/jira-creation.yml
+++ /dev/null
@@ -1,28 +0,0 @@
-# **what?**
-# Mirrors issues into Jira. Includes the information: title,
-# GitHub Issue ID and URL
-
-# **why?**
-# Jira is our tool for tracking and we need to see these issues in there
-
-# **when?**
-# On issue creation or when an issue is labeled `Jira`
-
-name: Jira Issue Creation
-
-on:
-  issues:
-    types: [opened, labeled]
-
-permissions:
-  issues: write
-
-jobs:
-  call-label-action:
-    uses: dbt-labs/actions/.github/workflows/jira-creation.yml@main
-    with:
-      project_key: ADAP
-    secrets:
-      JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
-      JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
-      JIRA_API_TOKEN: ${{ secrets.JIRA_API_TOKEN }}
diff --git a/.github/workflows/jira-label.yml b/.github/workflows/jira-label.yml
deleted file mode 100644
index 1637cbe38..000000000
--- a/.github/workflows/jira-label.yml
+++ /dev/null
@@ -1,28 +0,0 @@
-# **what?**
-# Calls mirroring Jira label Action. Includes adding a new label
-# to an existing issue or removing a label as well
-
-# **why?**
-# Jira is our tool for tracking and we need to see these labels in there
-
-# **when?**
-# On labels being added or removed from issues
-
-name: Jira Label Mirroring
-
-on:
-  issues:
-    types: [labeled, unlabeled]
-
-permissions:
-  issues: read
-
-jobs:
-  call-label-action:
-    uses: dbt-labs/actions/.github/workflows/jira-label.yml@main
-    with:
-      project_key: ADAP
-    secrets:
-      JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
-      JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
-      JIRA_API_TOKEN: ${{ secrets.JIRA_API_TOKEN }}
diff --git a/.github/workflows/jira-transition.yml b/.github/workflows/jira-transition.yml
deleted file mode 100644
index 99158a15f..000000000
--- a/.github/workflows/jira-transition.yml
+++ /dev/null
@@ -1,29 +0,0 @@
-# **what?**
-# Transition a Jira issue to a new state
-# Only supports these GitHub Issue transitions:
-#   closed, deleted, reopened
-
-# **why?**
-# Jira needs to be kept up-to-date
-
-# **when?**
-# On issue closing, deletion, reopened
-
-name: Jira Issue Transition
-
-on:
-  issues:
-    types: [closed, deleted, reopened]
-
-# no special access is needed
-permissions: read-all
-
-jobs:
-  call-label-action:
-    uses: dbt-labs/actions/.github/workflows/jira-transition.yml@main
-    with:
-      project_key: ADAP
-    secrets:
-      JIRA_BASE_URL: ${{ secrets.JIRA_BASE_URL }}
-      JIRA_USER_EMAIL: ${{ secrets.JIRA_USER_EMAIL }}
-      JIRA_API_TOKEN: ${{ secrets.JIRA_API_TOKEN }}

From 1b0476254dd463c9e2d5c5b9b06bf05826be1751 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 8 Feb 2024 17:28:09 -0800
Subject: [PATCH 763/860] Use dbt-common main in dev-requirements.txt (#1100)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* use dbt-common main to test against
---
 dev-requirements.txt | 1 +
 1 file changed, 1 insertion(+)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index d630d5175..0af563a7d 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -1,6 +1,7 @@
 # install latest changes in dbt-core
 # TODO: how to automate switching from develop to version branches?
 git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
+git+https://github.com/dbt-labs/dbt-common.git
 git+https://github.com/dbt-labs/dbt-adapters.git
 git+https://github.com/dbt-labs/dbt-adapters.git#subdirectory=dbt-tests-adapter
 # if version 1.x or greater -> pin to major version

From ea3abee14294a45f77e3190822a1cfe4d158b827 Mon Sep 17 00:00:00 2001
From: Michelle Ark <MichelleArk@users.noreply.github.com>
Date: Fri, 9 Feb 2024 11:48:06 -0500
Subject: [PATCH 764/860] [Bigquery] unit test tests + support for complex
 types (#1031)

* first pass: unit test typing

* expand test coverage, update safe_cast

* TestBigQueryUnitTestCaseInsensitivity

* changelog entry

* structs

* structs of structs, arrays of structs, clean up safe_cast

* typos

* restore dev-requirements

* changelog entry
---
 .../unreleased/Features-20240205-174614.yaml  |  7 ++
 .../bigquery/macros/utils/safe_cast.sql       | 24 +++++++
 .../adapter/unit_testing/test_unit_testing.py | 64 +++++++++++++++++++
 3 files changed, 95 insertions(+)
 create mode 100644 .changes/unreleased/Features-20240205-174614.yaml
 create mode 100644 tests/functional/adapter/unit_testing/test_unit_testing.py

diff --git a/.changes/unreleased/Features-20240205-174614.yaml b/.changes/unreleased/Features-20240205-174614.yaml
new file mode 100644
index 000000000..192273d3d
--- /dev/null
+++ b/.changes/unreleased/Features-20240205-174614.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: Support all types for unit testing in dbt-bigquery, expand coverage of
+  safe_cast macro
+time: 2024-02-05T17:46:14.505597-05:00
+custom:
+  Author: michelleark
+  Issue: "1090"
diff --git a/dbt/include/bigquery/macros/utils/safe_cast.sql b/dbt/include/bigquery/macros/utils/safe_cast.sql
index ac62bb050..ec312af11 100644
--- a/dbt/include/bigquery/macros/utils/safe_cast.sql
+++ b/dbt/include/bigquery/macros/utils/safe_cast.sql
@@ -1,3 +1,27 @@
 {% macro bigquery__safe_cast(field, type) %}
+{%- if type.lower().startswith('array') and field is iterable and (field is not string and field is not mapping) and field | length > 0 -%}
+    {#-- Extract nested type from 'array<nested_type>' --#}
+    {% set nested_type = type.lower()[6:-1] %}
+    {#-- BigQuery does not support direct casts to arrays. instead, each element must be cast individually + reaggregated into an array --#}
+    {%- if cast_from_string_unsupported_for(nested_type)  %}
+        (select array_agg(safe_cast(i as {{ nested_type }})) from unnest([
+            {%- for nested_field in field %}
+                {{ nested_field.strip('"').strip("'") }}{{ ',' if not loop.last }}
+            {%- endfor %}
+        ]) i)
+    {%- else -%}
+        (select array_agg(safe_cast(i as {{nested_type}})) from unnest({{field}}) i)
+    {%- endif -%}
+
+{%- elif type.lower() == 'json' and field is mapping -%}
+    safe_cast(json {{ dbt.string_literal(tojson(field)) }} as json)
+{%- elif cast_from_string_unsupported_for(type) and field is string -%}
+    safe_cast({{field.strip('"').strip("'")}} as {{type}})
+{%- else -%}
     safe_cast({{field}} as {{type}})
+{%- endif -%}
+{% endmacro %}
+
+{% macro cast_from_string_unsupported_for(type) %}
+    {{ return(type.lower().startswith('struct') or type.lower() == 'geography') }}
 {% endmacro %}
diff --git a/tests/functional/adapter/unit_testing/test_unit_testing.py b/tests/functional/adapter/unit_testing/test_unit_testing.py
new file mode 100644
index 000000000..f4d4ef1e2
--- /dev/null
+++ b/tests/functional/adapter/unit_testing/test_unit_testing.py
@@ -0,0 +1,64 @@
+import pytest
+from dbt.tests.adapter.unit_testing.test_types import BaseUnitTestingTypes
+from dbt.tests.adapter.unit_testing.test_case_insensitivity import BaseUnitTestCaseInsensivity
+from dbt.tests.adapter.unit_testing.test_invalid_input import BaseUnitTestInvalidInput
+
+
+class TestBigQueryUnitTestingTypes(BaseUnitTestingTypes):
+    @pytest.fixture
+    def data_types(self):
+        # sql_value, yaml_value
+        return [
+            ["1", "1"],
+            ["'1'", "1"],
+            ["cast('true' as boolean)", "true"],
+            ["1.0", "1.0"],
+            ["'string value'", "string value"],
+            ["cast(1.0 as numeric)", "1.0"],
+            ["cast(1 as bigint)", 1],
+            ["cast('2019-01-01' as date)", "2019-01-01"],
+            ["cast('2013-11-03 00:00:00-07' as timestamp)", "2013-11-03 00:00:00-07"],
+            ["st_geogpoint(75, 45)", "'st_geogpoint(75, 45)'"],
+            # arrays
+            ["cast(['a','b','c'] as array<string>)", "['a','b','c']"],
+            ["cast([1,2,3] as array<int>)", "[1,2,3]"],
+            ["cast([true,true,false] as array<bool>)", "[true,true,false]"],
+            # array of date
+            ["[date '2019-01-01']", "['2020-01-01']"],
+            ["[date '2019-01-01']", "[]"],
+            ["[date '2019-01-01']", "null"],
+            # array of timestamp
+            ["[timestamp '2019-01-01']", "['2020-01-01']"],
+            ["[timestamp '2019-01-01']", "[]"],
+            ["[timestamp '2019-01-01']", "null"],
+            # json
+            [
+                """json '{"name": "Cooper", "forname": "Alice"}'""",
+                """{"name": "Cooper", "forname": "Alice"}""",
+            ],
+            ["""json '{"name": "Cooper", "forname": "Alice"}'""", "{}"],
+            # structs
+            ["struct('Isha' as name, 22 as age)", """'struct("Isha" as name, 22 as age)'"""],
+            [
+                "struct('Kipketer' AS name, [23.2, 26.1, 27.3, 29.4] AS laps)",
+                """'struct("Kipketer" AS name, [23.2, 26.1, 27.3, 29.4] AS laps)'""",
+            ],
+            # struct of struct
+            [
+                "struct(struct(1 as id, 'blue' as color) as my_struct)",
+                """'struct(struct(1 as id, "blue" as color) as my_struct)'""",
+            ],
+            # array of struct
+            [
+                "[struct(st_geogpoint(75, 45) as my_point), struct(st_geogpoint(75, 35) as my_point)]",
+                "['struct(st_geogpoint(75, 45) as my_point)', 'struct(st_geogpoint(75, 35) as my_point)']",
+            ],
+        ]
+
+
+class TestBigQueryUnitTestCaseInsensitivity(BaseUnitTestCaseInsensivity):
+    pass
+
+
+class TestBigQueryUnitTestInvalidInput(BaseUnitTestInvalidInput):
+    pass

From 34eadae005afbd54442e34465d523875ed326264 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 14 Feb 2024 12:55:47 -0500
Subject: [PATCH 765/860] Implementation of metadata-based freshness (#1060)

* changelog

* turn on metadata-based source freshness capability

* add boundary test to confirm get_table raises an error properly

* add metadata-based source freshness for a relation

* remove unnecessary test setup

* remove unnecessary fixture from test

* update from main
---
 .../unreleased/Features-20231218-155409.yaml  |  6 ++++
 dbt/adapters/bigquery/impl.py                 | 31 +++++++++++++++++++
 .../relation_configs/_materialized_view.py    |  4 +--
 .../bigquery/relation_configs/_partition.py   |  2 +-
 tests/boundary/test_bigquery_sdk.py           | 18 +++++++++++
 .../adapter/sources_freshness_tests/files.py  | 23 ++++++++++++++
 .../test_get_relation_last_modified.py        | 30 ++++++++++++++++++
 7 files changed, 111 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Features-20231218-155409.yaml
 create mode 100644 tests/boundary/test_bigquery_sdk.py
 create mode 100644 tests/functional/adapter/sources_freshness_tests/files.py
 create mode 100644 tests/functional/adapter/sources_freshness_tests/test_get_relation_last_modified.py

diff --git a/.changes/unreleased/Features-20231218-155409.yaml b/.changes/unreleased/Features-20231218-155409.yaml
new file mode 100644
index 000000000..bc965b06f
--- /dev/null
+++ b/.changes/unreleased/Features-20231218-155409.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Add support for checking table-last-modified by metadata
+time: 2023-12-18T15:54:09.69635-05:00
+custom:
+  Author: mikealfare
+  Issue: "938"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 3e394ad26..eb2aec175 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -1,4 +1,5 @@
 from dataclasses import dataclass
+from datetime import datetime
 import json
 import threading
 from multiprocessing.context import SpawnContext
@@ -20,9 +21,12 @@
     SchemaSearchMap,
     available,
 )
+from dbt.adapters.base.impl import FreshnessResponse
 from dbt.adapters.cache import _make_ref_key_dict  # type: ignore
+from dbt.adapters.capability import CapabilityDict, CapabilitySupport, Support, Capability
 import dbt_common.clients.agate_helper
 from dbt.adapters.contracts.connection import AdapterResponse
+from dbt.adapters.contracts.macros import MacroResolverProtocol
 from dbt_common.contracts.constraints import ColumnLevelConstraint, ConstraintType, ModelLevelConstraint  # type: ignore
 from dbt_common.dataclass_schema import dbtClassMixin
 from dbt.adapters.events.logging import AdapterLogger
@@ -36,6 +40,7 @@
 import google.cloud.bigquery
 from google.cloud.bigquery import AccessEntry, SchemaField, Table as BigQueryTable
 import google.cloud.exceptions
+import pytz
 
 from dbt.adapters.bigquery import BigQueryColumn, BigQueryConnectionManager
 from dbt.adapters.bigquery.column import get_nested_column_data_types
@@ -118,6 +123,12 @@ class BigQueryAdapter(BaseAdapter):
         ConstraintType.foreign_key: ConstraintSupport.NOT_ENFORCED,
     }
 
+    _capabilities: CapabilityDict = CapabilityDict(
+        {
+            Capability.TableLastModifiedMetadata: CapabilitySupport(support=Support.Full),
+        }
+    )
+
     def __init__(self, config, mp_context: SpawnContext) -> None:
         super().__init__(config, mp_context)
         self.connections: BigQueryConnectionManager = self.connections
@@ -709,6 +720,26 @@ def _get_catalog_schemas(self, relation_config: Iterable[RelationConfig]) -> Sch
                 )
         return result
 
+    def calculate_freshness_from_metadata(
+        self,
+        source: BaseRelation,
+        macro_resolver: Optional[MacroResolverProtocol] = None,
+    ) -> Tuple[Optional[AdapterResponse], FreshnessResponse]:
+        conn = self.connections.get_thread_connection()
+        client: google.cloud.bigquery.Client = conn.handle
+
+        table_ref = self.get_table_ref_from_relation(source)
+        table = client.get_table(table_ref)
+        snapshot = datetime.now(tz=pytz.UTC)
+
+        freshness = FreshnessResponse(
+            max_loaded_at=table.modified,
+            snapshotted_at=snapshot,
+            age=(snapshot - table.modified).total_seconds(),
+        )
+
+        return None, freshness
+
     @available.parse(lambda *a, **k: {})
     def get_common_options(
         self, config: Dict[str, Any], node: Dict[str, Any], temporary: bool = False
diff --git a/dbt/adapters/bigquery/relation_configs/_materialized_view.py b/dbt/adapters/bigquery/relation_configs/_materialized_view.py
index fd0c191c3..81ca6b3de 100644
--- a/dbt/adapters/bigquery/relation_configs/_materialized_view.py
+++ b/dbt/adapters/bigquery/relation_configs/_materialized_view.py
@@ -75,10 +75,10 @@ def parse_relation_config(cls, relation_config: RelationConfig) -> Dict[str, Any
         }
 
         # optional
-        if "partition_by" in relation_config.config:
+        if relation_config.config and "partition_by" in relation_config.config:
             config_dict.update({"partition": PartitionConfig.parse_model_node(relation_config)})
 
-        if "cluster_by" in relation_config.config:
+        if relation_config.config and "cluster_by" in relation_config.config:
             config_dict.update(
                 {"cluster": BigQueryClusterConfig.parse_relation_config(relation_config)}
             )
diff --git a/dbt/adapters/bigquery/relation_configs/_partition.py b/dbt/adapters/bigquery/relation_configs/_partition.py
index 8fe8bf5d6..97e695fc0 100644
--- a/dbt/adapters/bigquery/relation_configs/_partition.py
+++ b/dbt/adapters/bigquery/relation_configs/_partition.py
@@ -111,7 +111,7 @@ def parse_model_node(cls, relation_config: RelationConfig) -> Dict[str, Any]:
             This doesn't currently collect `time_ingestion_partitioning` and `copy_partitions`
             because this was built for materialized views, which do not support those settings.
         """
-        config_dict = relation_config.config.extra.get("partition_by")  # type: ignore
+        config_dict: Dict[str, str] = relation_config.config.extra.get("partition_by")  # type: ignore
         if "time_ingestion_partitioning" in config_dict:
             del config_dict["time_ingestion_partitioning"]
         if "copy_partitions" in config_dict:
diff --git a/tests/boundary/test_bigquery_sdk.py b/tests/boundary/test_bigquery_sdk.py
new file mode 100644
index 000000000..b8e6c9995
--- /dev/null
+++ b/tests/boundary/test_bigquery_sdk.py
@@ -0,0 +1,18 @@
+import pytest
+
+from dbt.tests.util import get_connection
+from google.cloud.bigquery import Client, DatasetReference, TableReference
+from google.api_core.exceptions import NotFound
+
+
+@pytest.mark.parametrize("table_name", ["this_table_does_not_exist"])
+def test_get_table_does_not_exist(project, table_name):
+    """
+    TODO: replace dbt project methods with direct connection instantiation
+    """
+    with get_connection(project.adapter) as conn:
+        client: Client = conn.handle
+        dataset_ref = DatasetReference(project.database, project.test_schema)
+        table_ref = TableReference(dataset_ref, table_name)
+        with pytest.raises(NotFound):
+            client.get_table(table_ref)
diff --git a/tests/functional/adapter/sources_freshness_tests/files.py b/tests/functional/adapter/sources_freshness_tests/files.py
new file mode 100644
index 000000000..eaca96648
--- /dev/null
+++ b/tests/functional/adapter/sources_freshness_tests/files.py
@@ -0,0 +1,23 @@
+SCHEMA_YML = """version: 2
+sources:
+  - name: test_source
+    freshness:
+      warn_after: {count: 10, period: hour}
+      error_after: {count: 1, period: day}
+    schema: "{{ env_var('DBT_GET_LAST_RELATION_TEST_SCHEMA') }}"
+    tables:
+      - name: test_source
+"""
+
+SEED_TEST_SOURCE_CSV = """
+id,name
+1,Martin
+2,Jeter
+3,Ruth
+4,Gehrig
+5,DiMaggio
+6,Torre
+7,Mantle
+8,Berra
+9,Maris
+""".strip()
diff --git a/tests/functional/adapter/sources_freshness_tests/test_get_relation_last_modified.py b/tests/functional/adapter/sources_freshness_tests/test_get_relation_last_modified.py
new file mode 100644
index 000000000..08e263edb
--- /dev/null
+++ b/tests/functional/adapter/sources_freshness_tests/test_get_relation_last_modified.py
@@ -0,0 +1,30 @@
+import os
+import pytest
+
+from dbt.tests.util import run_dbt
+
+from tests.functional.adapter.sources_freshness_tests import files
+
+
+class TestGetLastRelationModified:
+    @pytest.fixture(scope="class")
+    def seeds(self):
+        return {"test_source.csv": files.SEED_TEST_SOURCE_CSV}
+
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {"schema.yml": files.SCHEMA_YML}
+
+    @pytest.fixture(scope="class", autouse=True)
+    def setup(self, project):
+        # we need the schema name for the sources section
+        os.environ["DBT_GET_LAST_RELATION_TEST_SCHEMA"] = project.test_schema
+        run_dbt(["seed"])
+        yield
+        del os.environ["DBT_GET_LAST_RELATION_TEST_SCHEMA"]
+
+    def test_get_last_relation_modified(self, project):
+        results = run_dbt(["source", "freshness"])
+        assert len(results) == 1
+        result = results[0]
+        assert result.status == "pass"

From 62a8f80b7ae38bd7c1b51e47a81ad088a358c043 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 14 Feb 2024 15:36:50 -0500
Subject: [PATCH 766/860]  Implement relation filtering on get_catalog macro
 (#1063)

* changelog

* turn on get_catalog by relations

* add get_catalog_relations, break apart get_catalog for reuse in get_catalog_relations

* update from main
---
 .../unreleased/Features-20231219-201203.yaml  |   6 +
 dbt/adapters/bigquery/impl.py                 |   3 +-
 .../bigquery/relation_configs/_partition.py   |   2 +-
 dbt/include/bigquery/macros/catalog.sql       | 231 ------------------
 .../bigquery/macros/catalog/by_relation.sql   |  36 +++
 .../bigquery/macros/catalog/by_schema.sql     |  32 +++
 .../bigquery/macros/catalog/catalog.sql       | 177 ++++++++++++++
 7 files changed, 254 insertions(+), 233 deletions(-)
 create mode 100644 .changes/unreleased/Features-20231219-201203.yaml
 delete mode 100644 dbt/include/bigquery/macros/catalog.sql
 create mode 100644 dbt/include/bigquery/macros/catalog/by_relation.sql
 create mode 100644 dbt/include/bigquery/macros/catalog/by_schema.sql
 create mode 100644 dbt/include/bigquery/macros/catalog/catalog.sql

diff --git a/.changes/unreleased/Features-20231219-201203.yaml b/.changes/unreleased/Features-20231219-201203.yaml
new file mode 100644
index 000000000..eee3f1026
--- /dev/null
+++ b/.changes/unreleased/Features-20231219-201203.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Support limiting get_catalog by object name
+time: 2023-12-19T20:12:03.990725-05:00
+custom:
+  Author: mikealfare
+  Issue: "950"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index eb2aec175..2df35bc65 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -23,7 +23,7 @@
 )
 from dbt.adapters.base.impl import FreshnessResponse
 from dbt.adapters.cache import _make_ref_key_dict  # type: ignore
-from dbt.adapters.capability import CapabilityDict, CapabilitySupport, Support, Capability
+from dbt.adapters.capability import Capability, CapabilityDict, CapabilitySupport, Support
 import dbt_common.clients.agate_helper
 from dbt.adapters.contracts.connection import AdapterResponse
 from dbt.adapters.contracts.macros import MacroResolverProtocol
@@ -126,6 +126,7 @@ class BigQueryAdapter(BaseAdapter):
     _capabilities: CapabilityDict = CapabilityDict(
         {
             Capability.TableLastModifiedMetadata: CapabilitySupport(support=Support.Full),
+            Capability.SchemaMetadataByRelations: CapabilitySupport(support=Support.Full),
         }
     )
 
diff --git a/dbt/adapters/bigquery/relation_configs/_partition.py b/dbt/adapters/bigquery/relation_configs/_partition.py
index 97e695fc0..555aa3664 100644
--- a/dbt/adapters/bigquery/relation_configs/_partition.py
+++ b/dbt/adapters/bigquery/relation_configs/_partition.py
@@ -111,7 +111,7 @@ def parse_model_node(cls, relation_config: RelationConfig) -> Dict[str, Any]:
             This doesn't currently collect `time_ingestion_partitioning` and `copy_partitions`
             because this was built for materialized views, which do not support those settings.
         """
-        config_dict: Dict[str, str] = relation_config.config.extra.get("partition_by")  # type: ignore
+        config_dict: Dict[str, Any] = relation_config.config.extra.get("partition_by")  # type: ignore
         if "time_ingestion_partitioning" in config_dict:
             del config_dict["time_ingestion_partitioning"]
         if "copy_partitions" in config_dict:
diff --git a/dbt/include/bigquery/macros/catalog.sql b/dbt/include/bigquery/macros/catalog.sql
deleted file mode 100644
index 25166c7b4..000000000
--- a/dbt/include/bigquery/macros/catalog.sql
+++ /dev/null
@@ -1,231 +0,0 @@
-
-{% macro bigquery__get_catalog(information_schema, schemas) -%}
-
-  {%- if (schemas | length) == 0 -%}
-    {# Hopefully nothing cares about the columns we return when there are no rows #}
-    {%- set query  = "select 1 as id limit 0" -%}
-  {%- else -%}
-
-  {%- set query -%}
-    with materialized_views as (
-        select
-            table_catalog as project_id,
-            table_schema as dataset_id,
-            table_name as table_id
-        from {{ information_schema.replace(information_schema_view='MATERIALIZED_VIEWS') }}
-    ),
-    tables as (
-        select
-            tables.project_id as table_database,
-            tables.dataset_id as table_schema,
-            tables.table_id as original_table_name,
-
-            concat(tables.project_id, '.', tables.dataset_id, '.', tables.table_id) as relation_id,
-
-            tables.row_count,
-            tables.size_bytes as size_bytes,
-            case
-                when materialized_views.table_id is not null then 'materialized view'
-                when tables.type = 1 then 'table'
-                when tables.type = 2 then 'view'
-                else 'external'
-            end as table_type,
-
-            REGEXP_CONTAINS(tables.table_id, '^.+[0-9]{8}$') and coalesce(type, 0) = 1 as is_date_shard,
-            REGEXP_EXTRACT(tables.table_id, '^(.+)[0-9]{8}$') as shard_base_name,
-            REGEXP_EXTRACT(tables.table_id, '^.+([0-9]{8})$') as shard_name
-
-        from {{ information_schema.replace(information_schema_view='__TABLES__') }} tables
-        left join materialized_views
-            on materialized_views.project_id = tables.project_id
-            and materialized_views.dataset_id = tables.dataset_id
-            and materialized_views.table_id = tables.table_id
-        where (
-          {%- for schema in schemas -%}
-            upper(tables.dataset_id) = upper('{{ schema }}'){%- if not loop.last %} or {% endif -%}
-          {%- endfor -%}
-        )
-    ),
-
-    table_options as (
-        select
-            concat(table_catalog, '.', table_schema, '.', table_name) as relation_id,
-            JSON_VALUE(option_value) as table_comment
-
-        from {{ information_schema.replace(information_schema_view='TABLE_OPTIONS') }}
-        where option_name = 'description'
-    ),
-    extracted as (
-
-        select *,
-            case
-                when is_date_shard then shard_base_name
-                else original_table_name
-            end as table_name
-
-        from tables
-
-    ),
-
-    unsharded_tables as (
-
-        select
-            table_database,
-            table_schema,
-            table_name,
-            coalesce(table_type, 'external') as table_type,
-            is_date_shard,
-
-            struct(
-                min(shard_name) as shard_min,
-                max(shard_name) as shard_max,
-                count(*) as shard_count
-            ) as table_shards,
-
-            sum(size_bytes) as size_bytes,
-            sum(row_count) as row_count,
-
-            max(relation_id) as relation_id
-
-        from extracted
-        group by 1,2,3,4,5
-
-    ),
-
-    info_schema_columns as (
-
-        select
-            concat(table_catalog, '.', table_schema, '.', table_name) as relation_id,
-            table_catalog as table_database,
-            table_schema,
-            table_name,
-
-            -- use the "real" column name from the paths query below
-            column_name as base_column_name,
-            ordinal_position as column_index,
-
-            is_partitioning_column,
-            clustering_ordinal_position
-
-        from {{ information_schema.replace(information_schema_view='COLUMNS') }}
-        where ordinal_position is not null
-
-    ),
-
-    info_schema_column_paths as (
-
-        select
-            concat(table_catalog, '.', table_schema, '.', table_name) as relation_id,
-            field_path as column_name,
-            data_type as column_type,
-            column_name as base_column_name,
-            description as column_comment
-
-        from {{ information_schema.replace(information_schema_view='COLUMN_FIELD_PATHS') }}
-
-    ),
-
-    columns as (
-
-        select * except (base_column_name)
-        from info_schema_columns
-        join info_schema_column_paths using (relation_id, base_column_name)
-
-    ),
-
-    column_stats as (
-
-        select
-            table_database,
-            table_schema,
-            table_name,
-            max(relation_id) as relation_id,
-            max(case when is_partitioning_column = 'YES' then 1 else 0 end) = 1 as is_partitioned,
-            max(case when is_partitioning_column = 'YES' then column_name else null end) as partition_column,
-            max(case when clustering_ordinal_position is not null then 1 else 0 end) = 1 as is_clustered,
-            array_to_string(
-                array_agg(
-                    case
-                        when clustering_ordinal_position is not null then column_name
-                        else null
-                    end ignore nulls
-                    order by clustering_ordinal_position
-                ), ', '
-            ) as clustering_columns
-
-        from columns
-        group by 1,2,3
-
-    )
-
-    select
-        unsharded_tables.table_database,
-        unsharded_tables.table_schema,
-        case
-            when is_date_shard then concat(unsharded_tables.table_name, '*')
-            else unsharded_tables.table_name
-        end as table_name,
-        unsharded_tables.table_type,
-        table_options.table_comment,
-
-        -- coalesce name and type for External tables - these columns are not
-        -- present in the COLUMN_FIELD_PATHS resultset
-        coalesce(columns.column_name, '<unknown>') as column_name,
-        -- invent a row number to account for nested fields -- BQ does
-        -- not treat these nested properties as independent fields
-        row_number() over (
-            partition by relation_id
-            order by columns.column_index, columns.column_name
-        ) as column_index,
-        coalesce(columns.column_type, '<unknown>') as column_type,
-        columns.column_comment,
-
-        'Shard count' as `stats__date_shards__label`,
-        table_shards.shard_count as `stats__date_shards__value`,
-        'The number of date shards in this table' as `stats__date_shards__description`,
-        is_date_shard as `stats__date_shards__include`,
-
-        'Shard (min)' as `stats__date_shard_min__label`,
-        table_shards.shard_min as `stats__date_shard_min__value`,
-        'The first date shard in this table' as `stats__date_shard_min__description`,
-        is_date_shard as `stats__date_shard_min__include`,
-
-        'Shard (max)' as `stats__date_shard_max__label`,
-        table_shards.shard_max as `stats__date_shard_max__value`,
-        'The last date shard in this table' as `stats__date_shard_max__description`,
-        is_date_shard as `stats__date_shard_max__include`,
-
-        '# Rows' as `stats__num_rows__label`,
-        row_count as `stats__num_rows__value`,
-        'Approximate count of rows in this table' as `stats__num_rows__description`,
-        (unsharded_tables.table_type = 'table') as `stats__num_rows__include`,
-
-        'Approximate Size' as `stats__num_bytes__label`,
-        size_bytes as `stats__num_bytes__value`,
-        'Approximate size of table as reported by BigQuery' as `stats__num_bytes__description`,
-        (unsharded_tables.table_type = 'table') as `stats__num_bytes__include`,
-
-        'Partitioned By' as `stats__partitioning_type__label`,
-        partition_column as `stats__partitioning_type__value`,
-        'The partitioning column for this table' as `stats__partitioning_type__description`,
-        is_partitioned as `stats__partitioning_type__include`,
-
-        'Clustered By' as `stats__clustering_fields__label`,
-        clustering_columns as `stats__clustering_fields__value`,
-        'The clustering columns for this table' as `stats__clustering_fields__description`,
-        is_clustered as `stats__clustering_fields__include`
-
-    -- join using relation_id (an actual relation, not a shard prefix) to make
-    -- sure that column metadata is picked up through the join. This will only
-    -- return the column information for the "max" table in a date-sharded table set
-    from unsharded_tables
-    left join table_options using (relation_id)
-    left join columns using (relation_id)
-    left join column_stats using (relation_id)
-  {%- endset -%}
-
-  {%- endif -%}
-
-  {{ return(run_query(query)) }}
-
-{%- endmacro %}
diff --git a/dbt/include/bigquery/macros/catalog/by_relation.sql b/dbt/include/bigquery/macros/catalog/by_relation.sql
new file mode 100644
index 000000000..adaa740f6
--- /dev/null
+++ b/dbt/include/bigquery/macros/catalog/by_relation.sql
@@ -0,0 +1,36 @@
+{% macro bigquery__get_catalog_relations(information_schema, relations) -%}
+
+    {%- if (relations | length) == 0 -%}
+        {# Hopefully nothing cares about the columns we return when there are no rows #}
+        {%- set query = "select 1 as id limit 0" -%}
+
+    {%- else -%}
+        {%- set query -%}
+            with
+                table_shards_stage as ({{ _bigquery__get_table_shards_sql(information_schema) }}),
+                table_shards as (
+                    select * from table_shards_stage
+                    where (
+                        {%- for relation in relations -%}
+                            (
+                                upper(table_schema) = upper('{{ relation.schema }}')
+                            and upper(table_name) = upper('{{ relation.identifier }}')
+                            )
+                            {%- if not loop.last %} or {% endif -%}
+                        {%- endfor -%}
+                    )
+                ),
+                tables as ({{ _bigquery__get_tables_sql() }}),
+                table_stats as ({{ _bigquery__get_table_stats_sql() }}),
+
+                columns as ({{ _bigquery__get_columns_sql(information_schema) }}),
+                column_stats as ({{ _bigquery__get_column_stats_sql() }})
+
+            {{ _bigquery__get_extended_catalog_sql() }}
+        {%- endset -%}
+
+    {%- endif -%}
+
+    {{ return(run_query(query)) }}
+
+{%- endmacro %}
diff --git a/dbt/include/bigquery/macros/catalog/by_schema.sql b/dbt/include/bigquery/macros/catalog/by_schema.sql
new file mode 100644
index 000000000..0d36f2b84
--- /dev/null
+++ b/dbt/include/bigquery/macros/catalog/by_schema.sql
@@ -0,0 +1,32 @@
+{% macro bigquery__get_catalog(information_schema, schemas) -%}
+
+    {%- if (schemas | length) == 0 -%}
+        {# Hopefully nothing cares about the columns we return when there are no rows #}
+        {%- set query = "select 1 as id limit 0" -%}
+
+    {%- else -%}
+        {%- set query -%}
+            with
+                table_shards as (
+                    {{ _bigquery__get_table_shards_sql(information_schema) }}
+                    where (
+                        {%- for schema in schemas -%}
+                            upper(tables.dataset_id) = upper('{{ schema }}')
+                            {%- if not loop.last %} or {% endif -%}
+                        {%- endfor -%}
+                    )
+                ),
+                tables as ({{ _bigquery__get_tables_sql() }}),
+                table_stats as ({{ _bigquery__get_table_stats_sql() }}),
+
+                columns as ({{ _bigquery__get_columns_sql(information_schema) }}),
+                column_stats as ({{ _bigquery__get_column_stats_sql() }})
+
+            {{ _bigquery__get_extended_catalog_sql() }}
+        {%- endset -%}
+
+    {%- endif -%}
+
+    {{ return(run_query(query)) }}
+
+{%- endmacro %}
diff --git a/dbt/include/bigquery/macros/catalog/catalog.sql b/dbt/include/bigquery/macros/catalog/catalog.sql
new file mode 100644
index 000000000..de16f82bf
--- /dev/null
+++ b/dbt/include/bigquery/macros/catalog/catalog.sql
@@ -0,0 +1,177 @@
+{% macro _bigquery__get_table_shards_sql(information_schema) %}
+    select
+        tables.project_id as table_catalog,
+        tables.dataset_id as table_schema,
+        coalesce(REGEXP_EXTRACT(tables.table_id, '^(.+)[0-9]{8}$'), tables.table_id) as table_name,
+        tables.table_id as shard_name,
+        REGEXP_EXTRACT(tables.table_id, '^.+([0-9]{8})$') as shard_index,
+        REGEXP_CONTAINS(tables.table_id, '^.+[0-9]{8}$') and tables.type = 1 as is_date_shard,
+        case
+            when materialized_views.table_name is not null then 'materialized view'
+            when tables.type = 1 then 'table'
+            when tables.type = 2 then 'view'
+            else 'external'
+        end as table_type,
+        tables.type = 1 as is_table,
+        JSON_VALUE(table_description.option_value) as table_comment,
+        tables.size_bytes,
+        tables.row_count
+    from {{ information_schema.replace(information_schema_view='__TABLES__') }} tables
+    left join {{ information_schema.replace(information_schema_view='MATERIALIZED_VIEWS') }} materialized_views
+        on materialized_views.table_catalog = tables.project_id
+        and materialized_views.table_schema = tables.dataset_id
+        and materialized_views.table_name = tables.table_id
+    left join {{ information_schema.replace(information_schema_view='TABLE_OPTIONS') }} table_description
+        on table_description.table_catalog = tables.project_id
+        and table_description.table_schema = tables.dataset_id
+        and table_description.table_name = tables.table_id
+        and table_description.option_name = 'description'
+{% endmacro %}
+
+
+{% macro _bigquery__get_tables_sql() %}
+    select distinct
+        table_catalog,
+        table_schema,
+        table_name,
+        is_date_shard,
+        table_type,
+        is_table,
+        table_comment
+    from table_shards
+{% endmacro %}
+
+
+{% macro _bigquery__get_table_stats_sql() %}
+    select
+        table_catalog,
+        table_schema,
+        table_name,
+        max(shard_name) as latest_shard_name,
+        min(shard_index) as shard_min,
+        max(shard_index) as shard_max,
+        count(shard_index) as shard_count,
+        sum(size_bytes) as size_bytes,
+        sum(row_count) as row_count
+    from table_shards
+    group by 1, 2, 3
+{% endmacro %}
+
+
+{% macro _bigquery__get_columns_sql(information_schema) %}
+    select
+        columns.table_catalog,
+        columns.table_schema,
+        columns.table_name as shard_name,
+        coalesce(paths.field_path, '<unknown>') as column_name,
+        -- invent a row number to account for nested fields
+        -- BQ does not treat these nested properties as independent fields
+        row_number() over (
+            partition by
+                columns.table_catalog,
+                columns.table_schema,
+                columns.table_name
+            order by
+                columns.ordinal_position,
+                paths.field_path
+        ) as column_index,
+        coalesce(paths.data_type, '<unknown>') as column_type,
+        paths.description as column_comment,
+        case when columns.is_partitioning_column = 'YES' then 1 else 0 end as is_partitioning_column,
+        case when columns.is_partitioning_column = 'YES' then paths.field_path end as partition_column,
+        case when columns.clustering_ordinal_position is not null then 1 else 0 end as is_clustering_column,
+        case when columns.clustering_ordinal_position is not null then paths.field_path end as cluster_column,
+        columns.clustering_ordinal_position
+    from {{ information_schema.replace(information_schema_view='COLUMNS') }} columns
+    join {{ information_schema.replace(information_schema_view='COLUMN_FIELD_PATHS') }} paths
+        on paths.table_catalog = columns.table_catalog
+        and paths.table_schema = columns.table_schema
+        and paths.table_name = columns.table_name
+        and paths.column_name = columns.column_name
+    where columns.ordinal_position is not null
+{% endmacro %}
+
+
+{% macro _bigquery__get_column_stats_sql() %}
+    select
+        table_catalog,
+        table_schema,
+        shard_name,
+        max(is_partitioning_column) = 1 as is_partitioned,
+        max(partition_column) as partition_column,
+        max(is_clustering_column) = 1 as is_clustered,
+        array_to_string(
+            array_agg(
+                cluster_column ignore nulls
+                order by clustering_ordinal_position
+            ), ', '
+        ) as clustering_columns
+    from columns
+    group by 1, 2, 3
+{% endmacro %}
+
+
+{% macro _bigquery__get_extended_catalog_sql() %}
+    select
+        tables.table_catalog as table_database,
+        tables.table_schema,
+        case
+            when tables.is_date_shard then concat(tables.table_name, '*')
+            else tables.table_name
+        end as table_name,
+        tables.table_type,
+        tables.table_comment,
+        columns.column_name,
+        columns.column_index,
+        columns.column_type,
+        columns.column_comment,
+
+        'Shard count' as `stats__date_shards__label`,
+        table_stats.shard_count as `stats__date_shards__value`,
+        'The number of date shards in this table' as `stats__date_shards__description`,
+        tables.is_date_shard as `stats__date_shards__include`,
+
+        'Shard (min)' as `stats__date_shard_min__label`,
+        table_stats.shard_min as `stats__date_shard_min__value`,
+        'The first date shard in this table' as `stats__date_shard_min__description`,
+        tables.is_date_shard as `stats__date_shard_min__include`,
+
+        'Shard (max)' as `stats__date_shard_max__label`,
+        table_stats.shard_max as `stats__date_shard_max__value`,
+        'The last date shard in this table' as `stats__date_shard_max__description`,
+        tables.is_date_shard as `stats__date_shard_max__include`,
+
+        '# Rows' as `stats__num_rows__label`,
+        table_stats.row_count as `stats__num_rows__value`,
+        'Approximate count of rows in this table' as `stats__num_rows__description`,
+        tables.is_table as `stats__num_rows__include`,
+
+        'Approximate Size' as `stats__num_bytes__label`,
+        table_stats.size_bytes as `stats__num_bytes__value`,
+        'Approximate size of table as reported by BigQuery' as `stats__num_bytes__description`,
+        tables.is_table as `stats__num_bytes__include`,
+
+        'Partitioned By' as `stats__partitioning_type__label`,
+        column_stats.partition_column as `stats__partitioning_type__value`,
+        'The partitioning column for this table' as `stats__partitioning_type__description`,
+        column_stats.is_partitioned as `stats__partitioning_type__include`,
+
+        'Clustered By' as `stats__clustering_fields__label`,
+        column_stats.clustering_columns as `stats__clustering_fields__value`,
+        'The clustering columns for this table' as `stats__clustering_fields__description`,
+        column_stats.is_clustered as `stats__clustering_fields__include`
+
+    from tables
+    join table_stats
+        on table_stats.table_catalog = tables.table_catalog
+        and table_stats.table_schema = tables.table_schema
+        and table_stats.table_name = tables.table_name
+    left join column_stats
+        on column_stats.table_catalog = tables.table_catalog
+        and column_stats.table_schema = tables.table_schema
+        and column_stats.shard_name = table_stats.latest_shard_name
+    left join columns
+        on columns.table_catalog = tables.table_catalog
+        and columns.table_schema = tables.table_schema
+        and columns.shard_name = table_stats.latest_shard_name
+{% endmacro %}

From f4cf22c336d40859bba18b73784e638ce53e15df Mon Sep 17 00:00:00 2001
From: tnk-ysk <61967017+tnk-ysk@users.noreply.github.com>
Date: Sat, 17 Feb 2024 02:16:21 +0900
Subject: [PATCH 767/860] Fix merge fails when partition is required (#986)

* Fix merge fails when partition is required

* Fix condition for avoid_require_partition

* Add changes

* Add test for require_partition_filter

* Refactoring avoid require_partition_filter

* Fix time_ingestion_partitioning

* Refactoring avoid require_partition_filter

* Update .changes/unreleased/Fixes-20231105-143145.yaml

---------

Co-authored-by: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .../unreleased/Fixes-20231105-143145.yaml     |  6 ++
 .../incremental_strategy/common.sql           | 20 +++++++
 .../incremental_strategy/merge.sql            |  8 ++-
 .../incremental_strategy_fixtures.py          | 57 +++++++++++++++++++
 .../test_incremental_strategies.py            |  7 ++-
 5 files changed, 95 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20231105-143145.yaml

diff --git a/.changes/unreleased/Fixes-20231105-143145.yaml b/.changes/unreleased/Fixes-20231105-143145.yaml
new file mode 100644
index 000000000..fd404e395
--- /dev/null
+++ b/.changes/unreleased/Fixes-20231105-143145.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: In incremental models, add dummy merge condition on source partition column when partition is required
+time: 2023-11-05T14:31:45.869783+09:00
+custom:
+  Author: tnk-ysk
+  Issue: "792"
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
index 9d71ba7c0..1c02f4912 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
@@ -11,3 +11,23 @@
   {%- endif -%}
 
 {% endmacro %}
+
+{% macro predicate_for_avoid_require_partition_filter(target='DBT_INTERNAL_DEST') %}
+
+    {%- set raw_partition_by = config.get('partition_by', none) -%}
+    {%- set partition_config = adapter.parse_partition_by(raw_partition_by) -%}
+    {%- set predicate = none -%}
+
+    {% if partition_config and config.get('require_partition_filter') -%}
+        {%- set partition_field = partition_config.time_partitioning_field() if partition_config.time_ingestion_partitioning else partition_config.field -%}
+        {% set predicate %}
+            (
+                `{{ target }}`.`{{ partition_field }}` is null
+                or `{{ target }}`.`{{ partition_field }}` is not null
+            )
+        {% endset %}
+    {%- endif -%}
+
+    {{ return(predicate) }}
+
+{% endmacro %}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
index 90af66f52..a204caed9 100644
--- a/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
@@ -21,7 +21,13 @@
         {%- endif -%}
     {%- endset -%}
 
-    {% set build_sql = get_merge_sql(target_relation, source_sql, unique_key, dest_columns, incremental_predicates) %}
+    {%- set predicates = [] if incremental_predicates is none else [] + incremental_predicates -%}
+    {%- set avoid_require_partition_filter = predicate_for_avoid_require_partition_filter() -%}
+    {%- if avoid_require_partition_filter is not none -%}
+        {% do predicates.append(avoid_require_partition_filter) %}
+    {%- endif -%}
+
+    {% set build_sql = get_merge_sql(target_relation, source_sql, unique_key, dest_columns, predicates) %}
 
     {{ return(build_sql) }}
 
diff --git a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
index a8f0004c5..17391b48d 100644
--- a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
+++ b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
@@ -90,6 +90,63 @@
 {% endif %}
 """.lstrip()
 
+merge_time_with_require_partition_sql = """
+{{
+    config(
+        materialized="incremental",
+        unique_key="id",
+        cluster_by="id",
+        partition_by={
+            "field": "date_time",
+            "data_type": "dateTime"
+        },
+        post_hook="
+            create or replace view `{{ schema }}.incremental_merge_time_with_require_partition_view`
+            as select * from {{ this }} where date_time is null or date_time is not null
+        ",
+        require_partition_filter=true
+    )
+}}
+
+
+
+with data as (
+
+    {% if not is_incremental() %}
+
+        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 4 as id, cast('2020-01-01' as datetime) as date_time
+
+    {% else %}
+
+        select 1 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 2 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 3 as id, cast('2020-01-01' as datetime) as date_time union all
+        select 4 as id, cast('2020-01-02' as datetime) as date_time union all
+        select 5 as id, cast('2020-01-02' as datetime) as date_time union all
+        select 6 as id, cast('2020-01-02' as datetime) as date_time
+
+    {% endif %}
+
+)
+
+select * from data
+
+{% if is_incremental() %}
+where date_time > (
+    select max(date_time)
+    from {{ this }}
+    where (
+        date_time is null
+        or date_time is not null
+    )
+)
+
+{% endif %}
+""".lstrip()
+
 overwrite_date_sql = """
 {{
     config(
diff --git a/tests/functional/adapter/incremental/test_incremental_strategies.py b/tests/functional/adapter/incremental/test_incremental_strategies.py
index b3a51ad09..1a339d601 100644
--- a/tests/functional/adapter/incremental/test_incremental_strategies.py
+++ b/tests/functional/adapter/incremental/test_incremental_strategies.py
@@ -17,6 +17,7 @@
 from tests.functional.adapter.incremental.incremental_strategy_fixtures import (
     merge_range_sql,
     merge_time_sql,
+    merge_time_with_require_partition_sql,
     overwrite_date_sql,
     overwrite_day_sql,
     overwrite_day_with_copy_partitions_sql,
@@ -39,6 +40,7 @@ def models(self):
         return {
             "incremental_merge_range.sql": merge_range_sql,
             "incremental_merge_time.sql": merge_time_sql,
+            "incremental_merge_time_with_require_partition.sql": merge_time_with_require_partition_sql,
             "incremental_overwrite_date.sql": overwrite_date_sql,
             "incremental_overwrite_day.sql": overwrite_day_sql,
             "incremental_overwrite_day_with_copy_partitions.sql": overwrite_day_with_copy_partitions_sql,
@@ -65,13 +67,14 @@ def seeds(self):
     def test__bigquery_assert_incremental_configurations_apply_the_right_strategy(self, project):
         run_dbt(["seed"])
         results = run_dbt()
-        assert len(results) == 11
+        assert len(results) == 12
 
         results = run_dbt()
-        assert len(results) == 11
+        assert len(results) == 12
         incremental_strategies = [
             ("incremental_merge_range", "merge_expected"),
             ("incremental_merge_time", "merge_expected"),
+            ("incremental_merge_time_with_require_partition_view", "merge_expected"),
             ("incremental_overwrite_time", "incremental_overwrite_time_expected"),
             ("incremental_overwrite_date", "incremental_overwrite_date_expected"),
             ("incremental_overwrite_partitions", "incremental_overwrite_date_expected"),

From d904740ededc11dbdaa0faa98d7dc8fb18381b5a Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Fri, 16 Feb 2024 14:11:12 -0800
Subject: [PATCH 768/860] Remove refresh token field from connection keys
 (#1105)

* Remove refresh token field from connection keys

* Add changie

---------

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20240216-133319.yaml | 6 ++++++
 dbt/adapters/bigquery/connections.py           | 1 -
 2 files changed, 6 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20240216-133319.yaml

diff --git a/.changes/unreleased/Fixes-20240216-133319.yaml b/.changes/unreleased/Fixes-20240216-133319.yaml
new file mode 100644
index 000000000..cb8a2156e
--- /dev/null
+++ b/.changes/unreleased/Fixes-20240216-133319.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: remove `token` field from connection keys
+time: 2024-02-16T13:33:19.524482-08:00
+custom:
+  Author: versusfacit
+  Issue: "1105"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index c74effcdc..f9bb791d2 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -208,7 +208,6 @@ def _connection_keys(self):
             "job_execution_timeout_seconds",
             "keyfile",
             "timeout_seconds",
-            "refresh_token",
             "client_id",
             "token_uri",
             "dataproc_region",

From 6b48ec89c1d10124dd31ebaf2e8c2f5aae4ec68a Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Tue, 20 Feb 2024 10:54:46 -0800
Subject: [PATCH 769/860] remove _json_from_json patch (#1061)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* remove _json_from_json patch

* fix import

* remove google-apis install from dev-requirements.txt

---------

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20231219-153446.yaml     |  6 ++++++
 dbt/adapters/bigquery/connections.py               | 13 ++-----------
 tests/functional/adapter/dbt_show/test_dbt_show.py |  8 ++++++++
 3 files changed, 16 insertions(+), 11 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20231219-153446.yaml

diff --git a/.changes/unreleased/Fixes-20231219-153446.yaml b/.changes/unreleased/Fixes-20231219-153446.yaml
new file mode 100644
index 000000000..44a858eb8
--- /dev/null
+++ b/.changes/unreleased/Fixes-20231219-153446.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: remove json patch to leverage bigquery-python improvement
+time: 2023-12-19T15:34:46.843931-08:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "1055"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index f9bb791d2..0f2f70f74 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -17,7 +17,7 @@
 
 import google.auth
 import google.auth.exceptions
-import google.cloud.bigquery as bigquery
+import google.cloud.bigquery
 import google.cloud.exceptions
 from google.api_core import retry, client_info
 from google.auth import impersonated_credentials
@@ -33,6 +33,7 @@
     DbtRuntimeError,
     DbtConfigError,
 )
+
 from dbt_common.exceptions import DbtDatabaseError
 from dbt.adapters.exceptions.connection import FailedToConnectError
 from dbt.adapters.base import BaseConnectionManager
@@ -63,16 +64,6 @@
 )
 
 
-# Override broken json deserializer for dbt show --inline
-# can remove once this is fixed: https://github.com/googleapis/python-bigquery/issues/1500
-def _json_from_json(value, _):
-    """NOOP string -> string coercion"""
-    return json.loads(value)
-
-
-bigquery._helpers._CELLDATA_FROM_JSON["JSON"] = _json_from_json
-
-
 @lru_cache()
 def get_bigquery_defaults(scopes=None) -> Tuple[Any, Optional[str]]:
     """
diff --git a/tests/functional/adapter/dbt_show/test_dbt_show.py b/tests/functional/adapter/dbt_show/test_dbt_show.py
index 203d7031b..acb54cc47 100644
--- a/tests/functional/adapter/dbt_show/test_dbt_show.py
+++ b/tests/functional/adapter/dbt_show/test_dbt_show.py
@@ -24,6 +24,10 @@
     limit 5
     """
 
+model_with_null_json_struct = """
+    select (struct<k json>(null)) as null_struct
+"""
+
 
 class TestBigQueryShowLimit(BaseShowLimit):
     pass
@@ -40,7 +44,11 @@ class TestBigQueryShowSqlWorksWithJSONStruct:
     def models(self):
         return {
             "json_struct_model.sql": model_with_json_struct,
+            "null_json_struct_model.sql": model_with_null_json_struct,
         }
 
     def test_sql_header(self, project):
         run_dbt(["show", "--select", "json_struct_model"])
+
+    def test_show_with_null_json_struct(self, project):
+        run_dbt(["show", "--select", "null_json_struct_model"])

From 5501cd34b12965e0380952bdbb65fd52f49b49f5 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Tue, 20 Feb 2024 14:54:42 -0800
Subject: [PATCH 770/860] Remove custom query job async timeout logic (#1109)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* use dbt-common main to test against

* emove custom query job async timeout logic

* remove unneeded unit test

---------

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .../unreleased/Fixes-20240219-103324.yaml     |  6 +++++
 dbt/adapters/bigquery/connections.py          | 22 +----------------
 tests/functional/test_job_timeout.py          |  3 ++-
 .../unit/test_bigquery_connection_manager.py  | 24 -------------------
 4 files changed, 9 insertions(+), 46 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20240219-103324.yaml

diff --git a/.changes/unreleased/Fixes-20240219-103324.yaml b/.changes/unreleased/Fixes-20240219-103324.yaml
new file mode 100644
index 000000000..16906db85
--- /dev/null
+++ b/.changes/unreleased/Fixes-20240219-103324.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Remove custom query job async timeout logic as it has been fixed in bigquery-python
+time: 2024-02-19T10:33:24.3385-08:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "1081"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 0f2f70f74..1e4708f0b 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -1,5 +1,3 @@
-import asyncio
-import functools
 import json
 import re
 from contextlib import contextmanager
@@ -740,25 +738,7 @@ def _query_and_results(
                 self._bq_job_link(query_job.location, query_job.project, query_job.job_id)
             )
 
-        # only use async logic if user specifies a timeout
-        if job_execution_timeout:
-            loop = asyncio.new_event_loop()
-            future_iterator = asyncio.wait_for(
-                loop.run_in_executor(None, functools.partial(query_job.result, max_results=limit)),
-                timeout=job_execution_timeout,
-            )
-
-            try:
-                iterator = loop.run_until_complete(future_iterator)
-            except asyncio.TimeoutError:
-                query_job.cancel()
-                raise DbtRuntimeError(
-                    f"Query exceeded configured timeout of {job_execution_timeout}s"
-                )
-            finally:
-                loop.close()
-        else:
-            iterator = query_job.result(max_results=limit)
+        iterator = query_job.result(max_results=limit, timeout=job_execution_timeout)
         return query_job, iterator
 
     def _retry_and_handle(self, msg, conn, fn):
diff --git a/tests/functional/test_job_timeout.py b/tests/functional/test_job_timeout.py
index be559e816..57172e133 100644
--- a/tests/functional/test_job_timeout.py
+++ b/tests/functional/test_job_timeout.py
@@ -59,4 +59,5 @@ def profiles_config_update(self, dbt_profile_target):
 
     def test_job_timeout(self, project):
         result = run_dbt(["run"], expect_pass=False)  # project setup will fail
-        assert f"Query exceeded configured timeout of {_SHORT_TIMEOUT}s" in result[0].message
+        expected_error = f"Operation did not complete within the designated timeout of {_SHORT_TIMEOUT} seconds."
+        assert expected_error in result[0].message
diff --git a/tests/unit/test_bigquery_connection_manager.py b/tests/unit/test_bigquery_connection_manager.py
index 6bb89ed36..564601b2f 100644
--- a/tests/unit/test_bigquery_connection_manager.py
+++ b/tests/unit/test_bigquery_connection_manager.py
@@ -1,18 +1,14 @@
-import time
 import json
-import pytest
 import unittest
 from contextlib import contextmanager
 from requests.exceptions import ConnectionError
 from unittest.mock import patch, MagicMock, Mock, ANY
 
 import dbt.adapters
-import dbt_common.dataclass_schema
 
 from dbt.adapters.bigquery import BigQueryCredentials
 from dbt.adapters.bigquery import BigQueryRelation
 from dbt.adapters.bigquery.connections import BigQueryConnectionManager
-import dbt_common.exceptions
 from dbt.logger import GLOBAL_LOGGER as logger  # noqa
 
 
@@ -123,26 +119,6 @@ def test_query_and_results(self, mock_bq):
             query="sql", job_config=mock_bq.QueryJobConfig(), timeout=15
         )
 
-    @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
-    def test_query_and_results_timeout(self, mock_bq):
-        self.mock_client.query = Mock(
-            return_value=Mock(result=lambda *args, **kwargs: time.sleep(4))
-        )
-        with pytest.raises(dbt_common.exceptions.DbtRuntimeError) as exc:
-            self.connections._query_and_results(
-                self.mock_client,
-                "sql",
-                {"job_param_1": "blah"},
-                job_creation_timeout=15,
-                job_execution_timeout=1,
-            )
-
-        mock_bq.QueryJobConfig.assert_called_once()
-        self.mock_client.query.assert_called_once_with(
-            query="sql", job_config=mock_bq.QueryJobConfig(), timeout=15
-        )
-        assert "Query exceeded configured timeout of 1s" in str(exc.value)
-
     def test_copy_bq_table_appends(self):
         self._copy_table(write_disposition=dbt.adapters.bigquery.impl.WRITE_APPEND)
         args, kwargs = self.mock_client.copy_table.call_args

From 79d53b16a2edbdb2bbacdbb2dce9259ea354f43f Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Tue, 27 Feb 2024 16:51:30 -0800
Subject: [PATCH 771/860] update dbt-common dependency to <2.0 (#1128)

* use dynamic schema in test_grant_access_to.py
* revert setup
* update dbt-common dependency to <2.0
---
 setup.py | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/setup.py b/setup.py
index 2e969e246..f8e16ded9 100644
--- a/setup.py
+++ b/setup.py
@@ -51,8 +51,8 @@ def _dbt_bigquery_version() -> str:
     packages=find_namespace_packages(include=["dbt", "dbt.*"]),
     include_package_data=True,
     install_requires=[
-        "dbt-common<1.0",
-        "dbt-adapters~=0.1.0a1",
+        "dbt-common<2.0",
+        "dbt-adapters<2.0",
         "google-cloud-bigquery~=3.0",
         "google-cloud-storage~=2.4",
         "google-cloud-dataproc~=5.0",

From bf721057c9299fa251e3ff9a9a0c3483a5471620 Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Fri, 1 Mar 2024 22:07:31 +0000
Subject: [PATCH 772/860] Bumping version to 1.8.0b1 and generate changelog

---
 .bumpversion.cfg                              |  2 +-
 .changes/1.8.0-b1.md                          | 57 ++++++++++++++++++
 .../Dependencies-20231002-164012.yaml         |  0
 .../Dependencies-20231009-005842.yaml         |  0
 .../Dependencies-20231011-002031.yaml         |  0
 .../Dependencies-20231016-002928.yaml         |  0
 .../Dependencies-20231018-010429.yaml         |  0
 .../Dependencies-20231027-132742.yaml         |  0
 .../Dependencies-20231027-201709.yaml         |  0
 .../Dependencies-20231109-005623.yaml         |  0
 .../Dependencies-20231113-002529.yaml         |  0
 .../Dependencies-20231113-002621.yaml         |  0
 .../Dependencies-20231116-001342.yaml         |  0
 .../Dependencies-20231127-004827.yaml         |  0
 .../Dependencies-20231128-005012.yaml         |  0
 .../Dependencies-20231128-005103.yaml         |  0
 .../Dependencies-20231129-001523.yaml         |  0
 .../Dependencies-20231204-003807.yaml         |  0
 .../Dependencies-20231213-003845.yaml         |  0
 .../Dependencies-20240124-120321.yaml         |  0
 .../Features-20231218-155409.yaml             |  0
 .../Features-20231219-201203.yaml             |  0
 .../Features-20240102-152030.yaml             |  0
 .../Features-20240205-174614.yaml             |  0
 .../Fixes-20231023-082312.yaml                |  0
 .../Fixes-20231025-131907.yaml                |  0
 .../Fixes-20231025-223003.yaml                |  0
 .../Fixes-20231030-222134.yaml                |  0
 .../Fixes-20231105-125740.yaml                |  0
 .../Fixes-20231105-143145.yaml                |  0
 .../Fixes-20231107-100905.yaml                |  0
 .../Fixes-20231107-174352.yaml                |  0
 .../Fixes-20231108-171128.yaml                |  0
 .../Fixes-20231111-150959.yaml                |  0
 .../Fixes-20231219-153446.yaml                |  0
 .../Fixes-20240216-133319.yaml                |  0
 .../Fixes-20240219-103324.yaml                |  0
 .../Under the Hood-20231109-095012.yaml       |  0
 .../Under the Hood-20231116-062142.yaml       |  0
 .../Under the Hood-20231117-121214.yaml       |  0
 .../Under the Hood-20240116-154305.yaml       |  0
 CHANGELOG.md                                  | 59 +++++++++++++++++++
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 44 files changed, 119 insertions(+), 3 deletions(-)
 create mode 100644 .changes/1.8.0-b1.md
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231002-164012.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231009-005842.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231011-002031.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231016-002928.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231018-010429.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231027-132742.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231027-201709.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231109-005623.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231113-002529.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231113-002621.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231116-001342.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231127-004827.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231128-005012.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231128-005103.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231129-001523.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231204-003807.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20231213-003845.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20240124-120321.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Features-20231218-155409.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Features-20231219-201203.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Features-20240102-152030.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Features-20240205-174614.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20231023-082312.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20231025-131907.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20231025-223003.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20231030-222134.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20231105-125740.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20231105-143145.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20231107-100905.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20231107-174352.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20231108-171128.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20231111-150959.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20231219-153446.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20240216-133319.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20240219-103324.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Under the Hood-20231109-095012.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Under the Hood-20231116-062142.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Under the Hood-20231117-121214.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Under the Hood-20240116-154305.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index e74c22bc8..c33a2c013 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.8.0a1
+current_version = 1.8.0b1
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.8.0-b1.md b/.changes/1.8.0-b1.md
new file mode 100644
index 000000000..84b8a17cb
--- /dev/null
+++ b/.changes/1.8.0-b1.md
@@ -0,0 +1,57 @@
+## dbt-bigquery 1.8.0-b1 - March 01, 2024
+
+### Features
+
+- Add support for checking table-last-modified by metadata ([#938](https://github.com/dbt-labs/dbt-bigquery/issues/938))
+- Support limiting get_catalog by object name ([#950](https://github.com/dbt-labs/dbt-bigquery/issues/950))
+- Update base adapter references as part of decoupling migration ([#1067](https://github.com/dbt-labs/dbt-bigquery/issues/1067))
+- Support all types for unit testing in dbt-bigquery, expand coverage of safe_cast macro ([#1090](https://github.com/dbt-labs/dbt-bigquery/issues/1090))
+
+### Fixes
+
+- Patch for json inline --show ([#972](https://github.com/dbt-labs/dbt-bigquery/issues/972))
+- Lower bound of `2.11.0` for `google-api-core` ([#979](https://github.com/dbt-labs/dbt-bigquery/issues/979))
+- Fix refresh syntax, config comparison with empty labels ([#983](https://github.com/dbt-labs/dbt-bigquery/issues/983))
+- Assign the correct relation type to materialized views in catalog queries ([#995](https://github.com/dbt-labs/dbt-bigquery/issues/995))
+- Fix inline comments (--) on the last line of an incremental model ([#896](https://github.com/dbt-labs/dbt-bigquery/issues/896))
+- In incremental models, add dummy merge condition on source partition column when partition is required ([#792](https://github.com/dbt-labs/dbt-bigquery/issues/792))
+- Support agate Integer type, test with empty seed ([#1003](https://github.com/dbt-labs/dbt-bigquery/issues/1003))
+- Fixed issue where materialized views were failing on re-run with minimal config parameters ([#1007](https://github.com/dbt-labs/dbt-bigquery/issues/1007))
+- Fix broken partition config granularity and batch_id being set to None ([#1006](https://github.com/dbt-labs/dbt-bigquery/issues/1006))
+- replace deterministic batch_id with uuid ([#1006](https://github.com/dbt-labs/dbt-bigquery/issues/1006))
+- remove json patch to leverage bigquery-python improvement ([#1055](https://github.com/dbt-labs/dbt-bigquery/issues/1055))
+- remove `token` field from connection keys ([#1105](https://github.com/dbt-labs/dbt-bigquery/issues/1105))
+- Remove custom query job async timeout logic as it has been fixed in bigquery-python ([#1081](https://github.com/dbt-labs/dbt-bigquery/issues/1081))
+
+### Under the Hood
+
+- Upgrade spark-bigquery Java deps for serverless to 2.13-0.34.0 ([#1006](https://github.com/dbt-labs/dbt-bigquery/issues/1006))
+- Primary and foreign key constraints are not enforced in BigQuery ([#1018](https://github.com/dbt-labs/dbt-bigquery/issues/1018))
+- Add tests for --empty flag ([#1029](https://github.com/dbt-labs/dbt-bigquery/issues/1029))
+- Migrate to dbt-common and dbt-adapters package ([#1071](https://github.com/dbt-labs/dbt-bigquery/issues/1071))
+
+### Dependencies
+
+- Update ddtrace requirement from ~=1.19 to ~=1.20 ([#948](https://github.com/dbt-labs/dbt-bigquery/pull/948))
+- Update pre-commit-hooks requirement from ~=4.4 to ~=4.5 ([#960](https://github.com/dbt-labs/dbt-bigquery/pull/960))
+- Bump mypy from 1.5.1 to 1.6.0 ([#963](https://github.com/dbt-labs/dbt-bigquery/pull/963))
+- Update pre-commit requirement from ~=3.4 to ~=3.5 ([#969](https://github.com/dbt-labs/dbt-bigquery/pull/969))
+- Update black requirement from ~=23.9 to ~=23.10 ([#973](https://github.com/dbt-labs/dbt-bigquery/pull/973))
+- Bump mypy from 1.6.0 to 1.6.1 ([#985](https://github.com/dbt-labs/dbt-bigquery/pull/985))
+- Update ddtrace requirement from ~=1.20 to ~=2.1 ([#989](https://github.com/dbt-labs/dbt-bigquery/pull/989))
+- Update black requirement from ~=23.10 to ~=23.11 ([#1013](https://github.com/dbt-labs/dbt-bigquery/pull/1013))
+- Update pytest-xdist requirement from ~=3.3 to ~=3.4 ([#1022](https://github.com/dbt-labs/dbt-bigquery/pull/1022))
+- Bump mypy from 1.6.1 to 1.7.0 ([#1023](https://github.com/dbt-labs/dbt-bigquery/pull/1023))
+- Update ddtrace requirement from ~=2.1 to ~=2.2 ([#1028](https://github.com/dbt-labs/dbt-bigquery/pull/1028))
+- Update wheel requirement from ~=0.41 to ~=0.42 ([#1033](https://github.com/dbt-labs/dbt-bigquery/pull/1033))
+- Bump mypy from 1.7.0 to 1.7.1 ([#1034](https://github.com/dbt-labs/dbt-bigquery/pull/1034))
+- Update ddtrace requirement from ~=2.2 to ~=2.3 ([#1035](https://github.com/dbt-labs/dbt-bigquery/pull/1035))
+- Update pytest-xdist requirement from ~=3.4 to ~=3.5 ([#1037](https://github.com/dbt-labs/dbt-bigquery/pull/1037))
+- Update freezegun requirement from ~=1.2 to ~=1.3 ([#1040](https://github.com/dbt-labs/dbt-bigquery/pull/1040))
+- Update black requirement from ~=23.11 to ~=23.12 ([#1056](https://github.com/dbt-labs/dbt-bigquery/pull/1056))
+- get dbt-tests-adapters from dbt-adapters repo ([#1077](https://github.com/dbt-labs/dbt-bigquery/pull/1077))
+
+### Contributors
+- [@gmyrianthous](https://github.com/gmyrianthous) ([#979](https://github.com/dbt-labs/dbt-bigquery/issues/979))
+- [@matt-winkler](https://github.com/matt-winkler) ([#972](https://github.com/dbt-labs/dbt-bigquery/issues/972))
+- [@tnk-ysk](https://github.com/tnk-ysk) ([#896](https://github.com/dbt-labs/dbt-bigquery/issues/896), [#792](https://github.com/dbt-labs/dbt-bigquery/issues/792))
diff --git a/.changes/unreleased/Dependencies-20231002-164012.yaml b/.changes/1.8.0/Dependencies-20231002-164012.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231002-164012.yaml
rename to .changes/1.8.0/Dependencies-20231002-164012.yaml
diff --git a/.changes/unreleased/Dependencies-20231009-005842.yaml b/.changes/1.8.0/Dependencies-20231009-005842.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231009-005842.yaml
rename to .changes/1.8.0/Dependencies-20231009-005842.yaml
diff --git a/.changes/unreleased/Dependencies-20231011-002031.yaml b/.changes/1.8.0/Dependencies-20231011-002031.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231011-002031.yaml
rename to .changes/1.8.0/Dependencies-20231011-002031.yaml
diff --git a/.changes/unreleased/Dependencies-20231016-002928.yaml b/.changes/1.8.0/Dependencies-20231016-002928.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231016-002928.yaml
rename to .changes/1.8.0/Dependencies-20231016-002928.yaml
diff --git a/.changes/unreleased/Dependencies-20231018-010429.yaml b/.changes/1.8.0/Dependencies-20231018-010429.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231018-010429.yaml
rename to .changes/1.8.0/Dependencies-20231018-010429.yaml
diff --git a/.changes/unreleased/Dependencies-20231027-132742.yaml b/.changes/1.8.0/Dependencies-20231027-132742.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231027-132742.yaml
rename to .changes/1.8.0/Dependencies-20231027-132742.yaml
diff --git a/.changes/unreleased/Dependencies-20231027-201709.yaml b/.changes/1.8.0/Dependencies-20231027-201709.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231027-201709.yaml
rename to .changes/1.8.0/Dependencies-20231027-201709.yaml
diff --git a/.changes/unreleased/Dependencies-20231109-005623.yaml b/.changes/1.8.0/Dependencies-20231109-005623.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231109-005623.yaml
rename to .changes/1.8.0/Dependencies-20231109-005623.yaml
diff --git a/.changes/unreleased/Dependencies-20231113-002529.yaml b/.changes/1.8.0/Dependencies-20231113-002529.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231113-002529.yaml
rename to .changes/1.8.0/Dependencies-20231113-002529.yaml
diff --git a/.changes/unreleased/Dependencies-20231113-002621.yaml b/.changes/1.8.0/Dependencies-20231113-002621.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231113-002621.yaml
rename to .changes/1.8.0/Dependencies-20231113-002621.yaml
diff --git a/.changes/unreleased/Dependencies-20231116-001342.yaml b/.changes/1.8.0/Dependencies-20231116-001342.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231116-001342.yaml
rename to .changes/1.8.0/Dependencies-20231116-001342.yaml
diff --git a/.changes/unreleased/Dependencies-20231127-004827.yaml b/.changes/1.8.0/Dependencies-20231127-004827.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231127-004827.yaml
rename to .changes/1.8.0/Dependencies-20231127-004827.yaml
diff --git a/.changes/unreleased/Dependencies-20231128-005012.yaml b/.changes/1.8.0/Dependencies-20231128-005012.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231128-005012.yaml
rename to .changes/1.8.0/Dependencies-20231128-005012.yaml
diff --git a/.changes/unreleased/Dependencies-20231128-005103.yaml b/.changes/1.8.0/Dependencies-20231128-005103.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231128-005103.yaml
rename to .changes/1.8.0/Dependencies-20231128-005103.yaml
diff --git a/.changes/unreleased/Dependencies-20231129-001523.yaml b/.changes/1.8.0/Dependencies-20231129-001523.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231129-001523.yaml
rename to .changes/1.8.0/Dependencies-20231129-001523.yaml
diff --git a/.changes/unreleased/Dependencies-20231204-003807.yaml b/.changes/1.8.0/Dependencies-20231204-003807.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231204-003807.yaml
rename to .changes/1.8.0/Dependencies-20231204-003807.yaml
diff --git a/.changes/unreleased/Dependencies-20231213-003845.yaml b/.changes/1.8.0/Dependencies-20231213-003845.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231213-003845.yaml
rename to .changes/1.8.0/Dependencies-20231213-003845.yaml
diff --git a/.changes/unreleased/Dependencies-20240124-120321.yaml b/.changes/1.8.0/Dependencies-20240124-120321.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20240124-120321.yaml
rename to .changes/1.8.0/Dependencies-20240124-120321.yaml
diff --git a/.changes/unreleased/Features-20231218-155409.yaml b/.changes/1.8.0/Features-20231218-155409.yaml
similarity index 100%
rename from .changes/unreleased/Features-20231218-155409.yaml
rename to .changes/1.8.0/Features-20231218-155409.yaml
diff --git a/.changes/unreleased/Features-20231219-201203.yaml b/.changes/1.8.0/Features-20231219-201203.yaml
similarity index 100%
rename from .changes/unreleased/Features-20231219-201203.yaml
rename to .changes/1.8.0/Features-20231219-201203.yaml
diff --git a/.changes/unreleased/Features-20240102-152030.yaml b/.changes/1.8.0/Features-20240102-152030.yaml
similarity index 100%
rename from .changes/unreleased/Features-20240102-152030.yaml
rename to .changes/1.8.0/Features-20240102-152030.yaml
diff --git a/.changes/unreleased/Features-20240205-174614.yaml b/.changes/1.8.0/Features-20240205-174614.yaml
similarity index 100%
rename from .changes/unreleased/Features-20240205-174614.yaml
rename to .changes/1.8.0/Features-20240205-174614.yaml
diff --git a/.changes/unreleased/Fixes-20231023-082312.yaml b/.changes/1.8.0/Fixes-20231023-082312.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20231023-082312.yaml
rename to .changes/1.8.0/Fixes-20231023-082312.yaml
diff --git a/.changes/unreleased/Fixes-20231025-131907.yaml b/.changes/1.8.0/Fixes-20231025-131907.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20231025-131907.yaml
rename to .changes/1.8.0/Fixes-20231025-131907.yaml
diff --git a/.changes/unreleased/Fixes-20231025-223003.yaml b/.changes/1.8.0/Fixes-20231025-223003.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20231025-223003.yaml
rename to .changes/1.8.0/Fixes-20231025-223003.yaml
diff --git a/.changes/unreleased/Fixes-20231030-222134.yaml b/.changes/1.8.0/Fixes-20231030-222134.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20231030-222134.yaml
rename to .changes/1.8.0/Fixes-20231030-222134.yaml
diff --git a/.changes/unreleased/Fixes-20231105-125740.yaml b/.changes/1.8.0/Fixes-20231105-125740.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20231105-125740.yaml
rename to .changes/1.8.0/Fixes-20231105-125740.yaml
diff --git a/.changes/unreleased/Fixes-20231105-143145.yaml b/.changes/1.8.0/Fixes-20231105-143145.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20231105-143145.yaml
rename to .changes/1.8.0/Fixes-20231105-143145.yaml
diff --git a/.changes/unreleased/Fixes-20231107-100905.yaml b/.changes/1.8.0/Fixes-20231107-100905.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20231107-100905.yaml
rename to .changes/1.8.0/Fixes-20231107-100905.yaml
diff --git a/.changes/unreleased/Fixes-20231107-174352.yaml b/.changes/1.8.0/Fixes-20231107-174352.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20231107-174352.yaml
rename to .changes/1.8.0/Fixes-20231107-174352.yaml
diff --git a/.changes/unreleased/Fixes-20231108-171128.yaml b/.changes/1.8.0/Fixes-20231108-171128.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20231108-171128.yaml
rename to .changes/1.8.0/Fixes-20231108-171128.yaml
diff --git a/.changes/unreleased/Fixes-20231111-150959.yaml b/.changes/1.8.0/Fixes-20231111-150959.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20231111-150959.yaml
rename to .changes/1.8.0/Fixes-20231111-150959.yaml
diff --git a/.changes/unreleased/Fixes-20231219-153446.yaml b/.changes/1.8.0/Fixes-20231219-153446.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20231219-153446.yaml
rename to .changes/1.8.0/Fixes-20231219-153446.yaml
diff --git a/.changes/unreleased/Fixes-20240216-133319.yaml b/.changes/1.8.0/Fixes-20240216-133319.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20240216-133319.yaml
rename to .changes/1.8.0/Fixes-20240216-133319.yaml
diff --git a/.changes/unreleased/Fixes-20240219-103324.yaml b/.changes/1.8.0/Fixes-20240219-103324.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20240219-103324.yaml
rename to .changes/1.8.0/Fixes-20240219-103324.yaml
diff --git a/.changes/unreleased/Under the Hood-20231109-095012.yaml b/.changes/1.8.0/Under the Hood-20231109-095012.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20231109-095012.yaml
rename to .changes/1.8.0/Under the Hood-20231109-095012.yaml
diff --git a/.changes/unreleased/Under the Hood-20231116-062142.yaml b/.changes/1.8.0/Under the Hood-20231116-062142.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20231116-062142.yaml
rename to .changes/1.8.0/Under the Hood-20231116-062142.yaml
diff --git a/.changes/unreleased/Under the Hood-20231117-121214.yaml b/.changes/1.8.0/Under the Hood-20231117-121214.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20231117-121214.yaml
rename to .changes/1.8.0/Under the Hood-20231117-121214.yaml
diff --git a/.changes/unreleased/Under the Hood-20240116-154305.yaml b/.changes/1.8.0/Under the Hood-20240116-154305.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20240116-154305.yaml
rename to .changes/1.8.0/Under the Hood-20240116-154305.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index ade60b8f6..6012f18df 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,6 +5,65 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.8.0-b1 - March 01, 2024
+
+### Features
+
+- Add support for checking table-last-modified by metadata ([#938](https://github.com/dbt-labs/dbt-bigquery/issues/938))
+- Support limiting get_catalog by object name ([#950](https://github.com/dbt-labs/dbt-bigquery/issues/950))
+- Update base adapter references as part of decoupling migration ([#1067](https://github.com/dbt-labs/dbt-bigquery/issues/1067))
+- Support all types for unit testing in dbt-bigquery, expand coverage of safe_cast macro ([#1090](https://github.com/dbt-labs/dbt-bigquery/issues/1090))
+
+### Fixes
+
+- Patch for json inline --show ([#972](https://github.com/dbt-labs/dbt-bigquery/issues/972))
+- Lower bound of `2.11.0` for `google-api-core` ([#979](https://github.com/dbt-labs/dbt-bigquery/issues/979))
+- Fix refresh syntax, config comparison with empty labels ([#983](https://github.com/dbt-labs/dbt-bigquery/issues/983))
+- Assign the correct relation type to materialized views in catalog queries ([#995](https://github.com/dbt-labs/dbt-bigquery/issues/995))
+- Fix inline comments (--) on the last line of an incremental model ([#896](https://github.com/dbt-labs/dbt-bigquery/issues/896))
+- In incremental models, add dummy merge condition on source partition column when partition is required ([#792](https://github.com/dbt-labs/dbt-bigquery/issues/792))
+- Support agate Integer type, test with empty seed ([#1003](https://github.com/dbt-labs/dbt-bigquery/issues/1003))
+- Fixed issue where materialized views were failing on re-run with minimal config parameters ([#1007](https://github.com/dbt-labs/dbt-bigquery/issues/1007))
+- Fix broken partition config granularity and batch_id being set to None ([#1006](https://github.com/dbt-labs/dbt-bigquery/issues/1006))
+- replace deterministic batch_id with uuid ([#1006](https://github.com/dbt-labs/dbt-bigquery/issues/1006))
+- remove json patch to leverage bigquery-python improvement ([#1055](https://github.com/dbt-labs/dbt-bigquery/issues/1055))
+- remove `token` field from connection keys ([#1105](https://github.com/dbt-labs/dbt-bigquery/issues/1105))
+- Remove custom query job async timeout logic as it has been fixed in bigquery-python ([#1081](https://github.com/dbt-labs/dbt-bigquery/issues/1081))
+
+### Under the Hood
+
+- Upgrade spark-bigquery Java deps for serverless to 2.13-0.34.0 ([#1006](https://github.com/dbt-labs/dbt-bigquery/issues/1006))
+- Primary and foreign key constraints are not enforced in BigQuery ([#1018](https://github.com/dbt-labs/dbt-bigquery/issues/1018))
+- Add tests for --empty flag ([#1029](https://github.com/dbt-labs/dbt-bigquery/issues/1029))
+- Migrate to dbt-common and dbt-adapters package ([#1071](https://github.com/dbt-labs/dbt-bigquery/issues/1071))
+
+### Dependencies
+
+- Update ddtrace requirement from ~=1.19 to ~=1.20 ([#948](https://github.com/dbt-labs/dbt-bigquery/pull/948))
+- Update pre-commit-hooks requirement from ~=4.4 to ~=4.5 ([#960](https://github.com/dbt-labs/dbt-bigquery/pull/960))
+- Bump mypy from 1.5.1 to 1.6.0 ([#963](https://github.com/dbt-labs/dbt-bigquery/pull/963))
+- Update pre-commit requirement from ~=3.4 to ~=3.5 ([#969](https://github.com/dbt-labs/dbt-bigquery/pull/969))
+- Update black requirement from ~=23.9 to ~=23.10 ([#973](https://github.com/dbt-labs/dbt-bigquery/pull/973))
+- Bump mypy from 1.6.0 to 1.6.1 ([#985](https://github.com/dbt-labs/dbt-bigquery/pull/985))
+- Update ddtrace requirement from ~=1.20 to ~=2.1 ([#989](https://github.com/dbt-labs/dbt-bigquery/pull/989))
+- Update black requirement from ~=23.10 to ~=23.11 ([#1013](https://github.com/dbt-labs/dbt-bigquery/pull/1013))
+- Update pytest-xdist requirement from ~=3.3 to ~=3.4 ([#1022](https://github.com/dbt-labs/dbt-bigquery/pull/1022))
+- Bump mypy from 1.6.1 to 1.7.0 ([#1023](https://github.com/dbt-labs/dbt-bigquery/pull/1023))
+- Update ddtrace requirement from ~=2.1 to ~=2.2 ([#1028](https://github.com/dbt-labs/dbt-bigquery/pull/1028))
+- Update wheel requirement from ~=0.41 to ~=0.42 ([#1033](https://github.com/dbt-labs/dbt-bigquery/pull/1033))
+- Bump mypy from 1.7.0 to 1.7.1 ([#1034](https://github.com/dbt-labs/dbt-bigquery/pull/1034))
+- Update ddtrace requirement from ~=2.2 to ~=2.3 ([#1035](https://github.com/dbt-labs/dbt-bigquery/pull/1035))
+- Update pytest-xdist requirement from ~=3.4 to ~=3.5 ([#1037](https://github.com/dbt-labs/dbt-bigquery/pull/1037))
+- Update freezegun requirement from ~=1.2 to ~=1.3 ([#1040](https://github.com/dbt-labs/dbt-bigquery/pull/1040))
+- Update black requirement from ~=23.11 to ~=23.12 ([#1056](https://github.com/dbt-labs/dbt-bigquery/pull/1056))
+- get dbt-tests-adapters from dbt-adapters repo ([#1077](https://github.com/dbt-labs/dbt-bigquery/pull/1077))
+
+### Contributors
+- [@gmyrianthous](https://github.com/gmyrianthous) ([#979](https://github.com/dbt-labs/dbt-bigquery/issues/979))
+- [@matt-winkler](https://github.com/matt-winkler) ([#972](https://github.com/dbt-labs/dbt-bigquery/issues/972))
+- [@tnk-ysk](https://github.com/tnk-ysk) ([#896](https://github.com/dbt-labs/dbt-bigquery/issues/896), [#792](https://github.com/dbt-labs/dbt-bigquery/issues/792))
+
+
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 - [1.6](https://github.com/dbt-labs/dbt-bigquery/blob/1.6.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index f15b401d1..6496f3e22 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.8.0a1"
+version = "1.8.0b1"
diff --git a/setup.py b/setup.py
index f8e16ded9..97cfdae5e 100644
--- a/setup.py
+++ b/setup.py
@@ -36,7 +36,7 @@ def _dbt_bigquery_version() -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.8.0a1"
+package_version = "1.8.0b1"
 description = """The BigQuery adapter plugin for dbt"""
 
 setup(

From 1fb42791644a8c507f101fc77c9a7332fe89b467 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Fri, 1 Mar 2024 15:13:51 -0800
Subject: [PATCH 773/860] allow for pre-release common/adapters (#1130)

* update install_requires to allow for pre-release common/adapters
---
 setup.py | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/setup.py b/setup.py
index 97cfdae5e..3e714c74d 100644
--- a/setup.py
+++ b/setup.py
@@ -51,8 +51,8 @@ def _dbt_bigquery_version() -> str:
     packages=find_namespace_packages(include=["dbt", "dbt.*"]),
     include_package_data=True,
     install_requires=[
-        "dbt-common<2.0",
-        "dbt-adapters<2.0",
+        "dbt-common>=0.1.0a1,<2.0",
+        "dbt-adapters>=0.1.0a1,<2.0",
         "google-cloud-bigquery~=3.0",
         "google-cloud-storage~=2.4",
         "google-cloud-dataproc~=5.0",

From 6c0afe4cfb69761dada5d16150fe632b8f72bf39 Mon Sep 17 00:00:00 2001
From: colin-rogers-dbt <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 7 Mar 2024 09:10:32 -0800
Subject: [PATCH 774/860] Fix wheel/sdist check (#1132)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* use import instead of `dbt --version`
---
 .github/workflows/main.yml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index b13c53e9f..fbaf30925 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -199,10 +199,10 @@ jobs:
           find ./dist/*.whl -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/
       - name: Check wheel distributions
         run: |
-          dbt --version
+          python -c "import dbt.adapters.bigquery"
       - name: Install source distributions
         run: |
           find ./dist/*.gz -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/
       - name: Check source distributions
         run: |
-          dbt --version
+          python -c "import dbt.adapters.bigquery"

From faf1a7f5bf46103c42cd598497147d4b4e196b0b Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Tue, 19 Mar 2024 15:39:21 -0500
Subject: [PATCH 775/860] Test ddtrace dependency range (#1141)

* set bounds for ddtrace due to tests failing

* hard pin ddtrace, create changelog
---
 .changes/unreleased/Dependencies-20240319-102258.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20240319-102258.yaml

diff --git a/.changes/unreleased/Dependencies-20240319-102258.yaml b/.changes/unreleased/Dependencies-20240319-102258.yaml
new file mode 100644
index 000000000..5df197537
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240319-102258.yaml
@@ -0,0 +1,6 @@
+kind: Dependencies
+body: hard pin ddtrace to 2.3.0
+time: 2024-03-19T10:22:58.3838-05:00
+custom:
+  Author: McKnight-42
+  PR: "1141"
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 0af563a7d..331d4e31f 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -9,7 +9,7 @@ git+https://github.com/dbt-labs/dbt-adapters.git#subdirectory=dbt-tests-adapter
 black~=23.12
 bumpversion~=0.6.0
 click~=8.1
-ddtrace~=2.3
+ddtrace==2.3.0
 flake8~=6.1
 flaky~=3.7
 freezegun~=1.3

From 5c1b70fbc36d88df219ae818207093f9b7c1e2e7 Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Wed, 20 Mar 2024 14:26:54 -0700
Subject: [PATCH 776/860] Add Bigquery internal build process workflow  (#1140)

* Add workflow file.

* Add changelog

---------

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .../unreleased/Features-20240318-032256.yaml  |  6 ++
 .github/workflows/release-internal.yml        | 64 +++++++++++++++++++
 2 files changed, 70 insertions(+)
 create mode 100644 .changes/unreleased/Features-20240318-032256.yaml
 create mode 100644 .github/workflows/release-internal.yml

diff --git a/.changes/unreleased/Features-20240318-032256.yaml b/.changes/unreleased/Features-20240318-032256.yaml
new file mode 100644
index 000000000..e4948f433
--- /dev/null
+++ b/.changes/unreleased/Features-20240318-032256.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Add new workflow for internal patch releases
+time: 2024-03-18T03:22:56.037781-07:00
+custom:
+  Author: versusfacit
+  Issue: "38"
diff --git a/.github/workflows/release-internal.yml b/.github/workflows/release-internal.yml
new file mode 100644
index 000000000..5bc6cd667
--- /dev/null
+++ b/.github/workflows/release-internal.yml
@@ -0,0 +1,64 @@
+name: Release internal patch
+
+on:
+  workflow_dispatch:
+    inputs:
+      version_number:
+        description: "The release version number (i.e. 1.0.0b1)"
+        type: string
+        required: true
+      sha:
+        description: "The sha to use (leave empty to use latest on main)"
+        type: string
+        required: false
+      package_test_command:
+        description: "Package test command"
+        type: string
+        default: "python -c \"import dbt.adapters.bigquery\""
+        required: true
+      dbms_name:
+        description: "The name of the warehouse the adapter connects to."
+        type: string
+        default: "bigquery"
+        required: true
+  workflow_call:
+    inputs:
+      version_number:
+        description: "The release version number (i.e. 1.0.0b1)"
+        type: string
+        required: true
+      sha:
+        description: "The sha to use (leave empty to use latest on main)"
+        type: string
+        required: false
+      package_test_command:
+        description: "Package test command"
+        type: string
+        default: "python -c \"import dbt.adapters.bigquery\""
+        required: true
+      dbms_name:
+        description: "The name of the warehouse the adapter connects to."
+        type: string
+        default: "bigquery"
+        required: true
+
+defaults:
+  run:
+    shell: bash
+
+env:
+  PYTHON_TARGET_VERSION: 3.11
+
+jobs:
+  invoke-reusable-workflow:
+    name: Build and Release Internally
+
+    uses: VersusFacit/dbt-release/.github/workflows/internal-archive-release.yml@main
+
+    with:
+      version_number: ${{ inputs.version_number }}
+      package_test_command: ${{ inputs.package_test_command }}
+      dbms_name: ${{ inputs.dbms_name }}
+      sha: ${{ inputs.sha }}
+
+    secrets: inherit

From 40024838e58e41385d4b4659fbd677faa5955725 Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Thu, 21 Mar 2024 08:31:24 -0700
Subject: [PATCH 777/860]  Fix the renamed relations code (#1125)

* Add test and move semantics.
* Add field import

---------

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .../Under the Hood-20240227-004659.yaml       |  6 ++++++
 dbt/adapters/bigquery/relation.py             | 21 +++++++++++++++----
 tests/unit/test_renamed_relations.py          | 16 ++++++++++++++
 3 files changed, 39 insertions(+), 4 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20240227-004659.yaml
 create mode 100644 tests/unit/test_renamed_relations.py

diff --git a/.changes/unreleased/Under the Hood-20240227-004659.yaml b/.changes/unreleased/Under the Hood-20240227-004659.yaml
new file mode 100644
index 000000000..6ef259019
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20240227-004659.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Add unit test for transaction semantics.
+time: 2024-02-27T00:46:59.188231-08:00
+custom:
+  Author: versusfacit
+  Issue: "1123"
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index 8abda577b..3a3a7fbe6 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -1,4 +1,4 @@
-from dataclasses import dataclass
+from dataclasses import dataclass, field
 from typing import FrozenSet, Optional, TypeVar
 
 from itertools import chain, islice
@@ -23,9 +23,22 @@
 class BigQueryRelation(BaseRelation):
     quote_character: str = "`"
     location: Optional[str] = None
-    renameable_relations: FrozenSet[RelationType] = frozenset({RelationType.Table})
-    replaceable_relations: FrozenSet[RelationType] = frozenset(
-        {RelationType.Table, RelationType.View}
+
+    renameable_relations: FrozenSet[RelationType] = field(
+        default_factory=lambda: frozenset(
+            {
+                RelationType.Table,
+            }
+        )
+    )
+
+    replaceable_relations: FrozenSet[RelationType] = field(
+        default_factory=lambda: frozenset(
+            {
+                RelationType.View,
+                RelationType.Table,
+            }
+        )
     )
 
     def matches(
diff --git a/tests/unit/test_renamed_relations.py b/tests/unit/test_renamed_relations.py
new file mode 100644
index 000000000..8e787e6a3
--- /dev/null
+++ b/tests/unit/test_renamed_relations.py
@@ -0,0 +1,16 @@
+from dbt.adapters.bigquery.relation import BigQueryRelation
+from dbt.adapters.contracts.relation import RelationType
+
+
+def test_renameable_relation():
+    relation = BigQueryRelation.create(
+        database="my_db",
+        schema="my_schema",
+        identifier="my_table",
+        type=RelationType.Table,
+    )
+    assert relation.renameable_relations == frozenset(
+        {
+            RelationType.Table,
+        }
+    )

From 955afbd74b19d37f6c7919a4a92dd2cc50eae905 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Mon, 25 Mar 2024 12:32:19 -0500
Subject: [PATCH 778/860] remove `keyfile` from `_connection_keys` (#1147)

* remove kefile from _connection_keys

* add changelog
---
 .changes/unreleased/Fixes-20240322-113720.yaml | 6 ++++++
 dbt/adapters/bigquery/connections.py           | 1 -
 2 files changed, 6 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20240322-113720.yaml

diff --git a/.changes/unreleased/Fixes-20240322-113720.yaml b/.changes/unreleased/Fixes-20240322-113720.yaml
new file mode 100644
index 000000000..9279c6ecd
--- /dev/null
+++ b/.changes/unreleased/Fixes-20240322-113720.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: remove `keyfile` from `_connection_keys`
+time: 2024-03-22T11:37:20.989189-05:00
+custom:
+  Author: McKnight-42
+  Issue: "1146"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 1e4708f0b..09558a14a 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -195,7 +195,6 @@ def _connection_keys(self):
             "job_retries",
             "job_creation_timeout_seconds",
             "job_execution_timeout_seconds",
-            "keyfile",
             "timeout_seconds",
             "client_id",
             "token_uri",

From e98ee4c3d5040b7e107f6de41a3d3949bf6972fd Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Tue, 26 Mar 2024 16:21:26 -0700
Subject: [PATCH 779/860] Finish internal build workflow (#1143)

* Add workflow file.

* Add changelog

* Tweaks to input params.

* Add changelog.

* Change python version to match Cloud.

* Finalize workflow.

* Change branch reference of workflow to main.

---------

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .github/workflows/release-internal.yml | 61 ++++++++++----------------
 1 file changed, 23 insertions(+), 38 deletions(-)

diff --git a/.github/workflows/release-internal.yml b/.github/workflows/release-internal.yml
index 5bc6cd667..f29de2fb2 100644
--- a/.github/workflows/release-internal.yml
+++ b/.github/workflows/release-internal.yml
@@ -1,4 +1,16 @@
-name: Release internal patch
+# What?
+#
+# Tag and release an arbitrary ref. Uploads to an internal archive for further processing.
+#
+# How?
+#
+# After checking out and testing the provided ref, the image is built and uploaded.
+#
+# When?
+#
+# Manual trigger.
+
+name: "Release internal patch"
 
 on:
   workflow_dispatch:
@@ -7,58 +19,31 @@ on:
         description: "The release version number (i.e. 1.0.0b1)"
         type: string
         required: true
-      sha:
-        description: "The sha to use (leave empty to use latest on main)"
-        type: string
-        required: false
-      package_test_command:
-        description: "Package test command"
-        type: string
-        default: "python -c \"import dbt.adapters.bigquery\""
-        required: true
-      dbms_name:
-        description: "The name of the warehouse the adapter connects to."
-        type: string
-        default: "bigquery"
-        required: true
-  workflow_call:
-    inputs:
-      version_number:
-        description: "The release version number (i.e. 1.0.0b1)"
+      ref:
+        description: "The ref (sha or branch name) to use"
         type: string
+        default: "main"
         required: true
-      sha:
-        description: "The sha to use (leave empty to use latest on main)"
-        type: string
-        required: false
       package_test_command:
         description: "Package test command"
         type: string
         default: "python -c \"import dbt.adapters.bigquery\""
         required: true
-      dbms_name:
-        description: "The name of the warehouse the adapter connects to."
-        type: string
-        default: "bigquery"
-        required: true
 
 defaults:
   run:
     shell: bash
 
-env:
-  PYTHON_TARGET_VERSION: 3.11
-
 jobs:
   invoke-reusable-workflow:
-    name: Build and Release Internally
+    name: "Build and Release Internally"
 
-    uses: VersusFacit/dbt-release/.github/workflows/internal-archive-release.yml@main
+    uses: "dbt-labs/dbt-release/.github/workflows/internal-archive-release.yml@main"
 
     with:
-      version_number: ${{ inputs.version_number }}
-      package_test_command: ${{ inputs.package_test_command }}
-      dbms_name: ${{ inputs.dbms_name }}
-      sha: ${{ inputs.sha }}
+      version_number: "${{ inputs.version_number }}"
+      package_test_command: "${{ inputs.package_test_command }}"
+      dbms_name: "bigquery"
+      ref: "${{ inputs.ref }}"
 
-    secrets: inherit
+    secrets: "inherit"

From 978a1aea7d18af79d6d78d0ab5e8d058c4b7c93c Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 27 Mar 2024 22:07:11 -0400
Subject: [PATCH 780/860] Add `pandas` extra for `google-cloud-bigquery` to
 pick up missing `pyarrow` (#1153)

* Add `pandas` extra for `google-cloud-bigquery` to pick up missing `pyarrow`
---
 .changes/unreleased/Fixes-20240327-210249.yaml | 7 +++++++
 setup.py                                       | 3 ++-
 2 files changed, 9 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20240327-210249.yaml

diff --git a/.changes/unreleased/Fixes-20240327-210249.yaml b/.changes/unreleased/Fixes-20240327-210249.yaml
new file mode 100644
index 000000000..04a5c5e38
--- /dev/null
+++ b/.changes/unreleased/Fixes-20240327-210249.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: Add `pandas` extra for `google-cloud-bigquery` to pick up missing `pyarrow`
+  dependency
+time: 2024-03-27T21:02:49.619691-04:00
+custom:
+  Author: mikealfare
+  Issue: "1152"
diff --git a/setup.py b/setup.py
index 3e714c74d..24d873689 100644
--- a/setup.py
+++ b/setup.py
@@ -53,7 +53,8 @@ def _dbt_bigquery_version() -> str:
     install_requires=[
         "dbt-common>=0.1.0a1,<2.0",
         "dbt-adapters>=0.1.0a1,<2.0",
-        "google-cloud-bigquery~=3.0",
+        # 3.20 introduced pyarrow>=3.0 under the `pandas` extra
+        "google-cloud-bigquery[pandas]>=3.0,<4.0",
         "google-cloud-storage~=2.4",
         "google-cloud-dataproc~=5.0",
         # ----

From 05b0968961932c9c166984250dd5b0abad42c275 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Thu, 28 Mar 2024 13:44:26 -0400
Subject: [PATCH 781/860] Pin `black>=24.3` (#1151)

---
 .changes/unreleased/Security-20240327-193211.yaml | 6 ++++++
 dev-requirements.txt                              | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Security-20240327-193211.yaml

diff --git a/.changes/unreleased/Security-20240327-193211.yaml b/.changes/unreleased/Security-20240327-193211.yaml
new file mode 100644
index 000000000..1b755959b
--- /dev/null
+++ b/.changes/unreleased/Security-20240327-193211.yaml
@@ -0,0 +1,6 @@
+kind: Security
+body: Pin `black>=24.3` in `dev-requirements.txt`
+time: 2024-03-27T19:32:11.383652-04:00
+custom:
+  Author: mikealfare
+  PR: "1151"
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 331d4e31f..cf7b1b87c 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -6,7 +6,7 @@ git+https://github.com/dbt-labs/dbt-adapters.git
 git+https://github.com/dbt-labs/dbt-adapters.git#subdirectory=dbt-tests-adapter
 # if version 1.x or greater -> pin to major version
 # if version 0.x -> pin to minor
-black~=23.12
+black>=24.3
 bumpversion~=0.6.0
 click~=8.1
 ddtrace==2.3.0

From bb0fb2fd992ddd2c05a8a9d66cb4987d28d31996 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 3 Apr 2024 14:55:10 -0400
Subject: [PATCH 782/860] Add `dbt-core~=1.8.0a1` as convenience dep (#1168)

* add `dbt-core~=1.8.0a1` as convenience dep

* changelog
---
 .changes/unreleased/Dependencies-20240403-134956.yaml | 6 ++++++
 setup.py                                              | 2 ++
 2 files changed, 8 insertions(+)
 create mode 100644 .changes/unreleased/Dependencies-20240403-134956.yaml

diff --git a/.changes/unreleased/Dependencies-20240403-134956.yaml b/.changes/unreleased/Dependencies-20240403-134956.yaml
new file mode 100644
index 000000000..8e72020b5
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240403-134956.yaml
@@ -0,0 +1,6 @@
+kind: Dependencies
+body: Add `dbt-core` as a dependency to preserve backwards compatibility for installation
+time: 2024-04-03T13:49:56.063802-04:00
+custom:
+  Author: mikealfare
+  PR: "1168"
diff --git a/setup.py b/setup.py
index 24d873689..1308e2dfd 100644
--- a/setup.py
+++ b/setup.py
@@ -60,6 +60,8 @@ def _dbt_bigquery_version() -> str:
         # ----
         # Expect compatibility with all new versions of these packages, so lower bounds only.
         "google-api-core>=2.11.0",
+        # add dbt-core to ensure backwards compatibility of installation, this is not a functional dependency
+        "dbt-core>=1.8.0a1",
     ],
     zip_safe=False,
     classifiers=[

From c3f8204fb005ce245b69ea4304ecce68717fa57d Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Wed, 3 Apr 2024 19:05:54 +0000
Subject: [PATCH 783/860] Bumping version to 1.8.0b2 and generate changelog

---
 .bumpversion.cfg                              |  2 +-
 .changes/1.8.0-b2.md                          | 23 ++++++++++++++++
 .../Dependencies-20240319-102258.yaml         |  0
 .../Dependencies-20240403-134956.yaml         |  0
 .../Features-20240318-032256.yaml             |  0
 .../Fixes-20240322-113720.yaml                |  0
 .../Fixes-20240327-210249.yaml                |  0
 .../Security-20240327-193211.yaml             |  0
 .../Under the Hood-20240227-004659.yaml       |  0
 CHANGELOG.md                                  | 27 ++++++++++++++++++-
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 12 files changed, 52 insertions(+), 4 deletions(-)
 create mode 100644 .changes/1.8.0-b2.md
 rename .changes/{unreleased => 1.8.0}/Dependencies-20240319-102258.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Dependencies-20240403-134956.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Features-20240318-032256.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20240322-113720.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Fixes-20240327-210249.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Security-20240327-193211.yaml (100%)
 rename .changes/{unreleased => 1.8.0}/Under the Hood-20240227-004659.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index c33a2c013..73edb0674 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.8.0b1
+current_version = 1.8.0b2
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.8.0-b2.md b/.changes/1.8.0-b2.md
new file mode 100644
index 000000000..64aa32c7a
--- /dev/null
+++ b/.changes/1.8.0-b2.md
@@ -0,0 +1,23 @@
+## dbt-bigquery 1.8.0-b2 - April 03, 2024
+
+### Features
+
+- Add new workflow for internal patch releases ([#38](https://github.com/dbt-labs/dbt-bigquery/issues/38))
+
+### Fixes
+
+- remove `keyfile` from `_connection_keys` ([#1146](https://github.com/dbt-labs/dbt-bigquery/issues/1146))
+- Add `pandas` extra for `google-cloud-bigquery` to pick up missing `pyarrow` dependency ([#1152](https://github.com/dbt-labs/dbt-bigquery/issues/1152))
+
+### Under the Hood
+
+- Add unit test for transaction semantics. ([#1123](https://github.com/dbt-labs/dbt-bigquery/issues/1123))
+
+### Dependencies
+
+- hard pin ddtrace to 2.3.0 ([#1141](https://github.com/dbt-labs/dbt-bigquery/pull/1141))
+- Add `dbt-core` as a dependency to preserve backwards compatibility for installation ([#1168](https://github.com/dbt-labs/dbt-bigquery/pull/1168))
+
+### Security
+
+- Pin `black>=24.3` in `dev-requirements.txt` ([#1151](https://github.com/dbt-labs/dbt-bigquery/pull/1151))
diff --git a/.changes/unreleased/Dependencies-20240319-102258.yaml b/.changes/1.8.0/Dependencies-20240319-102258.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20240319-102258.yaml
rename to .changes/1.8.0/Dependencies-20240319-102258.yaml
diff --git a/.changes/unreleased/Dependencies-20240403-134956.yaml b/.changes/1.8.0/Dependencies-20240403-134956.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20240403-134956.yaml
rename to .changes/1.8.0/Dependencies-20240403-134956.yaml
diff --git a/.changes/unreleased/Features-20240318-032256.yaml b/.changes/1.8.0/Features-20240318-032256.yaml
similarity index 100%
rename from .changes/unreleased/Features-20240318-032256.yaml
rename to .changes/1.8.0/Features-20240318-032256.yaml
diff --git a/.changes/unreleased/Fixes-20240322-113720.yaml b/.changes/1.8.0/Fixes-20240322-113720.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20240322-113720.yaml
rename to .changes/1.8.0/Fixes-20240322-113720.yaml
diff --git a/.changes/unreleased/Fixes-20240327-210249.yaml b/.changes/1.8.0/Fixes-20240327-210249.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20240327-210249.yaml
rename to .changes/1.8.0/Fixes-20240327-210249.yaml
diff --git a/.changes/unreleased/Security-20240327-193211.yaml b/.changes/1.8.0/Security-20240327-193211.yaml
similarity index 100%
rename from .changes/unreleased/Security-20240327-193211.yaml
rename to .changes/1.8.0/Security-20240327-193211.yaml
diff --git a/.changes/unreleased/Under the Hood-20240227-004659.yaml b/.changes/1.8.0/Under the Hood-20240227-004659.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20240227-004659.yaml
rename to .changes/1.8.0/Under the Hood-20240227-004659.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 6012f18df..62c0bc82a 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,6 +5,32 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.8.0-b2 - April 03, 2024
+
+### Features
+
+- Add new workflow for internal patch releases ([#38](https://github.com/dbt-labs/dbt-bigquery/issues/38))
+
+### Fixes
+
+- remove `keyfile` from `_connection_keys` ([#1146](https://github.com/dbt-labs/dbt-bigquery/issues/1146))
+- Add `pandas` extra for `google-cloud-bigquery` to pick up missing `pyarrow` dependency ([#1152](https://github.com/dbt-labs/dbt-bigquery/issues/1152))
+
+### Under the Hood
+
+- Add unit test for transaction semantics. ([#1123](https://github.com/dbt-labs/dbt-bigquery/issues/1123))
+
+### Dependencies
+
+- hard pin ddtrace to 2.3.0 ([#1141](https://github.com/dbt-labs/dbt-bigquery/pull/1141))
+- Add `dbt-core` as a dependency to preserve backwards compatibility for installation ([#1168](https://github.com/dbt-labs/dbt-bigquery/pull/1168))
+
+### Security
+
+- Pin `black>=24.3` in `dev-requirements.txt` ([#1151](https://github.com/dbt-labs/dbt-bigquery/pull/1151))
+
+
+
 ## dbt-bigquery 1.8.0-b1 - March 01, 2024
 
 ### Features
@@ -63,7 +89,6 @@
 - [@matt-winkler](https://github.com/matt-winkler) ([#972](https://github.com/dbt-labs/dbt-bigquery/issues/972))
 - [@tnk-ysk](https://github.com/tnk-ysk) ([#896](https://github.com/dbt-labs/dbt-bigquery/issues/896), [#792](https://github.com/dbt-labs/dbt-bigquery/issues/792))
 
-
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 - [1.6](https://github.com/dbt-labs/dbt-bigquery/blob/1.6.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 6496f3e22..7d16c28f0 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.8.0b1"
+version = "1.8.0b2"
diff --git a/setup.py b/setup.py
index 1308e2dfd..c972e6e5a 100644
--- a/setup.py
+++ b/setup.py
@@ -36,7 +36,7 @@ def _dbt_bigquery_version() -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.8.0b1"
+package_version = "1.8.0b2"
 description = """The BigQuery adapter plugin for dbt"""
 
 setup(

From bd1eb66dca5d860526f0093dff257c8fc6c9a294 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Fri, 12 Apr 2024 11:51:20 -0400
Subject: [PATCH 784/860] Update dependabot config to cover GHA (#1176)

* Update dependabot to cover GHA
---
 .changes/unreleased/Under the Hood-20240410-182121.yaml | 6 ++++++
 .github/dependabot.yml                                  | 6 +++++-
 2 files changed, 11 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Under the Hood-20240410-182121.yaml

diff --git a/.changes/unreleased/Under the Hood-20240410-182121.yaml b/.changes/unreleased/Under the Hood-20240410-182121.yaml
new file mode 100644
index 000000000..34b794a51
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20240410-182121.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Update dependabot config to cover GHA
+time: 2024-04-10T18:21:21.818397-04:00
+custom:
+  Author: mikealfare
+  Issue: "1176"
diff --git a/.github/dependabot.yml b/.github/dependabot.yml
index 2a6f34492..4673f47cf 100644
--- a/.github/dependabot.yml
+++ b/.github/dependabot.yml
@@ -1,8 +1,12 @@
 version: 2
 updates:
-  # python dependencies
   - package-ecosystem: "pip"
     directory: "/"
     schedule:
       interval: "daily"
     rebase-strategy: "disabled"
+  - package-ecosystem: "github-actions"
+    directory: "/"
+    schedule:
+      interval: "weekly"
+    rebase-strategy: "disabled"

From 55689b99b3522aa4890783000991782d74ef2d9f Mon Sep 17 00:00:00 2001
From: Benoit Perigaud <8754100+b-per@users.noreply.github.com>
Date: Wed, 17 Apr 2024 03:01:27 +0200
Subject: [PATCH 785/860] Create BQ version of string_literal with correct
 quoting (#1089)

* Create BQ version of string_literal with correct quoting

* Fix macro for whitespace and add test.

---------

Co-authored-by: dave-connors-3 <73915542+dave-connors-3@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20240205-102954.yaml  |  6 ++++++
 .../bigquery/macros/utils/string_literal.sql    |  3 +++
 .../adapter/test_string_literal_macro.py        | 17 +++++++++++++++++
 3 files changed, 26 insertions(+)
 create mode 100644 .changes/unreleased/Fixes-20240205-102954.yaml
 create mode 100644 dbt/include/bigquery/macros/utils/string_literal.sql
 create mode 100644 tests/functional/adapter/test_string_literal_macro.py

diff --git a/.changes/unreleased/Fixes-20240205-102954.yaml b/.changes/unreleased/Fixes-20240205-102954.yaml
new file mode 100644
index 000000000..85dae5dbd
--- /dev/null
+++ b/.changes/unreleased/Fixes-20240205-102954.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fix dbt.string_literal for BQ when there are newlines
+time: 2024-02-05T10:29:54.145326+01:00
+custom:
+  Author: b-per
+  Issue: "1088"
diff --git a/dbt/include/bigquery/macros/utils/string_literal.sql b/dbt/include/bigquery/macros/utils/string_literal.sql
new file mode 100644
index 000000000..07e67319a
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/string_literal.sql
@@ -0,0 +1,3 @@
+{%- macro bigquery__string_literal(value) -%}
+    '''{{ value }}'''
+{%- endmacro -%}
diff --git a/tests/functional/adapter/test_string_literal_macro.py b/tests/functional/adapter/test_string_literal_macro.py
new file mode 100644
index 000000000..d67f4be71
--- /dev/null
+++ b/tests/functional/adapter/test_string_literal_macro.py
@@ -0,0 +1,17 @@
+import pytest
+from dbt.tests.util import run_dbt
+
+
+_MODEL_SQL = """
+select {{ dbt.string_literal('my multiline
+string') }} as test
+"""
+
+
+class TestStringLiteralQuoting:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {"my_model.sql": _MODEL_SQL}
+
+    def test_string_literal_quoting(self, project):
+        run_dbt()

From b62ce7469a47de4e1f526fc551817b3e0919ddb2 Mon Sep 17 00:00:00 2001
From: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Fri, 19 Apr 2024 09:32:20 -0700
Subject: [PATCH 786/860] Preserve nicely formatted timeout exception (#1187)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* Add descriptive error message on query timeout

* Add descriptive error message on query timeout
---
 dbt/adapters/bigquery/connections.py | 10 +++++++---
 tests/unit/test_bigquery_adapter.py  |  2 +-
 2 files changed, 8 insertions(+), 4 deletions(-)

diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 09558a14a..f96bc1381 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -1,3 +1,4 @@
+from concurrent.futures import TimeoutError
 import json
 import re
 from contextlib import contextmanager
@@ -736,9 +737,12 @@ def _query_and_results(
             logger.debug(
                 self._bq_job_link(query_job.location, query_job.project, query_job.job_id)
             )
-
-        iterator = query_job.result(max_results=limit, timeout=job_execution_timeout)
-        return query_job, iterator
+        try:
+            iterator = query_job.result(max_results=limit, timeout=job_execution_timeout)
+            return query_job, iterator
+        except TimeoutError:
+            exc = f"Operation did not complete within the designated timeout of {job_execution_timeout} seconds."
+            raise TimeoutError(exc)
 
     def _retry_and_handle(self, msg, conn, fn):
         """retry a function call within the context of exception_handler."""
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 34abd0caf..248e8e2be 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -20,7 +20,7 @@
 from dbt.adapters.bigquery.connections import _sanitize_label, _VALIDATE_LABEL_LENGTH_LIMIT
 from dbt_common.clients import agate_helper
 import dbt_common.exceptions
-from dbt.context.manifest import generate_query_header_context
+from dbt.context.query_header import generate_query_header_context
 from dbt.contracts.files import FileHash
 from dbt.contracts.graph.manifest import ManifestStateCheck
 from dbt.logger import GLOBAL_LOGGER as logger  # noqa

From a81dee0ce27381a173d7fd7fbe71102225a962fe Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Fri, 19 Apr 2024 12:46:21 -0400
Subject: [PATCH 787/860] Add pre-commit check for dbt-core (#1188)

* add pre-commit check for dbt-core
* fix unit test ref

---------

Co-authored-by: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .pre-commit-config.yaml | 4 ++++
 1 file changed, 4 insertions(+)

diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index a98ba0417..74dbdf99a 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -13,6 +13,10 @@ repos:
   - id: end-of-file-fixer
   - id: trailing-whitespace
   - id: check-case-conflict
+- repo: https://github.com/dbt-labs/pre-commit-hooks
+  rev: v0.1.0a1
+  hooks:
+  - id: dbt-core-in-adapters-check
 - repo: https://github.com/psf/black
   rev: 23.1.0
   hooks:

From 49d6c1e1486e4cef470f18d51dcf607b84fe7dde Mon Sep 17 00:00:00 2001
From: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Wed, 24 Apr 2024 11:29:21 -0700
Subject: [PATCH 788/860] Stop adding aliases to render_limited output (#1190)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* Stop adding aliases to render_limited output

* add changie
---
 .changes/unreleased/Fixes-20240423-131752.yaml | 6 ++++++
 dbt/adapters/bigquery/relation.py              | 1 +
 tests/functional/adapter/empty/test_empty.py   | 6 +++++-
 3 files changed, 12 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20240423-131752.yaml

diff --git a/.changes/unreleased/Fixes-20240423-131752.yaml b/.changes/unreleased/Fixes-20240423-131752.yaml
new file mode 100644
index 000000000..048c23d8a
--- /dev/null
+++ b/.changes/unreleased/Fixes-20240423-131752.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Stop adding aliases to render_limited output
+time: 2024-04-23T13:17:52.059553-07:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "1190"
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index 3a3a7fbe6..086b4a2aa 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -23,6 +23,7 @@
 class BigQueryRelation(BaseRelation):
     quote_character: str = "`"
     location: Optional[str] = None
+    require_alias: bool = False
 
     renameable_relations: FrozenSet[RelationType] = field(
         default_factory=lambda: frozenset(
diff --git a/tests/functional/adapter/empty/test_empty.py b/tests/functional/adapter/empty/test_empty.py
index c224c51df..3bf47f35d 100644
--- a/tests/functional/adapter/empty/test_empty.py
+++ b/tests/functional/adapter/empty/test_empty.py
@@ -1,5 +1,9 @@
-from dbt.tests.adapter.empty.test_empty import BaseTestEmpty
+from dbt.tests.adapter.empty.test_empty import BaseTestEmpty, BaseTestEmptyInlineSourceRef
 
 
 class TestBigQueryEmpty(BaseTestEmpty):
     pass
+
+
+class TestBigQueryEmptyInlineSourceRef(BaseTestEmptyInlineSourceRef):
+    pass

From 3bd44ac570d54883bf3a45998f27662bfe10c985 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Thu, 25 Apr 2024 10:20:04 -0400
Subject: [PATCH 789/860] pin macos test runners to macos-12 (#1195)

---
 .github/scripts/integration-test-matrix.js | 4 ++--
 .github/workflows/main.yml                 | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/.github/scripts/integration-test-matrix.js b/.github/scripts/integration-test-matrix.js
index bf7fd2ef7..1a3136cf6 100644
--- a/.github/scripts/integration-test-matrix.js
+++ b/.github/scripts/integration-test-matrix.js
@@ -44,7 +44,7 @@ module.exports = ({ context }) => {
 
             if (labels.includes("test macos") || testAllLabel) {
               include.push({
-                os: "macos-latest",
+                os: "macos-12",
                 adapter,
                 "python-version": pythonVersion,
               });
@@ -78,7 +78,7 @@ module.exports = ({ context }) => {
   // additionally include runs for all adapters, on macos and windows,
   // but only for the default python version
   for (const adapter of supportedAdapters) {
-    for (const operatingSystem of ["windows-latest", "macos-latest"]) {
+    for (const operatingSystem of ["windows-latest", "macos-12"]) {
       include.push({
         os: operatingSystem,
         adapter: adapter,
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index fbaf30925..4b06bc468 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -173,7 +173,7 @@ jobs:
     strategy:
       fail-fast: false
       matrix:
-        os: [ubuntu-latest, macos-latest, windows-latest]
+        os: [ubuntu-latest, macos-12, windows-latest]
         python-version: ['3.8', '3.9', '3.10', '3.11']
 
     steps:

From f62d99cc3c3e199e78aa58e4e644a9020bf101be Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 25 Apr 2024 23:45:28 +0000
Subject: [PATCH 790/860] Bump actions/checkout from 3 to 4 (#1183)

* Bump actions/checkout from 3 to 4

Bumps [actions/checkout](https://github.com/actions/checkout) from 3 to 4.
- [Release notes](https://github.com/actions/checkout/releases)
- [Changelog](https://github.com/actions/checkout/blob/main/CHANGELOG.md)
- [Commits](https://github.com/actions/checkout/compare/v3...v4)

---
updated-dependencies:
- dependency-name: actions/checkout
  dependency-type: direct:production
  update-type: version-update:semver-major
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .../unreleased/Dependencies-20240412-155354.yaml     |  6 ++++++
 .github/workflows/integration.yml                    | 12 ++++++------
 .github/workflows/main.yml                           |  6 +++---
 .github/workflows/nightly-release.yml                |  2 +-
 4 files changed, 16 insertions(+), 10 deletions(-)
 create mode 100644 .changes/unreleased/Dependencies-20240412-155354.yaml

diff --git a/.changes/unreleased/Dependencies-20240412-155354.yaml b/.changes/unreleased/Dependencies-20240412-155354.yaml
new file mode 100644
index 000000000..082ecb6b7
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240412-155354.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump actions/checkout from 3 to 4"
+time: 2024-04-12T15:53:54.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1183
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 7df6973a8..70c01b308 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -69,13 +69,13 @@ jobs:
     steps:
       - name: Check out the repository (non-PR)
         if: github.event_name != 'pull_request_target'
-        uses: actions/checkout@v3
+        uses: actions/checkout@v4
         with:
           persist-credentials: false
 
       - name: Check out the repository (PR)
         if: github.event_name == 'pull_request_target'
-        uses: actions/checkout@v3
+        uses: actions/checkout@v4
         with:
           persist-credentials: false
           ref: ${{ github.event.pull_request.head.sha }}
@@ -143,7 +143,7 @@ jobs:
     steps:
       - name: Check out the repository
         if: github.event_name != 'pull_request_target'
-        uses: actions/checkout@v3
+        uses: actions/checkout@v4
         with:
           persist-credentials: false
 
@@ -151,7 +151,7 @@ jobs:
       # this is necessary for the `pull_request_target` event
       - name: Check out the repository (PR)
         if: github.event_name == 'pull_request_target'
-        uses: actions/checkout@v3
+        uses: actions/checkout@v4
         with:
           persist-credentials: false
           ref: ${{ github.event.pull_request.head.sha }}
@@ -223,7 +223,7 @@ jobs:
     steps:
       - name: Check out the repository
         if: github.event_name != 'pull_request_target'
-        uses: actions/checkout@v3
+        uses: actions/checkout@v4
         with:
           persist-credentials: false
 
@@ -231,7 +231,7 @@ jobs:
       # this is necessary for the `pull_request_target` event
       - name: Check out the repository (PR)
         if: github.event_name == 'pull_request_target'
-        uses: actions/checkout@v3
+        uses: actions/checkout@v4
         with:
           persist-credentials: false
           ref: ${{ github.event.pull_request.head.sha }}
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 4b06bc468..1b168b8a0 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -43,7 +43,7 @@ jobs:
 
     steps:
       - name: Check out the repository
-        uses: actions/checkout@v3
+        uses: actions/checkout@v4
         with:
           persist-credentials: false
 
@@ -79,7 +79,7 @@ jobs:
 
     steps:
       - name: Check out the repository
-        uses: actions/checkout@v3
+        uses: actions/checkout@v4
         with:
           persist-credentials: false
 
@@ -120,7 +120,7 @@ jobs:
 
     steps:
       - name: Check out the repository
-        uses: actions/checkout@v3
+        uses: actions/checkout@v4
         with:
           persist-credentials: false
 
diff --git a/.github/workflows/nightly-release.yml b/.github/workflows/nightly-release.yml
index ef210dacd..26a4164e1 100644
--- a/.github/workflows/nightly-release.yml
+++ b/.github/workflows/nightly-release.yml
@@ -39,7 +39,7 @@ jobs:
 
     steps:
       - name: "Checkout ${{ github.repository }} Branch ${{ env.RELEASE_BRANCH }}"
-        uses: actions/checkout@v3
+        uses: actions/checkout@v4
         with:
           ref: ${{ env.RELEASE_BRANCH }}
 

From 3b8c6e872d8388f46f1754853d997629fee827cf Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 25 Apr 2024 23:57:18 +0000
Subject: [PATCH 791/860] Bump actions/setup-python from 4 to 5 (#1182)

* Bump actions/setup-python from 4 to 5

Bumps [actions/setup-python](https://github.com/actions/setup-python) from 4 to 5.
- [Release notes](https://github.com/actions/setup-python/releases)
- [Commits](https://github.com/actions/setup-python/compare/v4...v5)

---
updated-dependencies:
- dependency-name: actions/setup-python
  dependency-type: direct:production
  update-type: version-update:semver-major
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20240412-155355.yaml | 6 ++++++
 .github/workflows/integration.yml                     | 4 ++--
 .github/workflows/main.yml                            | 8 ++++----
 3 files changed, 12 insertions(+), 6 deletions(-)
 create mode 100644 .changes/unreleased/Dependencies-20240412-155355.yaml

diff --git a/.changes/unreleased/Dependencies-20240412-155355.yaml b/.changes/unreleased/Dependencies-20240412-155355.yaml
new file mode 100644
index 000000000..c1bb923a1
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240412-155355.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump actions/setup-python from 4 to 5"
+time: 2024-04-12T15:53:54.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1182
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 70c01b308..9ad6872ef 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -157,7 +157,7 @@ jobs:
           ref: ${{ github.event.pull_request.head.sha }}
 
       - name: Set up Python ${{ matrix.python-version }}
-        uses: actions/setup-python@v4
+        uses: actions/setup-python@v5
         with:
           python-version: ${{ matrix.python-version }}
 
@@ -237,7 +237,7 @@ jobs:
           ref: ${{ github.event.pull_request.head.sha }}
 
       - name: Set up Python 3.8
-        uses: actions/setup-python@v4
+        uses: actions/setup-python@v5
         with:
           python-version: "3.8"
 
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 1b168b8a0..fdbd36dba 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -48,7 +48,7 @@ jobs:
           persist-credentials: false
 
       - name: Set up Python
-        uses: actions/setup-python@v4
+        uses: actions/setup-python@v5
         with:
           python-version: '3.8'
 
@@ -84,7 +84,7 @@ jobs:
           persist-credentials: false
 
       - name: Set up Python ${{ matrix.python-version }}
-        uses: actions/setup-python@v4
+        uses: actions/setup-python@v5
         with:
           python-version: ${{ matrix.python-version }}
 
@@ -125,7 +125,7 @@ jobs:
           persist-credentials: false
 
       - name: Set up Python
-        uses: actions/setup-python@v4
+        uses: actions/setup-python@v5
         with:
           python-version: '3.8'
 
@@ -178,7 +178,7 @@ jobs:
 
     steps:
       - name: Set up Python ${{ matrix.python-version }}
-        uses: actions/setup-python@v4
+        uses: actions/setup-python@v5
         with:
           python-version: ${{ matrix.python-version }}
       - name: Install python dependencies

From f923338d46a841803a6f7b01ac1bd9c366965fa7 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 25 Apr 2024 23:59:36 +0000
Subject: [PATCH 792/860] Bump actions/github-script from 6 to 7 (#1180)

* Bump actions/github-script from 6 to 7

Bumps [actions/github-script](https://github.com/actions/github-script) from 6 to 7.
- [Release notes](https://github.com/actions/github-script/releases)
- [Commits](https://github.com/actions/github-script/compare/v6...v7)

---
updated-dependencies:
- dependency-name: actions/github-script
  dependency-type: direct:production
  update-type: version-update:semver-major
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20240412-155237.yaml | 6 ++++++
 .github/workflows/integration.yml                     | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20240412-155237.yaml

diff --git a/.changes/unreleased/Dependencies-20240412-155237.yaml b/.changes/unreleased/Dependencies-20240412-155237.yaml
new file mode 100644
index 000000000..98f0b65a0
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240412-155237.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump actions/github-script from 6 to 7"
+time: 2024-04-12T15:52:37.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1180
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 9ad6872ef..c72324128 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -100,7 +100,7 @@ jobs:
 
       - name: Generate integration test matrix
         id: generate-matrix
-        uses: actions/github-script@v6
+        uses: actions/github-script@v7
         env:
           CHANGES: ${{ steps.get-changes.outputs.changes }}
         with:

From 8e139ebee293cc4b402d45a8774710fe2d0370cc Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 26 Apr 2024 00:02:01 +0000
Subject: [PATCH 793/860] Bump dorny/paths-filter from 2 to 3 (#1179)

* Bump dorny/paths-filter from 2 to 3

Bumps [dorny/paths-filter](https://github.com/dorny/paths-filter) from 2 to 3.
- [Release notes](https://github.com/dorny/paths-filter/releases)
- [Changelog](https://github.com/dorny/paths-filter/blob/master/CHANGELOG.md)
- [Commits](https://github.com/dorny/paths-filter/compare/v2...v3)

---
updated-dependencies:
- dependency-name: dorny/paths-filter
  dependency-type: direct:production
  update-type: version-update:semver-major
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20240412-155228.yaml | 6 ++++++
 .github/workflows/integration.yml                     | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20240412-155228.yaml

diff --git a/.changes/unreleased/Dependencies-20240412-155228.yaml b/.changes/unreleased/Dependencies-20240412-155228.yaml
new file mode 100644
index 000000000..b155025b8
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240412-155228.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump dorny/paths-filter from 2 to 3"
+time: 2024-04-12T15:52:28.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1179
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index c72324128..2ae9684d9 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -88,7 +88,7 @@ jobs:
         #  'false' - if none of changed files matches any of filter rules
         # also, returns:
         #  `changes` - JSON array with names of all filters matching any of the changed files
-        uses: dorny/paths-filter@v2
+        uses: dorny/paths-filter@v3
         id: get-changes
         with:
           token: ${{ secrets.GITHUB_TOKEN }}

From 461fa04abec0129304c2cde414624f051789f5a2 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 26 Apr 2024 00:05:31 +0000
Subject: [PATCH 794/860] Bump dbt-labs/actions from 1.1.0 to 1.1.1 (#1181)

* Bump dbt-labs/actions from 1.1.0 to 1.1.1

Bumps [dbt-labs/actions](https://github.com/dbt-labs/actions) from 1.1.0 to 1.1.1.
- [Commits](https://github.com/dbt-labs/actions/compare/v1.1.0...v1.1.1)

---
updated-dependencies:
- dependency-name: dbt-labs/actions
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20240412-155246.yaml | 6 ++++++
 .github/workflows/nightly-release.yml                 | 4 ++--
 2 files changed, 8 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Dependencies-20240412-155246.yaml

diff --git a/.changes/unreleased/Dependencies-20240412-155246.yaml b/.changes/unreleased/Dependencies-20240412-155246.yaml
new file mode 100644
index 000000000..a2181887f
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240412-155246.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump dbt-labs/actions from 1.1.0 to 1.1.1"
+time: 2024-04-12T15:52:46.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1181
diff --git a/.github/workflows/nightly-release.yml b/.github/workflows/nightly-release.yml
index 26a4164e1..517015854 100644
--- a/.github/workflows/nightly-release.yml
+++ b/.github/workflows/nightly-release.yml
@@ -57,7 +57,7 @@ jobs:
 
       - name: "Audit Version And Parse Into Parts"
         id: semver
-        uses: dbt-labs/actions/parse-semver@v1.1.0
+        uses: dbt-labs/actions/parse-semver@v1.1.1
         with:
           version: ${{ steps.version-number-sources.outputs.current_version }}
 
@@ -79,7 +79,7 @@ jobs:
           echo "number=$number" >> $GITHUB_OUTPUT
 
       - name: "Audit Nightly Release Version And Parse Into Parts"
-        uses: dbt-labs/actions/parse-semver@v1.1.0
+        uses: dbt-labs/actions/parse-semver@v1.1.1
         with:
           version: ${{ steps.nightly-release-version.outputs.number }}
 

From b2a3b51fda6a468478e0fb8565ec1cf4be1611db Mon Sep 17 00:00:00 2001
From: FishtownBuildBot <77737458+FishtownBuildBot@users.noreply.github.com>
Date: Fri, 3 May 2024 18:23:03 -0400
Subject: [PATCH 795/860] Cleanup main after cutting new 1.8.latest branch
 (#1223)

* Clean up changelog on main

* Bumping version to 1.9.0a1

* Code quality cleanup
---
 .bumpversion.cfg                              |  2 +-
 .changes/1.8.0-b1.md                          | 57 -------------
 .changes/1.8.0-b2.md                          | 23 -----
 .../1.8.0/Dependencies-20231002-164012.yaml   |  6 --
 .../1.8.0/Dependencies-20231009-005842.yaml   |  6 --
 .../1.8.0/Dependencies-20231011-002031.yaml   |  6 --
 .../1.8.0/Dependencies-20231016-002928.yaml   |  6 --
 .../1.8.0/Dependencies-20231018-010429.yaml   |  6 --
 .../1.8.0/Dependencies-20231027-132742.yaml   |  6 --
 .../1.8.0/Dependencies-20231027-201709.yaml   |  6 --
 .../1.8.0/Dependencies-20231109-005623.yaml   |  6 --
 .../1.8.0/Dependencies-20231113-002529.yaml   |  6 --
 .../1.8.0/Dependencies-20231113-002621.yaml   |  6 --
 .../1.8.0/Dependencies-20231116-001342.yaml   |  6 --
 .../1.8.0/Dependencies-20231127-004827.yaml   |  6 --
 .../1.8.0/Dependencies-20231128-005012.yaml   |  6 --
 .../1.8.0/Dependencies-20231128-005103.yaml   |  6 --
 .../1.8.0/Dependencies-20231129-001523.yaml   |  6 --
 .../1.8.0/Dependencies-20231204-003807.yaml   |  6 --
 .../1.8.0/Dependencies-20231213-003845.yaml   |  6 --
 .../1.8.0/Dependencies-20240124-120321.yaml   |  6 --
 .../1.8.0/Dependencies-20240319-102258.yaml   |  6 --
 .../1.8.0/Dependencies-20240403-134956.yaml   |  6 --
 .changes/1.8.0/Features-20231218-155409.yaml  |  6 --
 .changes/1.8.0/Features-20231219-201203.yaml  |  6 --
 .changes/1.8.0/Features-20240102-152030.yaml  |  6 --
 .changes/1.8.0/Features-20240205-174614.yaml  |  7 --
 .changes/1.8.0/Features-20240318-032256.yaml  |  6 --
 .changes/1.8.0/Fixes-20231023-082312.yaml     |  6 --
 .changes/1.8.0/Fixes-20231025-131907.yaml     |  6 --
 .changes/1.8.0/Fixes-20231025-223003.yaml     |  6 --
 .changes/1.8.0/Fixes-20231030-222134.yaml     |  6 --
 .changes/1.8.0/Fixes-20231105-125740.yaml     |  6 --
 .changes/1.8.0/Fixes-20231105-143145.yaml     |  6 --
 .changes/1.8.0/Fixes-20231107-100905.yaml     |  6 --
 .changes/1.8.0/Fixes-20231107-174352.yaml     |  7 --
 .changes/1.8.0/Fixes-20231108-171128.yaml     |  6 --
 .changes/1.8.0/Fixes-20231111-150959.yaml     |  6 --
 .changes/1.8.0/Fixes-20231219-153446.yaml     |  6 --
 .changes/1.8.0/Fixes-20240216-133319.yaml     |  6 --
 .changes/1.8.0/Fixes-20240219-103324.yaml     |  6 --
 .changes/1.8.0/Fixes-20240322-113720.yaml     |  6 --
 .changes/1.8.0/Fixes-20240327-210249.yaml     |  7 --
 .changes/1.8.0/Security-20240327-193211.yaml  |  6 --
 .../1.8.0/Under the Hood-20231109-095012.yaml |  6 --
 .../1.8.0/Under the Hood-20231116-062142.yaml |  6 --
 .../1.8.0/Under the Hood-20231117-121214.yaml |  6 --
 .../1.8.0/Under the Hood-20240116-154305.yaml |  6 --
 .../1.8.0/Under the Hood-20240227-004659.yaml |  6 --
 .../Dependencies-20240412-155228.yaml         |  6 --
 .../Dependencies-20240412-155237.yaml         |  6 --
 .../Dependencies-20240412-155246.yaml         |  6 --
 .../Dependencies-20240412-155354.yaml         |  6 --
 .../Dependencies-20240412-155355.yaml         |  6 --
 .../unreleased/Fixes-20240205-102954.yaml     |  6 --
 .../unreleased/Fixes-20240423-131752.yaml     |  6 --
 .../Under the Hood-20240410-182121.yaml       |  6 --
 CHANGELOG.md                                  | 84 -------------------
 dbt/adapters/bigquery/__version__.py          |  2 +-
 setup.py                                      |  2 +-
 60 files changed, 3 insertions(+), 494 deletions(-)
 delete mode 100644 .changes/1.8.0-b1.md
 delete mode 100644 .changes/1.8.0-b2.md
 delete mode 100644 .changes/1.8.0/Dependencies-20231002-164012.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231009-005842.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231011-002031.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231016-002928.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231018-010429.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231027-132742.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231027-201709.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231109-005623.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231113-002529.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231113-002621.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231116-001342.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231127-004827.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231128-005012.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231128-005103.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231129-001523.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231204-003807.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20231213-003845.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20240124-120321.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20240319-102258.yaml
 delete mode 100644 .changes/1.8.0/Dependencies-20240403-134956.yaml
 delete mode 100644 .changes/1.8.0/Features-20231218-155409.yaml
 delete mode 100644 .changes/1.8.0/Features-20231219-201203.yaml
 delete mode 100644 .changes/1.8.0/Features-20240102-152030.yaml
 delete mode 100644 .changes/1.8.0/Features-20240205-174614.yaml
 delete mode 100644 .changes/1.8.0/Features-20240318-032256.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20231023-082312.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20231025-131907.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20231025-223003.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20231030-222134.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20231105-125740.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20231105-143145.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20231107-100905.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20231107-174352.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20231108-171128.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20231111-150959.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20231219-153446.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20240216-133319.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20240219-103324.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20240322-113720.yaml
 delete mode 100644 .changes/1.8.0/Fixes-20240327-210249.yaml
 delete mode 100644 .changes/1.8.0/Security-20240327-193211.yaml
 delete mode 100644 .changes/1.8.0/Under the Hood-20231109-095012.yaml
 delete mode 100644 .changes/1.8.0/Under the Hood-20231116-062142.yaml
 delete mode 100644 .changes/1.8.0/Under the Hood-20231117-121214.yaml
 delete mode 100644 .changes/1.8.0/Under the Hood-20240116-154305.yaml
 delete mode 100644 .changes/1.8.0/Under the Hood-20240227-004659.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20240412-155228.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20240412-155237.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20240412-155246.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20240412-155354.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20240412-155355.yaml
 delete mode 100644 .changes/unreleased/Fixes-20240205-102954.yaml
 delete mode 100644 .changes/unreleased/Fixes-20240423-131752.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20240410-182121.yaml

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 73edb0674..3f0db0785 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.8.0b2
+current_version = 1.9.0a1
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.8.0-b1.md b/.changes/1.8.0-b1.md
deleted file mode 100644
index 84b8a17cb..000000000
--- a/.changes/1.8.0-b1.md
+++ /dev/null
@@ -1,57 +0,0 @@
-## dbt-bigquery 1.8.0-b1 - March 01, 2024
-
-### Features
-
-- Add support for checking table-last-modified by metadata ([#938](https://github.com/dbt-labs/dbt-bigquery/issues/938))
-- Support limiting get_catalog by object name ([#950](https://github.com/dbt-labs/dbt-bigquery/issues/950))
-- Update base adapter references as part of decoupling migration ([#1067](https://github.com/dbt-labs/dbt-bigquery/issues/1067))
-- Support all types for unit testing in dbt-bigquery, expand coverage of safe_cast macro ([#1090](https://github.com/dbt-labs/dbt-bigquery/issues/1090))
-
-### Fixes
-
-- Patch for json inline --show ([#972](https://github.com/dbt-labs/dbt-bigquery/issues/972))
-- Lower bound of `2.11.0` for `google-api-core` ([#979](https://github.com/dbt-labs/dbt-bigquery/issues/979))
-- Fix refresh syntax, config comparison with empty labels ([#983](https://github.com/dbt-labs/dbt-bigquery/issues/983))
-- Assign the correct relation type to materialized views in catalog queries ([#995](https://github.com/dbt-labs/dbt-bigquery/issues/995))
-- Fix inline comments (--) on the last line of an incremental model ([#896](https://github.com/dbt-labs/dbt-bigquery/issues/896))
-- In incremental models, add dummy merge condition on source partition column when partition is required ([#792](https://github.com/dbt-labs/dbt-bigquery/issues/792))
-- Support agate Integer type, test with empty seed ([#1003](https://github.com/dbt-labs/dbt-bigquery/issues/1003))
-- Fixed issue where materialized views were failing on re-run with minimal config parameters ([#1007](https://github.com/dbt-labs/dbt-bigquery/issues/1007))
-- Fix broken partition config granularity and batch_id being set to None ([#1006](https://github.com/dbt-labs/dbt-bigquery/issues/1006))
-- replace deterministic batch_id with uuid ([#1006](https://github.com/dbt-labs/dbt-bigquery/issues/1006))
-- remove json patch to leverage bigquery-python improvement ([#1055](https://github.com/dbt-labs/dbt-bigquery/issues/1055))
-- remove `token` field from connection keys ([#1105](https://github.com/dbt-labs/dbt-bigquery/issues/1105))
-- Remove custom query job async timeout logic as it has been fixed in bigquery-python ([#1081](https://github.com/dbt-labs/dbt-bigquery/issues/1081))
-
-### Under the Hood
-
-- Upgrade spark-bigquery Java deps for serverless to 2.13-0.34.0 ([#1006](https://github.com/dbt-labs/dbt-bigquery/issues/1006))
-- Primary and foreign key constraints are not enforced in BigQuery ([#1018](https://github.com/dbt-labs/dbt-bigquery/issues/1018))
-- Add tests for --empty flag ([#1029](https://github.com/dbt-labs/dbt-bigquery/issues/1029))
-- Migrate to dbt-common and dbt-adapters package ([#1071](https://github.com/dbt-labs/dbt-bigquery/issues/1071))
-
-### Dependencies
-
-- Update ddtrace requirement from ~=1.19 to ~=1.20 ([#948](https://github.com/dbt-labs/dbt-bigquery/pull/948))
-- Update pre-commit-hooks requirement from ~=4.4 to ~=4.5 ([#960](https://github.com/dbt-labs/dbt-bigquery/pull/960))
-- Bump mypy from 1.5.1 to 1.6.0 ([#963](https://github.com/dbt-labs/dbt-bigquery/pull/963))
-- Update pre-commit requirement from ~=3.4 to ~=3.5 ([#969](https://github.com/dbt-labs/dbt-bigquery/pull/969))
-- Update black requirement from ~=23.9 to ~=23.10 ([#973](https://github.com/dbt-labs/dbt-bigquery/pull/973))
-- Bump mypy from 1.6.0 to 1.6.1 ([#985](https://github.com/dbt-labs/dbt-bigquery/pull/985))
-- Update ddtrace requirement from ~=1.20 to ~=2.1 ([#989](https://github.com/dbt-labs/dbt-bigquery/pull/989))
-- Update black requirement from ~=23.10 to ~=23.11 ([#1013](https://github.com/dbt-labs/dbt-bigquery/pull/1013))
-- Update pytest-xdist requirement from ~=3.3 to ~=3.4 ([#1022](https://github.com/dbt-labs/dbt-bigquery/pull/1022))
-- Bump mypy from 1.6.1 to 1.7.0 ([#1023](https://github.com/dbt-labs/dbt-bigquery/pull/1023))
-- Update ddtrace requirement from ~=2.1 to ~=2.2 ([#1028](https://github.com/dbt-labs/dbt-bigquery/pull/1028))
-- Update wheel requirement from ~=0.41 to ~=0.42 ([#1033](https://github.com/dbt-labs/dbt-bigquery/pull/1033))
-- Bump mypy from 1.7.0 to 1.7.1 ([#1034](https://github.com/dbt-labs/dbt-bigquery/pull/1034))
-- Update ddtrace requirement from ~=2.2 to ~=2.3 ([#1035](https://github.com/dbt-labs/dbt-bigquery/pull/1035))
-- Update pytest-xdist requirement from ~=3.4 to ~=3.5 ([#1037](https://github.com/dbt-labs/dbt-bigquery/pull/1037))
-- Update freezegun requirement from ~=1.2 to ~=1.3 ([#1040](https://github.com/dbt-labs/dbt-bigquery/pull/1040))
-- Update black requirement from ~=23.11 to ~=23.12 ([#1056](https://github.com/dbt-labs/dbt-bigquery/pull/1056))
-- get dbt-tests-adapters from dbt-adapters repo ([#1077](https://github.com/dbt-labs/dbt-bigquery/pull/1077))
-
-### Contributors
-- [@gmyrianthous](https://github.com/gmyrianthous) ([#979](https://github.com/dbt-labs/dbt-bigquery/issues/979))
-- [@matt-winkler](https://github.com/matt-winkler) ([#972](https://github.com/dbt-labs/dbt-bigquery/issues/972))
-- [@tnk-ysk](https://github.com/tnk-ysk) ([#896](https://github.com/dbt-labs/dbt-bigquery/issues/896), [#792](https://github.com/dbt-labs/dbt-bigquery/issues/792))
diff --git a/.changes/1.8.0-b2.md b/.changes/1.8.0-b2.md
deleted file mode 100644
index 64aa32c7a..000000000
--- a/.changes/1.8.0-b2.md
+++ /dev/null
@@ -1,23 +0,0 @@
-## dbt-bigquery 1.8.0-b2 - April 03, 2024
-
-### Features
-
-- Add new workflow for internal patch releases ([#38](https://github.com/dbt-labs/dbt-bigquery/issues/38))
-
-### Fixes
-
-- remove `keyfile` from `_connection_keys` ([#1146](https://github.com/dbt-labs/dbt-bigquery/issues/1146))
-- Add `pandas` extra for `google-cloud-bigquery` to pick up missing `pyarrow` dependency ([#1152](https://github.com/dbt-labs/dbt-bigquery/issues/1152))
-
-### Under the Hood
-
-- Add unit test for transaction semantics. ([#1123](https://github.com/dbt-labs/dbt-bigquery/issues/1123))
-
-### Dependencies
-
-- hard pin ddtrace to 2.3.0 ([#1141](https://github.com/dbt-labs/dbt-bigquery/pull/1141))
-- Add `dbt-core` as a dependency to preserve backwards compatibility for installation ([#1168](https://github.com/dbt-labs/dbt-bigquery/pull/1168))
-
-### Security
-
-- Pin `black>=24.3` in `dev-requirements.txt` ([#1151](https://github.com/dbt-labs/dbt-bigquery/pull/1151))
diff --git a/.changes/1.8.0/Dependencies-20231002-164012.yaml b/.changes/1.8.0/Dependencies-20231002-164012.yaml
deleted file mode 100644
index 344aeb148..000000000
--- a/.changes/1.8.0/Dependencies-20231002-164012.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update ddtrace requirement from ~=1.19 to ~=1.20"
-time: 2023-10-02T16:40:12.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 948
diff --git a/.changes/1.8.0/Dependencies-20231009-005842.yaml b/.changes/1.8.0/Dependencies-20231009-005842.yaml
deleted file mode 100644
index acedd8d52..000000000
--- a/.changes/1.8.0/Dependencies-20231009-005842.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pre-commit-hooks requirement from ~=4.4 to ~=4.5"
-time: 2023-10-09T00:58:42.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 960
diff --git a/.changes/1.8.0/Dependencies-20231011-002031.yaml b/.changes/1.8.0/Dependencies-20231011-002031.yaml
deleted file mode 100644
index 5cc3c36a1..000000000
--- a/.changes/1.8.0/Dependencies-20231011-002031.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump mypy from 1.5.1 to 1.6.0"
-time: 2023-10-11T00:20:31.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 963
diff --git a/.changes/1.8.0/Dependencies-20231016-002928.yaml b/.changes/1.8.0/Dependencies-20231016-002928.yaml
deleted file mode 100644
index ac9470de7..000000000
--- a/.changes/1.8.0/Dependencies-20231016-002928.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pre-commit requirement from ~=3.4 to ~=3.5"
-time: 2023-10-16T00:29:28.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 969
diff --git a/.changes/1.8.0/Dependencies-20231018-010429.yaml b/.changes/1.8.0/Dependencies-20231018-010429.yaml
deleted file mode 100644
index fec345104..000000000
--- a/.changes/1.8.0/Dependencies-20231018-010429.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update black requirement from ~=23.9 to ~=23.10"
-time: 2023-10-18T01:04:29.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 973
diff --git a/.changes/1.8.0/Dependencies-20231027-132742.yaml b/.changes/1.8.0/Dependencies-20231027-132742.yaml
deleted file mode 100644
index d72ac124b..000000000
--- a/.changes/1.8.0/Dependencies-20231027-132742.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump mypy from 1.6.0 to 1.6.1"
-time: 2023-10-27T13:27:42.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 985
diff --git a/.changes/1.8.0/Dependencies-20231027-201709.yaml b/.changes/1.8.0/Dependencies-20231027-201709.yaml
deleted file mode 100644
index e9d2a1adb..000000000
--- a/.changes/1.8.0/Dependencies-20231027-201709.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update ddtrace requirement from ~=1.20 to ~=2.1"
-time: 2023-10-27T20:17:09.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 989
diff --git a/.changes/1.8.0/Dependencies-20231109-005623.yaml b/.changes/1.8.0/Dependencies-20231109-005623.yaml
deleted file mode 100644
index bc4ca2342..000000000
--- a/.changes/1.8.0/Dependencies-20231109-005623.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update black requirement from ~=23.10 to ~=23.11"
-time: 2023-11-09T00:56:23.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1013
diff --git a/.changes/1.8.0/Dependencies-20231113-002529.yaml b/.changes/1.8.0/Dependencies-20231113-002529.yaml
deleted file mode 100644
index ad7272882..000000000
--- a/.changes/1.8.0/Dependencies-20231113-002529.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pytest-xdist requirement from ~=3.3 to ~=3.4"
-time: 2023-11-13T00:25:29.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1022
diff --git a/.changes/1.8.0/Dependencies-20231113-002621.yaml b/.changes/1.8.0/Dependencies-20231113-002621.yaml
deleted file mode 100644
index d889da51e..000000000
--- a/.changes/1.8.0/Dependencies-20231113-002621.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump mypy from 1.6.1 to 1.7.0"
-time: 2023-11-13T00:26:21.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1023
diff --git a/.changes/1.8.0/Dependencies-20231116-001342.yaml b/.changes/1.8.0/Dependencies-20231116-001342.yaml
deleted file mode 100644
index 6af15169f..000000000
--- a/.changes/1.8.0/Dependencies-20231116-001342.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update ddtrace requirement from ~=2.1 to ~=2.2"
-time: 2023-11-16T00:13:42.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1028
diff --git a/.changes/1.8.0/Dependencies-20231127-004827.yaml b/.changes/1.8.0/Dependencies-20231127-004827.yaml
deleted file mode 100644
index a78708328..000000000
--- a/.changes/1.8.0/Dependencies-20231127-004827.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update wheel requirement from ~=0.41 to ~=0.42"
-time: 2023-11-27T00:48:27.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1033
diff --git a/.changes/1.8.0/Dependencies-20231128-005012.yaml b/.changes/1.8.0/Dependencies-20231128-005012.yaml
deleted file mode 100644
index 236ca1e52..000000000
--- a/.changes/1.8.0/Dependencies-20231128-005012.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump mypy from 1.7.0 to 1.7.1"
-time: 2023-11-28T00:50:12.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1034
diff --git a/.changes/1.8.0/Dependencies-20231128-005103.yaml b/.changes/1.8.0/Dependencies-20231128-005103.yaml
deleted file mode 100644
index 205ecd1d8..000000000
--- a/.changes/1.8.0/Dependencies-20231128-005103.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update ddtrace requirement from ~=2.2 to ~=2.3"
-time: 2023-11-28T00:51:03.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1035
diff --git a/.changes/1.8.0/Dependencies-20231129-001523.yaml b/.changes/1.8.0/Dependencies-20231129-001523.yaml
deleted file mode 100644
index e1c145ced..000000000
--- a/.changes/1.8.0/Dependencies-20231129-001523.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pytest-xdist requirement from ~=3.4 to ~=3.5"
-time: 2023-11-29T00:15:23.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1037
diff --git a/.changes/1.8.0/Dependencies-20231204-003807.yaml b/.changes/1.8.0/Dependencies-20231204-003807.yaml
deleted file mode 100644
index 18b9bb618..000000000
--- a/.changes/1.8.0/Dependencies-20231204-003807.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update freezegun requirement from ~=1.2 to ~=1.3"
-time: 2023-12-04T00:38:07.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1040
diff --git a/.changes/1.8.0/Dependencies-20231213-003845.yaml b/.changes/1.8.0/Dependencies-20231213-003845.yaml
deleted file mode 100644
index 93618b5e0..000000000
--- a/.changes/1.8.0/Dependencies-20231213-003845.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update black requirement from ~=23.11 to ~=23.12"
-time: 2023-12-13T00:38:45.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1056
diff --git a/.changes/1.8.0/Dependencies-20240124-120321.yaml b/.changes/1.8.0/Dependencies-20240124-120321.yaml
deleted file mode 100644
index ef725de67..000000000
--- a/.changes/1.8.0/Dependencies-20240124-120321.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Dependencies
-body: get dbt-tests-adapters from dbt-adapters repo
-time: 2024-01-24T12:03:21.523295-08:00
-custom:
-  Author: colin-rogers-dbt
-  PR: "1077"
diff --git a/.changes/1.8.0/Dependencies-20240319-102258.yaml b/.changes/1.8.0/Dependencies-20240319-102258.yaml
deleted file mode 100644
index 5df197537..000000000
--- a/.changes/1.8.0/Dependencies-20240319-102258.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Dependencies
-body: hard pin ddtrace to 2.3.0
-time: 2024-03-19T10:22:58.3838-05:00
-custom:
-  Author: McKnight-42
-  PR: "1141"
diff --git a/.changes/1.8.0/Dependencies-20240403-134956.yaml b/.changes/1.8.0/Dependencies-20240403-134956.yaml
deleted file mode 100644
index 8e72020b5..000000000
--- a/.changes/1.8.0/Dependencies-20240403-134956.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Dependencies
-body: Add `dbt-core` as a dependency to preserve backwards compatibility for installation
-time: 2024-04-03T13:49:56.063802-04:00
-custom:
-  Author: mikealfare
-  PR: "1168"
diff --git a/.changes/1.8.0/Features-20231218-155409.yaml b/.changes/1.8.0/Features-20231218-155409.yaml
deleted file mode 100644
index bc965b06f..000000000
--- a/.changes/1.8.0/Features-20231218-155409.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Add support for checking table-last-modified by metadata
-time: 2023-12-18T15:54:09.69635-05:00
-custom:
-  Author: mikealfare
-  Issue: "938"
diff --git a/.changes/1.8.0/Features-20231219-201203.yaml b/.changes/1.8.0/Features-20231219-201203.yaml
deleted file mode 100644
index eee3f1026..000000000
--- a/.changes/1.8.0/Features-20231219-201203.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Support limiting get_catalog by object name
-time: 2023-12-19T20:12:03.990725-05:00
-custom:
-  Author: mikealfare
-  Issue: "950"
diff --git a/.changes/1.8.0/Features-20240102-152030.yaml b/.changes/1.8.0/Features-20240102-152030.yaml
deleted file mode 100644
index 81c683de0..000000000
--- a/.changes/1.8.0/Features-20240102-152030.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Update base adapter references as part of decoupling migration
-time: 2024-01-02T15:20:30.038221-08:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "1067"
diff --git a/.changes/1.8.0/Features-20240205-174614.yaml b/.changes/1.8.0/Features-20240205-174614.yaml
deleted file mode 100644
index 192273d3d..000000000
--- a/.changes/1.8.0/Features-20240205-174614.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: Support all types for unit testing in dbt-bigquery, expand coverage of
-  safe_cast macro
-time: 2024-02-05T17:46:14.505597-05:00
-custom:
-  Author: michelleark
-  Issue: "1090"
diff --git a/.changes/1.8.0/Features-20240318-032256.yaml b/.changes/1.8.0/Features-20240318-032256.yaml
deleted file mode 100644
index e4948f433..000000000
--- a/.changes/1.8.0/Features-20240318-032256.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Add new workflow for internal patch releases
-time: 2024-03-18T03:22:56.037781-07:00
-custom:
-  Author: versusfacit
-  Issue: "38"
diff --git a/.changes/1.8.0/Fixes-20231023-082312.yaml b/.changes/1.8.0/Fixes-20231023-082312.yaml
deleted file mode 100644
index 368c58e95..000000000
--- a/.changes/1.8.0/Fixes-20231023-082312.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Patch for json inline --show
-time: 2023-10-23T08:23:12.245223-06:00
-custom:
-  Author: matt-winkler
-  Issue: "972"
diff --git a/.changes/1.8.0/Fixes-20231025-131907.yaml b/.changes/1.8.0/Fixes-20231025-131907.yaml
deleted file mode 100644
index 9a3b8d8a8..000000000
--- a/.changes/1.8.0/Fixes-20231025-131907.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Lower bound of `2.11.0` for `google-api-core`
-time: 2023-10-25T13:19:07.580826-06:00
-custom:
-  Author: gmyrianthous dbeatty10
-  Issue: "979"
diff --git a/.changes/1.8.0/Fixes-20231025-223003.yaml b/.changes/1.8.0/Fixes-20231025-223003.yaml
deleted file mode 100644
index ebec94a30..000000000
--- a/.changes/1.8.0/Fixes-20231025-223003.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Fix refresh syntax, config comparison with empty labels
-time: 2023-10-25T22:30:03.0034-04:00
-custom:
-  Author: mikealfare
-  Issue: "983"
diff --git a/.changes/1.8.0/Fixes-20231030-222134.yaml b/.changes/1.8.0/Fixes-20231030-222134.yaml
deleted file mode 100644
index 62bfc5f27..000000000
--- a/.changes/1.8.0/Fixes-20231030-222134.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Assign the correct relation type to materialized views in catalog queries
-time: 2023-10-30T22:21:34.401675-04:00
-custom:
-  Author: mikealfare
-  Issue: "995"
diff --git a/.changes/1.8.0/Fixes-20231105-125740.yaml b/.changes/1.8.0/Fixes-20231105-125740.yaml
deleted file mode 100644
index 928fbb302..000000000
--- a/.changes/1.8.0/Fixes-20231105-125740.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Fix inline comments (--) on the last line of an incremental model
-time: 2023-11-05T12:57:40.289399+09:00
-custom:
-  Author: tnk-ysk
-  Issue: "896"
diff --git a/.changes/1.8.0/Fixes-20231105-143145.yaml b/.changes/1.8.0/Fixes-20231105-143145.yaml
deleted file mode 100644
index fd404e395..000000000
--- a/.changes/1.8.0/Fixes-20231105-143145.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: In incremental models, add dummy merge condition on source partition column when partition is required
-time: 2023-11-05T14:31:45.869783+09:00
-custom:
-  Author: tnk-ysk
-  Issue: "792"
diff --git a/.changes/1.8.0/Fixes-20231107-100905.yaml b/.changes/1.8.0/Fixes-20231107-100905.yaml
deleted file mode 100644
index 942298ed9..000000000
--- a/.changes/1.8.0/Fixes-20231107-100905.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Support agate Integer type, test with empty seed
-time: 2023-11-07T10:09:05.723451-05:00
-custom:
-  Author: gshank
-  Issue: "1003"
diff --git a/.changes/1.8.0/Fixes-20231107-174352.yaml b/.changes/1.8.0/Fixes-20231107-174352.yaml
deleted file mode 100644
index 80592758d..000000000
--- a/.changes/1.8.0/Fixes-20231107-174352.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: Fixed issue where materialized views were failing on re-run with minimal config
-  parameters
-time: 2023-11-07T17:43:52.972135-05:00
-custom:
-  Author: "mikealfare"
-  Issue: "1007"
diff --git a/.changes/1.8.0/Fixes-20231108-171128.yaml b/.changes/1.8.0/Fixes-20231108-171128.yaml
deleted file mode 100644
index 116ff00d2..000000000
--- a/.changes/1.8.0/Fixes-20231108-171128.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Fix broken partition config granularity and batch_id being set to None
-time: 2023-11-08T17:11:28.819877-08:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "1006"
diff --git a/.changes/1.8.0/Fixes-20231111-150959.yaml b/.changes/1.8.0/Fixes-20231111-150959.yaml
deleted file mode 100644
index 3d9f245a6..000000000
--- a/.changes/1.8.0/Fixes-20231111-150959.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: replace deterministic batch_id with uuid
-time: 2023-11-11T15:09:59.243797-08:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "1006"
diff --git a/.changes/1.8.0/Fixes-20231219-153446.yaml b/.changes/1.8.0/Fixes-20231219-153446.yaml
deleted file mode 100644
index 44a858eb8..000000000
--- a/.changes/1.8.0/Fixes-20231219-153446.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: remove json patch to leverage bigquery-python improvement
-time: 2023-12-19T15:34:46.843931-08:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "1055"
diff --git a/.changes/1.8.0/Fixes-20240216-133319.yaml b/.changes/1.8.0/Fixes-20240216-133319.yaml
deleted file mode 100644
index cb8a2156e..000000000
--- a/.changes/1.8.0/Fixes-20240216-133319.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: remove `token` field from connection keys
-time: 2024-02-16T13:33:19.524482-08:00
-custom:
-  Author: versusfacit
-  Issue: "1105"
diff --git a/.changes/1.8.0/Fixes-20240219-103324.yaml b/.changes/1.8.0/Fixes-20240219-103324.yaml
deleted file mode 100644
index 16906db85..000000000
--- a/.changes/1.8.0/Fixes-20240219-103324.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Remove custom query job async timeout logic as it has been fixed in bigquery-python
-time: 2024-02-19T10:33:24.3385-08:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "1081"
diff --git a/.changes/1.8.0/Fixes-20240322-113720.yaml b/.changes/1.8.0/Fixes-20240322-113720.yaml
deleted file mode 100644
index 9279c6ecd..000000000
--- a/.changes/1.8.0/Fixes-20240322-113720.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: remove `keyfile` from `_connection_keys`
-time: 2024-03-22T11:37:20.989189-05:00
-custom:
-  Author: McKnight-42
-  Issue: "1146"
diff --git a/.changes/1.8.0/Fixes-20240327-210249.yaml b/.changes/1.8.0/Fixes-20240327-210249.yaml
deleted file mode 100644
index 04a5c5e38..000000000
--- a/.changes/1.8.0/Fixes-20240327-210249.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: Add `pandas` extra for `google-cloud-bigquery` to pick up missing `pyarrow`
-  dependency
-time: 2024-03-27T21:02:49.619691-04:00
-custom:
-  Author: mikealfare
-  Issue: "1152"
diff --git a/.changes/1.8.0/Security-20240327-193211.yaml b/.changes/1.8.0/Security-20240327-193211.yaml
deleted file mode 100644
index 1b755959b..000000000
--- a/.changes/1.8.0/Security-20240327-193211.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Security
-body: Pin `black>=24.3` in `dev-requirements.txt`
-time: 2024-03-27T19:32:11.383652-04:00
-custom:
-  Author: mikealfare
-  PR: "1151"
diff --git a/.changes/1.8.0/Under the Hood-20231109-095012.yaml b/.changes/1.8.0/Under the Hood-20231109-095012.yaml
deleted file mode 100644
index a93215e8f..000000000
--- a/.changes/1.8.0/Under the Hood-20231109-095012.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Upgrade spark-bigquery Java deps for serverless to 2.13-0.34.0
-time: 2023-11-09T09:50:12.252774-08:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "1006"
diff --git a/.changes/1.8.0/Under the Hood-20231116-062142.yaml b/.changes/1.8.0/Under the Hood-20231116-062142.yaml
deleted file mode 100644
index c28270898..000000000
--- a/.changes/1.8.0/Under the Hood-20231116-062142.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Primary and foreign key constraints are not enforced in BigQuery
-time: 2023-11-16T06:21:42.935367-08:00
-custom:
-  Author: dbeatty10
-  Issue: "1018"
diff --git a/.changes/1.8.0/Under the Hood-20231117-121214.yaml b/.changes/1.8.0/Under the Hood-20231117-121214.yaml
deleted file mode 100644
index 61b0617ad..000000000
--- a/.changes/1.8.0/Under the Hood-20231117-121214.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Add tests for --empty flag
-time: 2023-11-17T12:12:14.643365-05:00
-custom:
-  Author: michelleark
-  Issue: "1029"
diff --git a/.changes/1.8.0/Under the Hood-20240116-154305.yaml b/.changes/1.8.0/Under the Hood-20240116-154305.yaml
deleted file mode 100644
index bb115abd6..000000000
--- a/.changes/1.8.0/Under the Hood-20240116-154305.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Migrate to dbt-common and dbt-adapters package
-time: 2024-01-16T15:43:05.046735-08:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "1071"
diff --git a/.changes/1.8.0/Under the Hood-20240227-004659.yaml b/.changes/1.8.0/Under the Hood-20240227-004659.yaml
deleted file mode 100644
index 6ef259019..000000000
--- a/.changes/1.8.0/Under the Hood-20240227-004659.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Add unit test for transaction semantics.
-time: 2024-02-27T00:46:59.188231-08:00
-custom:
-  Author: versusfacit
-  Issue: "1123"
diff --git a/.changes/unreleased/Dependencies-20240412-155228.yaml b/.changes/unreleased/Dependencies-20240412-155228.yaml
deleted file mode 100644
index b155025b8..000000000
--- a/.changes/unreleased/Dependencies-20240412-155228.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump dorny/paths-filter from 2 to 3"
-time: 2024-04-12T15:52:28.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1179
diff --git a/.changes/unreleased/Dependencies-20240412-155237.yaml b/.changes/unreleased/Dependencies-20240412-155237.yaml
deleted file mode 100644
index 98f0b65a0..000000000
--- a/.changes/unreleased/Dependencies-20240412-155237.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump actions/github-script from 6 to 7"
-time: 2024-04-12T15:52:37.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1180
diff --git a/.changes/unreleased/Dependencies-20240412-155246.yaml b/.changes/unreleased/Dependencies-20240412-155246.yaml
deleted file mode 100644
index a2181887f..000000000
--- a/.changes/unreleased/Dependencies-20240412-155246.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump dbt-labs/actions from 1.1.0 to 1.1.1"
-time: 2024-04-12T15:52:46.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1181
diff --git a/.changes/unreleased/Dependencies-20240412-155354.yaml b/.changes/unreleased/Dependencies-20240412-155354.yaml
deleted file mode 100644
index 082ecb6b7..000000000
--- a/.changes/unreleased/Dependencies-20240412-155354.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump actions/checkout from 3 to 4"
-time: 2024-04-12T15:53:54.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1183
diff --git a/.changes/unreleased/Dependencies-20240412-155355.yaml b/.changes/unreleased/Dependencies-20240412-155355.yaml
deleted file mode 100644
index c1bb923a1..000000000
--- a/.changes/unreleased/Dependencies-20240412-155355.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump actions/setup-python from 4 to 5"
-time: 2024-04-12T15:53:54.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1182
diff --git a/.changes/unreleased/Fixes-20240205-102954.yaml b/.changes/unreleased/Fixes-20240205-102954.yaml
deleted file mode 100644
index 85dae5dbd..000000000
--- a/.changes/unreleased/Fixes-20240205-102954.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Fix dbt.string_literal for BQ when there are newlines
-time: 2024-02-05T10:29:54.145326+01:00
-custom:
-  Author: b-per
-  Issue: "1088"
diff --git a/.changes/unreleased/Fixes-20240423-131752.yaml b/.changes/unreleased/Fixes-20240423-131752.yaml
deleted file mode 100644
index 048c23d8a..000000000
--- a/.changes/unreleased/Fixes-20240423-131752.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Stop adding aliases to render_limited output
-time: 2024-04-23T13:17:52.059553-07:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "1190"
diff --git a/.changes/unreleased/Under the Hood-20240410-182121.yaml b/.changes/unreleased/Under the Hood-20240410-182121.yaml
deleted file mode 100644
index 34b794a51..000000000
--- a/.changes/unreleased/Under the Hood-20240410-182121.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Update dependabot config to cover GHA
-time: 2024-04-10T18:21:21.818397-04:00
-custom:
-  Author: mikealfare
-  Issue: "1176"
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 62c0bc82a..ade60b8f6 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,90 +5,6 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
-## dbt-bigquery 1.8.0-b2 - April 03, 2024
-
-### Features
-
-- Add new workflow for internal patch releases ([#38](https://github.com/dbt-labs/dbt-bigquery/issues/38))
-
-### Fixes
-
-- remove `keyfile` from `_connection_keys` ([#1146](https://github.com/dbt-labs/dbt-bigquery/issues/1146))
-- Add `pandas` extra for `google-cloud-bigquery` to pick up missing `pyarrow` dependency ([#1152](https://github.com/dbt-labs/dbt-bigquery/issues/1152))
-
-### Under the Hood
-
-- Add unit test for transaction semantics. ([#1123](https://github.com/dbt-labs/dbt-bigquery/issues/1123))
-
-### Dependencies
-
-- hard pin ddtrace to 2.3.0 ([#1141](https://github.com/dbt-labs/dbt-bigquery/pull/1141))
-- Add `dbt-core` as a dependency to preserve backwards compatibility for installation ([#1168](https://github.com/dbt-labs/dbt-bigquery/pull/1168))
-
-### Security
-
-- Pin `black>=24.3` in `dev-requirements.txt` ([#1151](https://github.com/dbt-labs/dbt-bigquery/pull/1151))
-
-
-
-## dbt-bigquery 1.8.0-b1 - March 01, 2024
-
-### Features
-
-- Add support for checking table-last-modified by metadata ([#938](https://github.com/dbt-labs/dbt-bigquery/issues/938))
-- Support limiting get_catalog by object name ([#950](https://github.com/dbt-labs/dbt-bigquery/issues/950))
-- Update base adapter references as part of decoupling migration ([#1067](https://github.com/dbt-labs/dbt-bigquery/issues/1067))
-- Support all types for unit testing in dbt-bigquery, expand coverage of safe_cast macro ([#1090](https://github.com/dbt-labs/dbt-bigquery/issues/1090))
-
-### Fixes
-
-- Patch for json inline --show ([#972](https://github.com/dbt-labs/dbt-bigquery/issues/972))
-- Lower bound of `2.11.0` for `google-api-core` ([#979](https://github.com/dbt-labs/dbt-bigquery/issues/979))
-- Fix refresh syntax, config comparison with empty labels ([#983](https://github.com/dbt-labs/dbt-bigquery/issues/983))
-- Assign the correct relation type to materialized views in catalog queries ([#995](https://github.com/dbt-labs/dbt-bigquery/issues/995))
-- Fix inline comments (--) on the last line of an incremental model ([#896](https://github.com/dbt-labs/dbt-bigquery/issues/896))
-- In incremental models, add dummy merge condition on source partition column when partition is required ([#792](https://github.com/dbt-labs/dbt-bigquery/issues/792))
-- Support agate Integer type, test with empty seed ([#1003](https://github.com/dbt-labs/dbt-bigquery/issues/1003))
-- Fixed issue where materialized views were failing on re-run with minimal config parameters ([#1007](https://github.com/dbt-labs/dbt-bigquery/issues/1007))
-- Fix broken partition config granularity and batch_id being set to None ([#1006](https://github.com/dbt-labs/dbt-bigquery/issues/1006))
-- replace deterministic batch_id with uuid ([#1006](https://github.com/dbt-labs/dbt-bigquery/issues/1006))
-- remove json patch to leverage bigquery-python improvement ([#1055](https://github.com/dbt-labs/dbt-bigquery/issues/1055))
-- remove `token` field from connection keys ([#1105](https://github.com/dbt-labs/dbt-bigquery/issues/1105))
-- Remove custom query job async timeout logic as it has been fixed in bigquery-python ([#1081](https://github.com/dbt-labs/dbt-bigquery/issues/1081))
-
-### Under the Hood
-
-- Upgrade spark-bigquery Java deps for serverless to 2.13-0.34.0 ([#1006](https://github.com/dbt-labs/dbt-bigquery/issues/1006))
-- Primary and foreign key constraints are not enforced in BigQuery ([#1018](https://github.com/dbt-labs/dbt-bigquery/issues/1018))
-- Add tests for --empty flag ([#1029](https://github.com/dbt-labs/dbt-bigquery/issues/1029))
-- Migrate to dbt-common and dbt-adapters package ([#1071](https://github.com/dbt-labs/dbt-bigquery/issues/1071))
-
-### Dependencies
-
-- Update ddtrace requirement from ~=1.19 to ~=1.20 ([#948](https://github.com/dbt-labs/dbt-bigquery/pull/948))
-- Update pre-commit-hooks requirement from ~=4.4 to ~=4.5 ([#960](https://github.com/dbt-labs/dbt-bigquery/pull/960))
-- Bump mypy from 1.5.1 to 1.6.0 ([#963](https://github.com/dbt-labs/dbt-bigquery/pull/963))
-- Update pre-commit requirement from ~=3.4 to ~=3.5 ([#969](https://github.com/dbt-labs/dbt-bigquery/pull/969))
-- Update black requirement from ~=23.9 to ~=23.10 ([#973](https://github.com/dbt-labs/dbt-bigquery/pull/973))
-- Bump mypy from 1.6.0 to 1.6.1 ([#985](https://github.com/dbt-labs/dbt-bigquery/pull/985))
-- Update ddtrace requirement from ~=1.20 to ~=2.1 ([#989](https://github.com/dbt-labs/dbt-bigquery/pull/989))
-- Update black requirement from ~=23.10 to ~=23.11 ([#1013](https://github.com/dbt-labs/dbt-bigquery/pull/1013))
-- Update pytest-xdist requirement from ~=3.3 to ~=3.4 ([#1022](https://github.com/dbt-labs/dbt-bigquery/pull/1022))
-- Bump mypy from 1.6.1 to 1.7.0 ([#1023](https://github.com/dbt-labs/dbt-bigquery/pull/1023))
-- Update ddtrace requirement from ~=2.1 to ~=2.2 ([#1028](https://github.com/dbt-labs/dbt-bigquery/pull/1028))
-- Update wheel requirement from ~=0.41 to ~=0.42 ([#1033](https://github.com/dbt-labs/dbt-bigquery/pull/1033))
-- Bump mypy from 1.7.0 to 1.7.1 ([#1034](https://github.com/dbt-labs/dbt-bigquery/pull/1034))
-- Update ddtrace requirement from ~=2.2 to ~=2.3 ([#1035](https://github.com/dbt-labs/dbt-bigquery/pull/1035))
-- Update pytest-xdist requirement from ~=3.4 to ~=3.5 ([#1037](https://github.com/dbt-labs/dbt-bigquery/pull/1037))
-- Update freezegun requirement from ~=1.2 to ~=1.3 ([#1040](https://github.com/dbt-labs/dbt-bigquery/pull/1040))
-- Update black requirement from ~=23.11 to ~=23.12 ([#1056](https://github.com/dbt-labs/dbt-bigquery/pull/1056))
-- get dbt-tests-adapters from dbt-adapters repo ([#1077](https://github.com/dbt-labs/dbt-bigquery/pull/1077))
-
-### Contributors
-- [@gmyrianthous](https://github.com/gmyrianthous) ([#979](https://github.com/dbt-labs/dbt-bigquery/issues/979))
-- [@matt-winkler](https://github.com/matt-winkler) ([#972](https://github.com/dbt-labs/dbt-bigquery/issues/972))
-- [@tnk-ysk](https://github.com/tnk-ysk) ([#896](https://github.com/dbt-labs/dbt-bigquery/issues/896), [#792](https://github.com/dbt-labs/dbt-bigquery/issues/792))
-
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 - [1.6](https://github.com/dbt-labs/dbt-bigquery/blob/1.6.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 7d16c28f0..6698ed64c 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.8.0b2"
+version = "1.9.0a1"
diff --git a/setup.py b/setup.py
index c972e6e5a..caf668ed3 100644
--- a/setup.py
+++ b/setup.py
@@ -36,7 +36,7 @@ def _dbt_bigquery_version() -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.8.0b2"
+package_version = "1.9.0a1"
 description = """The BigQuery adapter plugin for dbt"""
 
 setup(

From 32fe41593662bcc66f927d0e2a331724a7fcd9bb Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Tue, 7 May 2024 08:09:04 -0700
Subject: [PATCH 796/860] Add missing changelog backlinks. (#1225)

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 CHANGELOG.md | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index ade60b8f6..4a408c580 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -7,6 +7,8 @@
 
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
+- [1.8](https://github.com/dbt-labs/dbt-bigquery/blob/1.8.latest/CHANGELOG.md)
+- [1.7](https://github.com/dbt-labs/dbt-bigquery/blob/1.7.latest/CHANGELOG.md)
 - [1.6](https://github.com/dbt-labs/dbt-bigquery/blob/1.6.latest/CHANGELOG.md)
 - [1.5](https://github.com/dbt-labs/dbt-bigquery/blob/1.5.latest/CHANGELOG.md)
 - [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)

From 82253130acf188a8b9e180a66f69cc7d47cfcb9b Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Tue, 7 May 2024 17:05:19 -0400
Subject: [PATCH 797/860] Add configuration options for `enable_list_inference`
 and `intermediate_format` for python models (#1205)

* add configuration options for enable_list_inference and intermediate_format for python models
* update test classes so that they run as part of the python model tests in tox
* reconfigure tests to force serial run instead of parallel
---
 .../unreleased/Features-20240426-105319.yaml  |   7 +
 .../unreleased/Fixes-20240426-105224.yaml     |   7 +
 dbt/adapters/bigquery/impl.py                 |   2 +
 .../macros/materializations/table.sql         |   7 +
 .../functional/python_model_tests/__init__.py |   0
 tests/functional/python_model_tests/files.py  | 125 ++++++++++++++++++
 .../python_model_tests/test_list_inference.py |  36 +++++
 7 files changed, 184 insertions(+)
 create mode 100644 .changes/unreleased/Features-20240426-105319.yaml
 create mode 100644 .changes/unreleased/Fixes-20240426-105224.yaml
 create mode 100644 tests/functional/python_model_tests/__init__.py
 create mode 100644 tests/functional/python_model_tests/files.py
 create mode 100644 tests/functional/python_model_tests/test_list_inference.py

diff --git a/.changes/unreleased/Features-20240426-105319.yaml b/.changes/unreleased/Features-20240426-105319.yaml
new file mode 100644
index 000000000..0af2f9aa8
--- /dev/null
+++ b/.changes/unreleased/Features-20240426-105319.yaml
@@ -0,0 +1,7 @@
+kind: Features
+body: Add configuration options `enable_list_inference` and `intermediate_format` for python
+  models
+time: 2024-04-26T10:53:19.874239-04:00
+custom:
+  Author: mikealfare
+  Issue: 1047 1114
diff --git a/.changes/unreleased/Fixes-20240426-105224.yaml b/.changes/unreleased/Fixes-20240426-105224.yaml
new file mode 100644
index 000000000..624006ba5
--- /dev/null
+++ b/.changes/unreleased/Fixes-20240426-105224.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: Default `enableListInference` to `True` for python models to support nested
+  lists
+time: 2024-04-26T10:52:24.827314-04:00
+custom:
+  Author: mikealfare
+  Issue: 1047 1114
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 2df35bc65..700c03719 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -99,6 +99,8 @@ class BigqueryConfig(AdapterConfig):
     enable_refresh: Optional[bool] = None
     refresh_interval_minutes: Optional[int] = None
     max_staleness: Optional[str] = None
+    enable_list_inference: Optional[bool] = None
+    intermediate_format: Optional[str] = None
 
 
 class BigQueryAdapter(BaseAdapter):
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index 68117b06a..e3c5b3598 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -49,12 +49,19 @@
 from pyspark.sql import SparkSession
 {%- set raw_partition_by = config.get('partition_by', none) -%}
 {%- set raw_cluster_by = config.get('cluster_by', none) -%}
+{%- set enable_list_inference = config.get('enable_list_inference', true) -%}
+{%- set intermediate_format = config.get('intermediate_format', none) -%}
+
 {%- set partition_config = adapter.parse_partition_by(raw_partition_by) %}
 
 spark = SparkSession.builder.appName('smallTest').getOrCreate()
 
 spark.conf.set("viewsEnabled","true")
 spark.conf.set("temporaryGcsBucket","{{target.gcs_bucket}}")
+spark.conf.set("enableListInference", "{{ enable_list_inference }}")
+{% if intermediate_format %}
+spark.conf.set("intermediateFormat", "{{ intermediate_format }}")
+{% endif %}
 
 {{ compiled_code }}
 dbt = dbtObj(spark.read.format("bigquery").load)
diff --git a/tests/functional/python_model_tests/__init__.py b/tests/functional/python_model_tests/__init__.py
new file mode 100644
index 000000000..e69de29bb
diff --git a/tests/functional/python_model_tests/files.py b/tests/functional/python_model_tests/files.py
new file mode 100644
index 000000000..1cb95602a
--- /dev/null
+++ b/tests/functional/python_model_tests/files.py
@@ -0,0 +1,125 @@
+SINGLE_RECORD = """
+import pandas as pd
+
+def model(dbt, session):
+
+    dbt.config(
+        submission_method="serverless",
+        materialized="table"
+    )
+
+    df = pd.DataFrame(
+        [
+            {"column_name": {"name": "hello", "my_list": ["h", "e", "l", "l", "o"]}},
+        ]
+    )
+
+    return df
+"""
+
+
+MULTI_RECORD = """
+import pandas as pd
+
+def model(dbt, session):
+
+    dbt.config(
+        submission_method="serverless",
+        materialized="table",
+    )
+
+    df = pd.DataFrame(
+        [
+            {"column_name": [{"name": "hello", "my_list": ["h", "e", "l", "l", "o"]}]},
+        ]
+    )
+
+    return df
+"""
+
+
+ORC_FORMAT = """
+import pandas as pd
+
+def model(dbt, session):
+
+    dbt.config(
+        submission_method="serverless",
+        materialized="table",
+        intermediate_format="orc",
+    )
+
+    df = pd.DataFrame(
+        [
+            {"column_name": [{"name": "hello", "my_list": ["h", "e", "l", "l", "o"]}]},
+        ]
+    )
+
+    return df
+"""
+
+
+ENABLE_LIST_INFERENCE = """
+import pandas as pd
+
+def model(dbt, session):
+
+    dbt.config(
+        submission_method="serverless",
+        materialized="table",
+        enable_list_inference="true",
+    )
+
+    df = pd.DataFrame(
+        [
+            {"column_name": [{"name": "hello", "my_list": ["h", "e", "l", "l", "o"]}]},
+        ]
+    )
+
+    return df
+"""
+
+
+ENABLE_LIST_INFERENCE_PARQUET_FORMAT = """
+import pandas as pd
+
+def model(dbt, session):
+
+    dbt.config(
+        submission_method="serverless",
+        materialized="table",
+        enable_list_inference="true",
+        intermediate_format="parquet",
+    )
+
+    df = pd.DataFrame(
+        [
+            {"column_name": [{"name": "hello", "my_list": ["h", "e", "l", "l", "o"]}]},
+        ]
+    )
+
+    return df
+"""
+
+
+DISABLE_LIST_INFERENCE_ORC_FORMAT = """
+import pandas as pd
+
+def model(dbt, session):
+
+    dbt.config(
+        submission_method="serverless",
+        materialized="table",
+        enable_list_inference="false",
+        intermediate_format="orc",
+    )
+
+    df = pd.DataFrame(
+        [
+            {"column_name": [{"name": "hello", "my_list": ["h", "e", "l", "l", "o"]}]},
+        ]
+    )
+
+    return df
+
+"""
diff --git a/tests/functional/python_model_tests/test_list_inference.py b/tests/functional/python_model_tests/test_list_inference.py
new file mode 100644
index 000000000..581e2010b
--- /dev/null
+++ b/tests/functional/python_model_tests/test_list_inference.py
@@ -0,0 +1,36 @@
+"""
+This test case addresses this regression: https://github.com/dbt-labs/dbt-bigquery/issues/1047
+
+As the comments point out, the issue appears when the settings are:
+    - list inference: off
+    - intermediate format: parquet
+
+Adjusting either of these alleviates the issue.
+
+When the regression was first reported, `files.MULTI_RECORD` failed while the other models passed.
+"""
+from dbt.tests.util import run_dbt_and_capture
+import pytest
+
+from tests.functional.python_model_tests import files
+
+
+class TestListInference:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            # this is what worked prior to this issue
+            "single_record.py": files.SINGLE_RECORD,
+            # this is the model that initially failed for this issue
+            "multi_record.py": files.MULTI_RECORD,
+            # these are explicit versions of the default settings
+            "enable_list_inference.py": files.ENABLE_LIST_INFERENCE,
+            "enable_list_inference_parquet_format.py": files.ENABLE_LIST_INFERENCE_PARQUET_FORMAT,
+            # orc format also resolves the issue, regardless of list inference
+            "orc_format.py": files.ORC_FORMAT,
+            "disable_list_inference_orc_format.py": files.DISABLE_LIST_INFERENCE_ORC_FORMAT,
+        }
+
+    def test_models_success(self, project, models):
+        result, output = run_dbt_and_capture(["run"])
+        assert len(result) == len(models)

From fcc4735ea85f7021ce4454f0cf0c9414c2d6e8b4 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 8 May 2024 10:36:51 -0400
Subject: [PATCH 798/860] Update CODEOWNERS (#1226)

---
 .github/CODEOWNERS | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS
index f6283d123..02ed72d45 100644
--- a/.github/CODEOWNERS
+++ b/.github/CODEOWNERS
@@ -1,3 +1,3 @@
 # This codeowners file is used to ensure all PRs require reviews from the adapters team
 
-* @dbt-labs/core-adapters
+* @dbt-labs/adapters

From edf3344eee40e922ba030f4b156d2cb206252880 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Wed, 8 May 2024 15:31:41 -0500
Subject: [PATCH 799/860] update TestListInference to TestPythonListInference
 to move it to python test run (#1229)

---
 tests/functional/python_model_tests/test_list_inference.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/tests/functional/python_model_tests/test_list_inference.py b/tests/functional/python_model_tests/test_list_inference.py
index 581e2010b..143a61e88 100644
--- a/tests/functional/python_model_tests/test_list_inference.py
+++ b/tests/functional/python_model_tests/test_list_inference.py
@@ -15,7 +15,7 @@
 from tests.functional.python_model_tests import files
 
 
-class TestListInference:
+class TestPythonListInference:
     @pytest.fixture(scope="class")
     def models(self):
         return {

From 0d37ba6032d1c1d4d1661ebe53af5469ba9e6bd9 Mon Sep 17 00:00:00 2001
From: salimmoulouel <36620917+salimmoulouel@users.noreply.github.com>
Date: Fri, 10 May 2024 16:29:33 +0200
Subject: [PATCH 800/860] implement choice of delimiter for seed files (#1122)

* implement choice of delimiter for seed files

* adding change log entry

* implementation of test of TestBigQuerySeedWithUniqueDelimiter

* Update dbt/adapters/bigquery/impl.py

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>

* Update dbt/include/bigquery/macros/materializations/seed.sql

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>

* Update dbt/include/bigquery/macros/materializations/seed.sql

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>

* Update dbt/adapters/bigquery/impl.py

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>

* Update .changes/unreleased/Fixes-20240226-233024.yaml

---------

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Co-authored-by: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .../unreleased/Fixes-20240226-233024.yaml     |  6 ++++
 dbt/adapters/bigquery/impl.py                 |  6 ++--
 .../bigquery/macros/materializations/seed.sql |  2 +-
 tests/functional/adapter/test_simple_seed.py  | 36 ++++++++++++++++++-
 4 files changed, 46 insertions(+), 4 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20240226-233024.yaml

diff --git a/.changes/unreleased/Fixes-20240226-233024.yaml b/.changes/unreleased/Fixes-20240226-233024.yaml
new file mode 100644
index 000000000..efb1b077c
--- /dev/null
+++ b/.changes/unreleased/Fixes-20240226-233024.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: make seed delimiter configurable via `field_delimeter` in model config
+time: 2024-02-26T23:30:24.141213+01:00
+custom:
+  Author: salimmoulouel
+  Issue: "1119"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 700c03719..36222f44f 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -654,7 +654,9 @@ def alter_table_add_columns(self, relation, columns):
         client.update_table(new_table, ["schema"])
 
     @available.parse_none
-    def load_dataframe(self, database, schema, table_name, agate_table, column_override):
+    def load_dataframe(
+        self, database, schema, table_name, agate_table, column_override, field_delimiter
+    ):
         bq_schema = self._agate_to_schema(agate_table, column_override)
         conn = self.connections.get_thread_connection()
         client = conn.handle
@@ -664,7 +666,7 @@ def load_dataframe(self, database, schema, table_name, agate_table, column_overr
         load_config = google.cloud.bigquery.LoadJobConfig()
         load_config.skip_leading_rows = 1
         load_config.schema = bq_schema
-
+        load_config.field_delimiter = field_delimiter
         with open(agate_table.original_abspath, "rb") as f:
             job = client.load_table_from_file(f, table_ref, rewind=True, job_config=load_config)
 
diff --git a/dbt/include/bigquery/macros/materializations/seed.sql b/dbt/include/bigquery/macros/materializations/seed.sql
index 6ac7337f3..c89d00598 100644
--- a/dbt/include/bigquery/macros/materializations/seed.sql
+++ b/dbt/include/bigquery/macros/materializations/seed.sql
@@ -11,7 +11,7 @@
 
   {%- set column_override = model['config'].get('column_types', {}) -%}
   {{ adapter.load_dataframe(model['database'], model['schema'], model['alias'],
-  							agate_table, column_override) }}
+  							agate_table, column_override, model['config']['delimiter']) }}
 
   {% call statement() %}
     alter table {{ this.render() }} set {{ bigquery_table_options(config, model) }}
diff --git a/tests/functional/adapter/test_simple_seed.py b/tests/functional/adapter/test_simple_seed.py
index b01f99346..5ec19d420 100644
--- a/tests/functional/adapter/test_simple_seed.py
+++ b/tests/functional/adapter/test_simple_seed.py
@@ -5,7 +5,6 @@
 from dbt.tests.adapter.simple_seed.test_seed import BaseTestEmptySeed
 from dbt.tests.adapter.utils.base_utils import run_dbt
 
-
 _SEED_CONFIGS_CSV = """
 seed_id,stuff
 1,a
@@ -156,3 +155,38 @@ def test__bigquery_seed_table_with_labels_config_bigquery(self, project):
 
 class TestBigQueryEmptySeed(BaseTestEmptySeed):
     pass
+
+
+class TestBigQuerySeedWithUniqueDelimiter(TestSimpleSeedConfigs):
+    @pytest.fixture(scope="class")
+    def seeds(self):
+        return {
+            "seed_enabled.csv": seeds__enabled_in_config_csv.replace(",", "|"),
+            "seed_tricky.csv": seeds__tricky_csv.replace(",", "\t"),
+            "seed_configs.csv": _SEED_CONFIGS_CSV,
+        }
+
+    @pytest.fixture(scope="class")
+    def project_config_update(self):
+        return {
+            "config-version": 2,
+            "seeds": {
+                "test": {
+                    "enabled": False,
+                    "quote_columns": True,
+                    "seed_enabled": {
+                        "enabled": True,
+                        "+column_types": self.seed_enabled_types(),
+                        "delimiter": "|",
+                    },
+                    "seed_tricky": {
+                        "enabled": True,
+                        "+column_types": self.seed_tricky_types(),
+                        "delimiter": "\t",
+                    },
+                    "seed_configs": {
+                        "enabled": True,
+                    },
+                },
+            },
+        }

From d148cc391f5806f2ed76ae9d9a880ef76ff71066 Mon Sep 17 00:00:00 2001
From: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Mon, 13 May 2024 14:00:34 -0700
Subject: [PATCH 801/860] remove dependency on dbt-core legacy logger (#1233)

* use dynamic schema in test_grant_access_to.py
* revert setup
* remove broken import
---
 tests/unit/test_bigquery_adapter.py            | 1 -
 tests/unit/test_bigquery_connection_manager.py | 1 -
 2 files changed, 2 deletions(-)

diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 248e8e2be..19d9dbd08 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -23,7 +23,6 @@
 from dbt.context.query_header import generate_query_header_context
 from dbt.contracts.files import FileHash
 from dbt.contracts.graph.manifest import ManifestStateCheck
-from dbt.logger import GLOBAL_LOGGER as logger  # noqa
 from dbt.context.providers import RuntimeConfigObject, generate_runtime_macro_context
 
 from google.cloud.bigquery import AccessEntry
diff --git a/tests/unit/test_bigquery_connection_manager.py b/tests/unit/test_bigquery_connection_manager.py
index 564601b2f..9dc8fe219 100644
--- a/tests/unit/test_bigquery_connection_manager.py
+++ b/tests/unit/test_bigquery_connection_manager.py
@@ -9,7 +9,6 @@
 from dbt.adapters.bigquery import BigQueryCredentials
 from dbt.adapters.bigquery import BigQueryRelation
 from dbt.adapters.bigquery.connections import BigQueryConnectionManager
-from dbt.logger import GLOBAL_LOGGER as logger  # noqa
 
 
 class TestBigQueryConnectionManager(unittest.TestCase):

From b7bb0bc0818644617b8c10eb0723560ce98147a5 Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Mon, 13 May 2024 17:24:29 -0600
Subject: [PATCH 802/860] Cross-database `date` macro (#1219)

* Cross-database `date` macro

* Temporary dev requirements for testing

* Update changelog entry

* Revert "Temporary dev requirements for testing"

This reverts commit b46183aea28c684592b68e6dae7d2396b0fe02bc.
---
 .changes/unreleased/Features-20240501-151902.yaml | 6 ++++++
 dbt/include/bigquery/macros/utils/date.sql        | 3 +++
 tests/functional/adapter/utils/test_utils.py      | 5 +++++
 3 files changed, 14 insertions(+)
 create mode 100644 .changes/unreleased/Features-20240501-151902.yaml
 create mode 100644 dbt/include/bigquery/macros/utils/date.sql

diff --git a/.changes/unreleased/Features-20240501-151902.yaml b/.changes/unreleased/Features-20240501-151902.yaml
new file mode 100644
index 000000000..1522e9775
--- /dev/null
+++ b/.changes/unreleased/Features-20240501-151902.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Cross-database `date` macro
+time: 2024-05-01T15:19:02.929676-06:00
+custom:
+  Author: dbeatty10
+  Issue: 1221
diff --git a/dbt/include/bigquery/macros/utils/date.sql b/dbt/include/bigquery/macros/utils/date.sql
new file mode 100644
index 000000000..0f3b85aca
--- /dev/null
+++ b/dbt/include/bigquery/macros/utils/date.sql
@@ -0,0 +1,3 @@
+{% macro bigquery__date(year, month, day) -%}
+    date({{ year }}, {{ month }}, {{ day }})
+{%- endmacro %}
diff --git a/tests/functional/adapter/utils/test_utils.py b/tests/functional/adapter/utils/test_utils.py
index 58cecdc7e..7ad4bdd6e 100644
--- a/tests/functional/adapter/utils/test_utils.py
+++ b/tests/functional/adapter/utils/test_utils.py
@@ -11,6 +11,7 @@
 from dbt.tests.adapter.utils.test_cast_bool_to_text import BaseCastBoolToText
 from dbt.tests.adapter.utils.test_concat import BaseConcat
 from dbt.tests.adapter.utils.test_current_timestamp import BaseCurrentTimestampAware
+from dbt.tests.adapter.utils.test_date import BaseDate
 from dbt.tests.adapter.utils.test_dateadd import BaseDateAdd
 from dbt.tests.adapter.utils.test_datediff import BaseDateDiff
 from dbt.tests.adapter.utils.test_date_spine import BaseDateSpine
@@ -117,6 +118,10 @@ class TestCurrentTimestamp(BaseCurrentTimestampAware):
     pass
 
 
+class TestDate(BaseDate):
+    pass
+
+
 class TestDateAdd(BaseDateAdd):
     pass
 

From 93268ba7ad1af15fa4136225e2d06fe4d304c8cb Mon Sep 17 00:00:00 2001
From: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Date: Tue, 14 May 2024 13:03:44 -0600
Subject: [PATCH 803/860] Import relevant pytest(s) for cross-database `cast`
 macro (#1215)

Co-authored-by: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .changes/unreleased/Features-20240430-185650.yaml | 6 ++++++
 tests/functional/adapter/utils/test_utils.py      | 5 +++++
 2 files changed, 11 insertions(+)
 create mode 100644 .changes/unreleased/Features-20240430-185650.yaml

diff --git a/.changes/unreleased/Features-20240430-185650.yaml b/.changes/unreleased/Features-20240430-185650.yaml
new file mode 100644
index 000000000..0c0eef567
--- /dev/null
+++ b/.changes/unreleased/Features-20240430-185650.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Add tests for cross-database `cast` macro
+time: 2024-04-30T18:56:50.238027-06:00
+custom:
+  Author: dbeatty10
+  Issue: "1214"
diff --git a/tests/functional/adapter/utils/test_utils.py b/tests/functional/adapter/utils/test_utils.py
index 7ad4bdd6e..384b17108 100644
--- a/tests/functional/adapter/utils/test_utils.py
+++ b/tests/functional/adapter/utils/test_utils.py
@@ -8,6 +8,7 @@
 from dbt.tests.adapter.utils.test_array_construct import BaseArrayConstruct
 from dbt.tests.adapter.utils.test_any_value import BaseAnyValue
 from dbt.tests.adapter.utils.test_bool_or import BaseBoolOr
+from dbt.tests.adapter.utils.test_cast import BaseCast
 from dbt.tests.adapter.utils.test_cast_bool_to_text import BaseCastBoolToText
 from dbt.tests.adapter.utils.test_concat import BaseConcat
 from dbt.tests.adapter.utils.test_current_timestamp import BaseCurrentTimestampAware
@@ -105,6 +106,10 @@ class TestBoolOr(BaseBoolOr):
     pass
 
 
+class TestCast(BaseCast):
+    pass
+
+
 class TestCastBoolToText(BaseCastBoolToText):
     pass
 

From f08d82c9748a22473aad6b5ce886e075d64ac024 Mon Sep 17 00:00:00 2001
From: Jeremy Cohen <jeremy@dbtlabs.com>
Date: Tue, 14 May 2024 23:52:32 +0200
Subject: [PATCH 804/860] Bump deps on common, adapters, core (#1235)

Co-authored-by: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 setup.py | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/setup.py b/setup.py
index caf668ed3..1983713aa 100644
--- a/setup.py
+++ b/setup.py
@@ -51,8 +51,8 @@ def _dbt_bigquery_version() -> str:
     packages=find_namespace_packages(include=["dbt", "dbt.*"]),
     include_package_data=True,
     install_requires=[
-        "dbt-common>=0.1.0a1,<2.0",
-        "dbt-adapters>=0.1.0a1,<2.0",
+        "dbt-common>=1.0.4,<2.0",
+        "dbt-adapters>=1.1.1,<2.0",
         # 3.20 introduced pyarrow>=3.0 under the `pandas` extra
         "google-cloud-bigquery[pandas]>=3.0,<4.0",
         "google-cloud-storage~=2.4",
@@ -61,7 +61,7 @@ def _dbt_bigquery_version() -> str:
         # Expect compatibility with all new versions of these packages, so lower bounds only.
         "google-api-core>=2.11.0",
         # add dbt-core to ensure backwards compatibility of installation, this is not a functional dependency
-        "dbt-core>=1.8.0a1",
+        "dbt-core>=1.8.0",
     ],
     zip_safe=False,
     classifiers=[

From dd300f9c5d13dce9628ad6cacc8403118c44fd01 Mon Sep 17 00:00:00 2001
From: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Wed, 15 May 2024 15:35:02 -0700
Subject: [PATCH 805/860] fix Core version bump in setup.py during release
 (#1241)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* remove duplicate package_version and avoid bumping dbt-core version
---
 .bumpversion.cfg | 2 --
 setup.py         | 1 -
 2 files changed, 3 deletions(-)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 3f0db0785..24e904ac0 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -32,6 +32,4 @@ first_value = 1
 
 [bumpversion:part:nightly]
 
-[bumpversion:file:setup.py]
-
 [bumpversion:file:dbt/adapters/bigquery/__version__.py]
diff --git a/setup.py b/setup.py
index 1983713aa..3f4bad228 100644
--- a/setup.py
+++ b/setup.py
@@ -36,7 +36,6 @@ def _dbt_bigquery_version() -> str:
 
 
 package_name = "dbt-bigquery"
-package_version = "1.9.0a1"
 description = """The BigQuery adapter plugin for dbt"""
 
 setup(

From 32371732bfe69e2aef56d470363f6952a3c425d3 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Mon, 20 May 2024 19:01:29 -0400
Subject: [PATCH 806/860]  Add docker release to the full release process for
 final releases (#1170)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

* add docker release to release pipeline
* add docker to dependabot
* update Dockerfile to match new release pipeline
* update release pipeline to match new release pipeline
* update readme to match the new container
* update make recipe
* update dev docker image
* add permissions for nightly release
* don’t release docker when testing, allow to release only docker
* Update docker/Dockerfile
* remove unused test script

---------

Co-authored-by: Emily Rockman <emily.rockman@dbtlabs.com>
---
 .github/dependabot.yml                |  5 +++
 .github/workflows/nightly-release.yml |  1 +
 .github/workflows/release.yml         | 54 +++++++++++++------------
 Makefile                              | 20 ++++-----
 docker/Dockerfile                     | 37 +++++++++++++++++
 docker/README.md                      | 58 +++++++++++++++++++++++++++
 docker/dev.Dockerfile                 | 50 +++++++++++++++++++++++
 docker_dev/README.md                  |  5 ---
 docker_dev/ubuntu.Dockerfile          | 50 -----------------------
 9 files changed, 186 insertions(+), 94 deletions(-)
 create mode 100644 docker/Dockerfile
 create mode 100644 docker/README.md
 create mode 100644 docker/dev.Dockerfile
 delete mode 100644 docker_dev/README.md
 delete mode 100644 docker_dev/ubuntu.Dockerfile

diff --git a/.github/dependabot.yml b/.github/dependabot.yml
index 4673f47cf..ae2be43aa 100644
--- a/.github/dependabot.yml
+++ b/.github/dependabot.yml
@@ -10,3 +10,8 @@ updates:
     schedule:
       interval: "weekly"
     rebase-strategy: "disabled"
+  - package-ecosystem: "docker"
+    directory: "/docker"
+    schedule:
+      interval: "weekly"
+    rebase-strategy: "disabled"
diff --git a/.github/workflows/nightly-release.yml b/.github/workflows/nightly-release.yml
index 517015854..1dc396154 100644
--- a/.github/workflows/nightly-release.yml
+++ b/.github/workflows/nightly-release.yml
@@ -20,6 +20,7 @@ on:
 
 permissions:
   contents: write # this is the permission that allows creating a new release
+  packages: write # allows creating a Docker release as a GitHub package on GHCR
 
 defaults:
   run:
diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml
index 88942e251..ad7cf76b4 100644
--- a/.github/workflows/release.yml
+++ b/.github/workflows/release.yml
@@ -13,7 +13,8 @@
 #
 # **when?**
 # This workflow can be run manually on demand or can be called by other workflows
-name: Release to GitHub and PyPI
+name: "Release to GitHub, PyPI, and Docker"
+run-name: "Release ${{ inputs.version_number }} to GitHub, PyPI, and Docker"
 
 on:
   workflow_dispatch:
@@ -60,6 +61,11 @@ on:
         type: boolean
         default: false
         required: false
+      only_docker:
+        description: "Only release Docker image, skip GitHub & PyPI"
+        type: boolean
+        default: false
+        required: false
   workflow_call:
     inputs:
       sha:
@@ -128,12 +134,11 @@ jobs:
           echo Package test command:               ${{ inputs.package_test_command }}
           echo Test run:                           ${{ inputs.test_run }}
           echo Nightly release:                    ${{ inputs.nightly_release }}
+          echo Only Docker:                        ${{ inputs.only_docker }}
 
   bump-version-generate-changelog:
     name: Bump package version, Generate changelog
-
     uses: dbt-labs/dbt-release/.github/workflows/release-prep.yml@main
-
     with:
       sha: ${{ inputs.sha }}
       version_number: ${{ inputs.version_number }}
@@ -141,17 +146,13 @@ jobs:
       env_setup_script_path: ${{ inputs.env_setup_script_path }}
       test_run: ${{ inputs.test_run }}
       nightly_release: ${{ inputs.nightly_release }}
-
     secrets: inherit
 
   log-outputs-bump-version-generate-changelog:
     name: "[Log output] Bump package version, Generate changelog"
-    if: ${{ !failure() && !cancelled() }}
-
+    if: ${{ !failure() && !cancelled() && !inputs.only_docker }}
     needs: [bump-version-generate-changelog]
-
     runs-on: ubuntu-latest
-
     steps:
       - name: Print variables
         run: |
@@ -160,11 +161,9 @@ jobs:
 
   build-test-package:
     name: Build, Test, Package
-    if: ${{ !failure() && !cancelled() }}
+    if: ${{ !failure() && !cancelled() && !inputs.only_docker }}
     needs: [bump-version-generate-changelog]
-
     uses: dbt-labs/dbt-release/.github/workflows/build.yml@main
-
     with:
       sha: ${{ needs.bump-version-generate-changelog.outputs.final_sha }}
       version_number: ${{ inputs.version_number }}
@@ -174,19 +173,15 @@ jobs:
       package_test_command: ${{ inputs.package_test_command }}
       test_run: ${{ inputs.test_run }}
       nightly_release: ${{ inputs.nightly_release }}
-
     secrets:
       AWS_ACCESS_KEY_ID: ${{ secrets.AWS_ACCESS_KEY_ID }}
       AWS_SECRET_ACCESS_KEY: ${{ secrets.AWS_SECRET_ACCESS_KEY }}
 
   github-release:
     name: GitHub Release
-    if: ${{ !failure() && !cancelled() }}
-
+    if: ${{ !failure() && !cancelled() && !inputs.only_docker }}
     needs: [bump-version-generate-changelog, build-test-package]
-
     uses: dbt-labs/dbt-release/.github/workflows/github-release.yml@main
-
     with:
       sha: ${{ needs.bump-version-generate-changelog.outputs.final_sha }}
       version_number: ${{ inputs.version_number }}
@@ -195,34 +190,41 @@ jobs:
 
   pypi-release:
     name: PyPI Release
-
-    needs: [github-release]
-
+    if: ${{ !failure() && !cancelled() && !inputs.only_docker }}
+    needs: [bump-version-generate-changelog, build-test-package]
     uses: dbt-labs/dbt-release/.github/workflows/pypi-release.yml@main
-
     with:
       version_number: ${{ inputs.version_number }}
       test_run: ${{ inputs.test_run }}
-
     secrets:
       PYPI_API_TOKEN: ${{ secrets.PYPI_API_TOKEN }}
       TEST_PYPI_API_TOKEN: ${{ secrets.TEST_PYPI_API_TOKEN }}
 
+  docker-release:
+    name: "Docker Release"
+    # We cannot release to docker on a test run because it uses the tag in GitHub as
+    # what we need to release but draft releases don't actually tag the commit so it
+    # finds nothing to release
+    if: ${{ !failure() && !cancelled() && (!inputs.test_run || inputs.only_docker) }}
+    needs: [bump-version-generate-changelog, build-test-package, github-release]
+    permissions:
+      packages: write
+    uses: dbt-labs/dbt-release/.github/workflows/release-docker.yml@main
+    with:
+      version_number: ${{ inputs.version_number }}
+      test_run: ${{ inputs.test_run }}
+
   slack-notification:
     name: Slack Notification
     if: ${{ failure() && (!inputs.test_run || inputs.nightly_release) }}
-
     needs:
       [
-        bump-version-generate-changelog,
-        build-test-package,
         github-release,
         pypi-release,
+        docker-release,
       ]
-
     uses: dbt-labs/dbt-release/.github/workflows/slack-post-notification.yml@main
     with:
       status: "failure"
-
     secrets:
       SLACK_WEBHOOK_URL: ${{ secrets.SLACK_DEV_ADAPTER_ALERTS }}
diff --git a/Makefile b/Makefile
index fc6536f98..bdacb538b 100644
--- a/Makefile
+++ b/Makefile
@@ -11,17 +11,11 @@ dev-uninstall: ## Uninstalls all packages while maintaining the virtual environm
 	pip freeze | grep -v "^-e" | cut -d "@" -f1 | xargs pip uninstall -y
 	pip uninstall -y dbt-bigquery
 
-.PHONY: ubuntu-py311
-ubuntu-py311: ## Builds and runs an Ubuntu Python 3.11 development container
-	docker build -f docker_dev/ubuntu.Dockerfile -t dbt-bigquery-ubuntu-py311 .
-	docker run --rm -it --name dbt-bigquery-ubuntu-py311 -v $(shell pwd):/opt/code dbt-bigquery-ubuntu-py311
+.PHONY: docker-dev
+docker-dev:
+	docker build -f docker/dev.Dockerfile -t dbt-bigquery-dev .
+	docker run --rm -it --name dbt-bigquery-dev -v $(shell pwd):/opt/code dbt-bigquery-dev
 
-.PHONY: ubuntu-py39
-ubuntu-py39: ## Builds and runs an Ubuntu Python 3.9 development container
-	docker build -f docker_dev/ubuntu.Dockerfile -t dbt-bigquery-ubuntu-py39 . --build-arg version=3.9
-	docker run --rm -it --name dbt-bigquery-ubuntu-py39 -v $(shell pwd):/opt/code dbt-bigquery-ubuntu-py39
-
-.PHONY: ubuntu-py38
-ubuntu-py38: ## Builds and runs an Ubuntu Python 3.8 development container
-	docker build -f docker_dev/ubuntu.Dockerfile -t dbt-bigquery-ubuntu-py38 . --build-arg version=3.8
-	docker run --rm -it --name dbt-bigquery-ubuntu-py38 -v $(shell pwd):/opt/code dbt-bigquery-ubuntu-py38
+.PHONY: docker-prod
+docker-prod:
+	docker build -f docker/Dockerfile -t dbt-bigquery .
diff --git a/docker/Dockerfile b/docker/Dockerfile
new file mode 100644
index 000000000..3b9431fd1
--- /dev/null
+++ b/docker/Dockerfile
@@ -0,0 +1,37 @@
+# this image gets published to GHCR for production use
+ARG py_version=3.11.2
+
+FROM python:$py_version-slim-bullseye as base
+
+RUN apt-get update \
+  && apt-get dist-upgrade -y \
+  && apt-get install -y --no-install-recommends \
+    build-essential=12.9 \
+    ca-certificates=20210119 \
+    git=1:2.30.2-1+deb11u2 \
+    libpq-dev=13.14-0+deb11u1 \
+    make=4.3-4.1 \
+    openssh-client=1:8.4p1-5+deb11u3 \
+    software-properties-common=0.96.20.2-2.1 \
+  && apt-get clean \
+  && rm -rf \
+    /var/lib/apt/lists/* \
+    /tmp/* \
+    /var/tmp/*
+
+ENV PYTHONIOENCODING=utf-8
+ENV LANG=C.UTF-8
+
+RUN python -m pip install --upgrade "pip==24.0" "setuptools==69.2.0" "wheel==0.43.0" --no-cache-dir
+
+
+FROM base as dbt-bigquery
+
+ARG commit_ref=main
+
+HEALTHCHECK CMD dbt --version || exit 1
+
+WORKDIR /usr/app/dbt/
+ENTRYPOINT ["dbt"]
+
+RUN python -m pip install --no-cache-dir "dbt-bigquery @ git+https://github.com/dbt-labs/dbt-bigquery@${commit_ref}"
diff --git a/docker/README.md b/docker/README.md
new file mode 100644
index 000000000..8c60deaa3
--- /dev/null
+++ b/docker/README.md
@@ -0,0 +1,58 @@
+# Docker for dbt
+This docker file is suitable for building dbt Docker images locally or using with CI/CD to automate populating a container registry.
+
+
+## Building an image:
+This Dockerfile can create images for the following target: `dbt-bigquery`
+
+In order to build a new image, run the following docker command.
+```shell
+docker build --tag <your_image_name> --target dbt-bigquery <path/to/dockerfile>
+```
+---
+> **Note:**  Docker must be configured to use [BuildKit](https://docs.docker.com/develop/develop-images/build_enhancements/) in order for images to build properly!
+
+---
+
+By default the image will be populated with the latest version of `dbt-bigquery` on `main`.
+If you need to use a different version you can specify it by git ref using the `--build-arg` flag:
+```shell
+docker build --tag <your_image_name> \
+  --target dbt-bigquery \
+  --build-arg commit_ref=<commit_ref> \
+  <path/to/dockerfile>
+```
+
+### Examples:
+To build an image named "my-dbt" that supports Snowflake using the latest releases:
+```shell
+cd dbt-core/docker
+docker build --tag my-dbt --target dbt-bigquery .
+```
+
+To build an image named "my-other-dbt" that supports Snowflake using the adapter version 1.0.0b1:
+```shell
+cd dbt-core/docker
+docker build \
+  --tag my-other-dbt \
+  --target dbt-bigquery \
+  --build-arg commit_ref=v1.0.0b1 \
+ .
+```
+
+## Running an image in a container:
+The `ENTRYPOINT` for this Dockerfile is the command `dbt` so you can bind-mount your project to `/usr/app` and use dbt as normal:
+```shell
+docker run \
+  --network=host \
+  --mount type=bind,source=path/to/project,target=/usr/app \
+  --mount type=bind,source=path/to/profiles.yml,target=/root/.dbt/profiles.yml \
+  my-dbt \
+  ls
+```
+---
+**Notes:**
+* Bind-mount sources _must_ be an absolute path
+* You may need to make adjustments to the docker networking setting depending on the specifics of your data warehouse/database host.
+
+---
diff --git a/docker/dev.Dockerfile b/docker/dev.Dockerfile
new file mode 100644
index 000000000..2ffef29c5
--- /dev/null
+++ b/docker/dev.Dockerfile
@@ -0,0 +1,50 @@
+# this image does not get published, it is intended for local development only, see `Makefile` for usage
+FROM ubuntu:22.04 as base
+
+# prevent python installation from asking for time zone region
+ARG DEBIAN_FRONTEND=noninteractive
+
+# add python repository
+RUN apt-get update \
+  && apt-get install -y software-properties-common=0.99.22.9 \
+  && add-apt-repository -y ppa:deadsnakes/ppa \
+  && apt-get clean \
+  && rm -rf \
+    /var/lib/apt/lists/* \
+    /tmp/* \
+    /var/tmp/*
+
+# install python
+RUN apt-get update \
+  && apt-get install -y --no-install-recommends \
+    build-essential=12.9ubuntu3 \
+    git-all=1:2.34.1-1ubuntu1.10 \
+    python3.8=3.8.19-1+jammy1 \
+    python3.8-dev=3.8.19-1+jammy1 \
+    python3.8-distutils=3.8.19-1+jammy1 \
+    python3.8-venv=3.8.19-1+jammy1 \
+    python3-pip=22.0.2+dfsg-1ubuntu0.4 \
+    python3-wheel=0.37.1-2ubuntu0.22.04.1 \
+  && apt-get clean \
+  && rm -rf \
+    /var/lib/apt/lists/* \
+    /tmp/* \
+    /var/tmp/*
+
+# update the default system interpreter to the newly installed version
+RUN update-alternatives --install /usr/bin/python3 python3 /usr/bin/python3.8 1
+
+
+FROM base as dbt-bigquery-dev
+
+HEALTHCHECK CMD python3 --version || exit 1
+
+# send stdout/stderr to terminal
+ENV PYTHONUNBUFFERED=1
+
+# setup mount for local code
+WORKDIR /opt/code
+VOLUME /opt/code
+
+# create a virtual environment
+RUN python3 -m venv /opt/venv
diff --git a/docker_dev/README.md b/docker_dev/README.md
deleted file mode 100644
index dd487fea7..000000000
--- a/docker_dev/README.md
+++ /dev/null
@@ -1,5 +0,0 @@
-# Docker Dev Images
-
-These images are solely for development purposes. They are
-saved here for convenience. There should be no expectation
-of stability or maintenance.
diff --git a/docker_dev/ubuntu.Dockerfile b/docker_dev/ubuntu.Dockerfile
deleted file mode 100644
index bac3f5993..000000000
--- a/docker_dev/ubuntu.Dockerfile
+++ /dev/null
@@ -1,50 +0,0 @@
-FROM ubuntu:latest
-
-# default to py3.11, this can be overridden at build, e.g. `docker build ... --build-arg version=3.10`
-ARG version=3.11
-
-# prevent python installation from asking for time zone region
-ARG DEBIAN_FRONTEND=noninteractive
-
-# get add-apt-repository
-RUN apt-get update && \
-    apt-get install -y software-properties-common
-
-# add the python repository
-RUN apt-get update && \
-    add-apt-repository -y ppa:deadsnakes/ppa
-
-# install python and git (for installing dbt-core)
-RUN apt-get update && \
-    apt-get install -y --no-install-recommends \
-        python$version \
-        python$version-dev \
-        python$version-distutils \
-        python$version-venv \
-        python3-pip \
-        python3-wheel \
-        build-essential \
-        git-all
-
-# clean up
-RUN apt-get clean && \
-    rm -rf \
-        /var/lib/apt/lists/* \
-        /tmp/* \
-        /var/tmp/*
-
-# update the default system interpreter to the newly installed version
-RUN update-alternatives --install /usr/bin/python3 python3 /usr/bin/python$version 1
-
-# setup mount for our code
-WORKDIR /opt/code
-VOLUME /opt/code
-
-# install tox in the system interpreter (it creates it's own virtual environments)
-RUN pip install tox
-
-# explicitly create a virtual environment as well for interactive testing
-RUN python3 -m venv /opt/venv
-
-# send stdout/stderr to terminal
-ENV PYTHONUNBUFFERED=1

From f454c47fc5cc1538f6b87d203b3fdc3b88b5242d Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Tue, 21 May 2024 01:00:32 -0400
Subject: [PATCH 807/860] Bump ubuntu from 22.04 to 24.04 in /docker (#1247)

* ---
updated-dependencies:
- dependency-name: ubuntu
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
---
 .changes/unreleased/Dependencies-20240520-230208.yaml | 6 ++++++
 docker/dev.Dockerfile                                 | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20240520-230208.yaml

diff --git a/.changes/unreleased/Dependencies-20240520-230208.yaml b/.changes/unreleased/Dependencies-20240520-230208.yaml
new file mode 100644
index 000000000..f89057233
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240520-230208.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump ubuntu from 22.04 to 24.04 in /docker"
+time: 2024-05-20T23:02:08.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1247
diff --git a/docker/dev.Dockerfile b/docker/dev.Dockerfile
index 2ffef29c5..2afad0a95 100644
--- a/docker/dev.Dockerfile
+++ b/docker/dev.Dockerfile
@@ -1,5 +1,5 @@
 # this image does not get published, it is intended for local development only, see `Makefile` for usage
-FROM ubuntu:22.04 as base
+FROM ubuntu:24.04 as base
 
 # prevent python installation from asking for time zone region
 ARG DEBIAN_FRONTEND=noninteractive

From 526eefa420006cf257c745ae9b89030d0fc88d00 Mon Sep 17 00:00:00 2001
From: Robele Baker <robele.baker@alumni.ubc.ca>
Date: Mon, 10 Jun 2024 11:42:06 -0700
Subject: [PATCH 808/860] Allow base64-service-account-json key auth Issue:
 #923 (#1245)

* added base64 functionality and basic testing

* Change log

* fix conftest to allow json

* Change method name from camel to snake case

* change type hinting to be py3.9 compatible

---------

Co-authored-by: Robele Baker <>
---
 .../unreleased/Features-20240516-125735.yaml  |  6 ++
 dbt/adapters/bigquery/connections.py          |  6 +-
 dbt/adapters/bigquery/utility.py              | 40 ++++++++-
 tests/conftest.py                             |  3 +
 tests/functional/adapter/test_json_keyfile.py | 82 +++++++++++++++++++
 5 files changed, 135 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Features-20240516-125735.yaml
 create mode 100644 tests/functional/adapter/test_json_keyfile.py

diff --git a/.changes/unreleased/Features-20240516-125735.yaml b/.changes/unreleased/Features-20240516-125735.yaml
new file mode 100644
index 000000000..d84b098b2
--- /dev/null
+++ b/.changes/unreleased/Features-20240516-125735.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Add support for base 64 encoded json keyfile credentials
+time: 2024-05-16T12:57:35.383416-07:00
+custom:
+    Author: robeleb1
+    Issue: "923"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index f96bc1381..4a3feae48 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -40,9 +40,11 @@
 from dbt.adapters.events.types import SQLQuery
 from dbt_common.events.functions import fire_event
 from dbt.adapters.bigquery import __version__ as dbt_version
+from dbt.adapters.bigquery.utility import is_base64, base64_to_string
 
 from dbt_common.dataclass_schema import ExtensibleDbtClassMixin, StrEnum
 
+
 logger = AdapterLogger("BigQuery")
 
 BQ_QUERY_JOB_SPLIT = "-----Query Job SQL Follows-----"
@@ -125,7 +127,7 @@ class BigQueryCredentials(Credentials):
     job_creation_timeout_seconds: Optional[int] = None
     job_execution_timeout_seconds: Optional[int] = None
 
-    # Keyfile json creds
+    # Keyfile json creds (unicode or base 64 encoded)
     keyfile: Optional[str] = None
     keyfile_json: Optional[Dict[str, Any]] = None
 
@@ -332,6 +334,8 @@ def get_google_credentials(cls, profile_credentials) -> GoogleCredentials:
 
         elif method == BigQueryConnectionMethod.SERVICE_ACCOUNT_JSON:
             details = profile_credentials.keyfile_json
+            if is_base64(profile_credentials.keyfile_json):
+                details = base64_to_string(details)
             return creds.from_service_account_info(details, scopes=profile_credentials.scopes)
 
         elif method == BigQueryConnectionMethod.OAUTH_SECRETS:
diff --git a/dbt/adapters/bigquery/utility.py b/dbt/adapters/bigquery/utility.py
index 5914280a3..557986b38 100644
--- a/dbt/adapters/bigquery/utility.py
+++ b/dbt/adapters/bigquery/utility.py
@@ -1,5 +1,7 @@
+import base64
+import binascii
 import json
-from typing import Any, Optional
+from typing import Any, Optional, Union
 
 import dbt_common.exceptions
 
@@ -43,3 +45,39 @@ def sql_escape(string):
     if not isinstance(string, str):
         raise dbt_common.exceptions.CompilationError(f"cannot escape a non-string: {string}")
     return json.dumps(string)[1:-1]
+
+
+def is_base64(s: Union[str, bytes]) -> bool:
+    """
+    Checks if the given string or bytes object is valid Base64 encoded.
+
+    Args:
+        s: The string or bytes object to check.
+
+    Returns:
+        True if the input is valid Base64, False otherwise.
+    """
+
+    if isinstance(s, str):
+        # For strings, ensure they consist only of valid Base64 characters
+        if not s.isascii():
+            return False
+        # Convert to bytes for decoding
+        s = s.encode("ascii")
+
+    try:
+        # Use the 'validate' parameter to enforce strict Base64 decoding rules
+        base64.b64decode(s, validate=True)
+        return True
+    except TypeError:
+        return False
+    except binascii.Error:  # Catch specific errors from the base64 module
+        return False
+
+
+def base64_to_string(b):
+    return base64.b64decode(b).decode("utf-8")
+
+
+def string_to_base64(s):
+    return base64.b64encode(s.encode("utf-8"))
diff --git a/tests/conftest.py b/tests/conftest.py
index 78f3d82e1..6dc9e6443 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -1,6 +1,7 @@
 import pytest
 import os
 import json
+from dbt.adapters.bigquery.utility import is_base64, base64_to_string
 
 # Import the fuctional fixtures as a plugin
 # Note: fixtures with session scope need to be local
@@ -38,6 +39,8 @@ def oauth_target():
 
 def service_account_target():
     credentials_json_str = os.getenv("BIGQUERY_TEST_SERVICE_ACCOUNT_JSON").replace("'", "")
+    if is_base64(credentials_json_str):
+        credentials_json_str = base64_to_string(credentials_json_str)
     credentials = json.loads(credentials_json_str)
     project_id = credentials.get("project_id")
     return {
diff --git a/tests/functional/adapter/test_json_keyfile.py b/tests/functional/adapter/test_json_keyfile.py
new file mode 100644
index 000000000..91e41a3f1
--- /dev/null
+++ b/tests/functional/adapter/test_json_keyfile.py
@@ -0,0 +1,82 @@
+import json
+import pytest
+from dbt.adapters.bigquery.utility import string_to_base64, is_base64
+
+
+@pytest.fixture
+def example_json_keyfile():
+    keyfile = json.dumps(
+        {
+            "type": "service_account",
+            "project_id": "",
+            "private_key_id": "",
+            "private_key": "-----BEGIN PRIVATE KEY----------END PRIVATE KEY-----\n",
+            "client_email": "",
+            "client_id": "",
+            "auth_uri": "https://accounts.google.com/o/oauth2/auth",
+            "token_uri": "https://oauth2.googleapis.com/token",
+            "auth_provider_x509_cert_url": "https://www.googleapis.com/oauth2/v1/certs",
+            "client_x509_cert_url": "",
+        }
+    )
+
+    return keyfile
+
+
+@pytest.fixture
+def example_json_keyfile_b64():
+    keyfile = json.dumps(
+        {
+            "type": "service_account",
+            "project_id": "",
+            "private_key_id": "",
+            "private_key": "-----BEGIN PRIVATE KEY----------END PRIVATE KEY-----\n",
+            "client_email": "",
+            "client_id": "",
+            "auth_uri": "https://accounts.google.com/o/oauth2/auth",
+            "token_uri": "https://oauth2.googleapis.com/token",
+            "auth_provider_x509_cert_url": "https://www.googleapis.com/oauth2/v1/certs",
+            "client_x509_cert_url": "",
+        }
+    )
+
+    return string_to_base64(keyfile)
+
+
+def test_valid_base64_strings(example_json_keyfile_b64):
+    valid_strings = [
+        "SGVsbG8gV29ybGQh",  # "Hello World!"
+        "Zm9vYmFy",  # "foobar"
+        "QUJDREVGR0hJSktMTU5PUFFSU1RVVldYWVowMTIzNDU2Nzg5",  # A long string
+        "",  # Empty string
+        example_json_keyfile_b64.decode("utf-8"),
+    ]
+
+    for s in valid_strings:
+        assert is_base64(s) is True
+
+
+def test_valid_base64_bytes(example_json_keyfile_b64):
+    valid_bytes = [
+        b"SGVsbG8gV29ybGQh",  # "Hello World!"
+        b"Zm9vYmFy",  # "foobar"
+        b"QUJDREVGR0hJSktMTU5PUFFSU1RVVldYWVowMTIzNDU2Nzg5",  # A long string
+        b"",  # Empty bytes
+        example_json_keyfile_b64,
+    ]
+    for s in valid_bytes:
+        assert is_base64(s) is True
+
+
+def test_invalid_base64(example_json_keyfile):
+    invalid_inputs = [
+        "This is not Base64",
+        "SGVsbG8gV29ybGQ",  # Incorrect padding
+        "Invalid#Base64",
+        12345,  # Not a string or bytes
+        b"Invalid#Base64",
+        "H\xffGVsbG8gV29ybGQh",  # Contains invalid character \xff
+        example_json_keyfile,
+    ]
+    for s in invalid_inputs:
+        assert is_base64(s) is False

From 995ebcba164241f1390b637953bf4192c06d63db Mon Sep 17 00:00:00 2001
From: Jeremy Cohen <jeremy@dbtlabs.com>
Date: Wed, 12 Jun 2024 15:00:24 +0200
Subject: [PATCH 809/860] Fix null column index issue during `dbt docs
 generate` for external tables (#1085) (#1257)

* coalesce column metadata columns with a value to avoid null errors for external tables in docs generate

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Fixes-20240201-145323.yaml  |  6 ++++++
 dbt/include/bigquery/macros/catalog/catalog.sql | 10 ++++++----
 2 files changed, 12 insertions(+), 4 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20240201-145323.yaml

diff --git a/.changes/unreleased/Fixes-20240201-145323.yaml b/.changes/unreleased/Fixes-20240201-145323.yaml
new file mode 100644
index 000000000..ea198e54a
--- /dev/null
+++ b/.changes/unreleased/Fixes-20240201-145323.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fix null column index issue during `dbt docs generate` for external tables
+time: 2024-02-01T14:53:23.434624-05:00
+custom:
+  Author: mikealfare
+  Issue: "1079"
diff --git a/dbt/include/bigquery/macros/catalog/catalog.sql b/dbt/include/bigquery/macros/catalog/catalog.sql
index de16f82bf..268debc5f 100644
--- a/dbt/include/bigquery/macros/catalog/catalog.sql
+++ b/dbt/include/bigquery/macros/catalog/catalog.sql
@@ -121,10 +121,12 @@
         end as table_name,
         tables.table_type,
         tables.table_comment,
-        columns.column_name,
-        columns.column_index,
-        columns.column_type,
-        columns.column_comment,
+        -- coalesce column metadata fields to ensure they are non-null for catalog generation
+        -- external table columns are not present in COLUMN_FIELD_PATHS
+        coalesce(columns.column_name, '<unknown>') as column_name,
+        coalesce(columns.column_index, 1) as column_index,
+        coalesce(columns.column_type, '<unknown>') as column_type,
+        coalesce(columns.column_comment, '') as column_comment,
 
         'Shard count' as `stats__date_shards__label`,
         table_stats.shard_count as `stats__date_shards__value`,

From e67848972cce8ab51d68c6d1b1ced0f75816108c Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Thu, 13 Jun 2024 02:51:59 +0000
Subject: [PATCH 810/860] Adap 1162/merge agate lazy load (#1250)

* lazy load agate

* Add test and documentation.

* Fix test.

* Don't need a test for this.

---------

Co-authored-by: dwreeves <xdanielreeves@gmail.com>
Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .../Under the Hood-20240331-101418.yaml       |  6 +++
 dbt/adapters/bigquery/connections.py          | 16 ++++--
 dbt/adapters/bigquery/impl.py                 | 52 +++++++++++++------
 .../bigquery/relation_configs/_base.py        | 12 +++--
 4 files changed, 63 insertions(+), 23 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20240331-101418.yaml

diff --git a/.changes/unreleased/Under the Hood-20240331-101418.yaml b/.changes/unreleased/Under the Hood-20240331-101418.yaml
new file mode 100644
index 000000000..baea00248
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20240331-101418.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Lazy load `agate`
+time: 2024-03-31T10:14:18.260074-04:00
+custom:
+  Author: dwreeves
+  Issue: "1162"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 4a3feae48..14f958a05 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -10,9 +10,8 @@
 from mashumaro.helper import pass_through
 
 from functools import lru_cache
-import agate
 from requests.exceptions import ConnectionError
-from typing import Optional, Any, Dict, Tuple
+from typing import Optional, Any, Dict, Tuple, TYPE_CHECKING
 
 import google.auth
 import google.auth.exceptions
@@ -26,7 +25,6 @@
 )
 
 from dbt.adapters.bigquery import gcloud
-from dbt_common.clients import agate_helper
 from dbt.adapters.contracts.connection import ConnectionState, AdapterResponse, Credentials
 from dbt_common.exceptions import (
     DbtRuntimeError,
@@ -44,6 +42,10 @@
 
 from dbt_common.dataclass_schema import ExtensibleDbtClassMixin, StrEnum
 
+if TYPE_CHECKING:
+    # Indirectly imported via agate_helper, which is lazy loaded further downfile.
+    # Used by mypy for earlier type hints.
+    import agate
 
 logger = AdapterLogger("BigQuery")
 
@@ -432,7 +434,9 @@ def get_job_retry_deadline_seconds(cls, conn):
         return credentials.job_retry_deadline_seconds
 
     @classmethod
-    def get_table_from_response(cls, resp):
+    def get_table_from_response(cls, resp) -> "agate.Table":
+        from dbt_common.clients import agate_helper
+
         column_names = [field.name for field in resp.schema]
         return agate_helper.table_from_data_flat(resp, column_names)
 
@@ -499,7 +503,7 @@ def fn():
 
     def execute(
         self, sql, auto_begin=False, fetch=None, limit: Optional[int] = None
-    ) -> Tuple[BigQueryAdapterResponse, agate.Table]:
+    ) -> Tuple[BigQueryAdapterResponse, "agate.Table"]:
         sql = self._add_query_comment(sql)
         # auto_begin is ignored on bigquery, and only included for consistency
         query_job, iterator = self.raw_execute(sql, limit=limit)
@@ -507,6 +511,8 @@ def execute(
         if fetch:
             table = self.get_table_from_response(iterator)
         else:
+            from dbt_common.clients import agate_helper
+
             table = agate_helper.empty_table()
 
         message = "OK"
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 36222f44f..dc5cf6e17 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -5,9 +5,20 @@
 from multiprocessing.context import SpawnContext
 
 import time
-from typing import Any, Dict, List, Optional, Type, Set, Union, FrozenSet, Tuple, Iterable
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Type,
+    Set,
+    Union,
+    FrozenSet,
+    Tuple,
+    Iterable,
+    TYPE_CHECKING,
+)
 
-import agate
 from dbt.adapters.contracts.relation import RelationConfig
 
 import dbt_common.exceptions.base
@@ -24,7 +35,6 @@
 from dbt.adapters.base.impl import FreshnessResponse
 from dbt.adapters.cache import _make_ref_key_dict  # type: ignore
 from dbt.adapters.capability import Capability, CapabilityDict, CapabilitySupport, Support
-import dbt_common.clients.agate_helper
 from dbt.adapters.contracts.connection import AdapterResponse
 from dbt.adapters.contracts.macros import MacroResolverProtocol
 from dbt_common.contracts.constraints import ColumnLevelConstraint, ConstraintType, ModelLevelConstraint  # type: ignore
@@ -58,6 +68,10 @@
 )
 from dbt.adapters.bigquery.utility import sql_escape
 
+if TYPE_CHECKING:
+    # Indirectly imported via agate_helper, which is lazy loaded further downfile.
+    # Used by mypy for earlier type hints.
+    import agate
 
 logger = AdapterLogger("BigQuery")
 
@@ -334,32 +348,34 @@ def quote(cls, identifier: str) -> str:
         return "`{}`".format(identifier)
 
     @classmethod
-    def convert_text_type(cls, agate_table: agate.Table, col_idx: int) -> str:
+    def convert_text_type(cls, agate_table: "agate.Table", col_idx: int) -> str:
         return "string"
 
     @classmethod
-    def convert_number_type(cls, agate_table: agate.Table, col_idx: int) -> str:
+    def convert_number_type(cls, agate_table: "agate.Table", col_idx: int) -> str:
+        import agate
+
         decimals = agate_table.aggregate(agate.MaxPrecision(col_idx))  # type: ignore[attr-defined]
         return "float64" if decimals else "int64"
 
     @classmethod
-    def convert_integer_type(cls, agate_table: agate.Table, col_idx: int) -> str:
+    def convert_integer_type(cls, agate_table: "agate.Table", col_idx: int) -> str:
         return "int64"
 
     @classmethod
-    def convert_boolean_type(cls, agate_table: agate.Table, col_idx: int) -> str:
+    def convert_boolean_type(cls, agate_table: "agate.Table", col_idx: int) -> str:
         return "bool"
 
     @classmethod
-    def convert_datetime_type(cls, agate_table: agate.Table, col_idx: int) -> str:
+    def convert_datetime_type(cls, agate_table: "agate.Table", col_idx: int) -> str:
         return "datetime"
 
     @classmethod
-    def convert_date_type(cls, agate_table: agate.Table, col_idx: int) -> str:
+    def convert_date_type(cls, agate_table: "agate.Table", col_idx: int) -> str:
         return "date"
 
     @classmethod
-    def convert_time_type(cls, agate_table: agate.Table, col_idx: int) -> str:
+    def convert_time_type(cls, agate_table: "agate.Table", col_idx: int) -> str:
         return "time"
 
     ###
@@ -387,7 +403,7 @@ def _get_dbt_columns_from_bq_table(self, table) -> List[BigQueryColumn]:
         return columns
 
     def _agate_to_schema(
-        self, agate_table: agate.Table, column_override: Dict[str, str]
+        self, agate_table: "agate.Table", column_override: Dict[str, str]
     ) -> List[SchemaField]:
         """Convert agate.Table with column names to a list of bigquery schemas."""
         bq_schema = []
@@ -655,7 +671,13 @@ def alter_table_add_columns(self, relation, columns):
 
     @available.parse_none
     def load_dataframe(
-        self, database, schema, table_name, agate_table, column_override, field_delimiter
+        self,
+        database,
+        schema,
+        table_name,
+        agate_table: "agate.Table",
+        column_override,
+        field_delimiter,
     ):
         bq_schema = self._agate_to_schema(agate_table, column_override)
         conn = self.connections.get_thread_connection()
@@ -667,7 +689,7 @@ def load_dataframe(
         load_config.skip_leading_rows = 1
         load_config.schema = bq_schema
         load_config.field_delimiter = field_delimiter
-        with open(agate_table.original_abspath, "rb") as f:
+        with open(agate_table.original_abspath, "rb") as f:  # type: ignore
             job = client.load_table_from_file(f, table_ref, rewind=True, job_config=load_config)
 
         timeout = self.connections.get_job_execution_timeout_seconds(conn) or 300
@@ -699,8 +721,8 @@ def upload_file(
 
     @classmethod
     def _catalog_filter_table(
-        cls, table: agate.Table, used_schemas: FrozenSet[Tuple[str, str]]
-    ) -> agate.Table:
+        cls, table: "agate.Table", used_schemas: FrozenSet[Tuple[str, str]]
+    ) -> "agate.Table":
         table = table.rename(
             column_names={col.name: col.name.replace("__", ":") for col in table.columns}
         )
diff --git a/dbt/adapters/bigquery/relation_configs/_base.py b/dbt/adapters/bigquery/relation_configs/_base.py
index be34a08ac..45e29b99f 100644
--- a/dbt/adapters/bigquery/relation_configs/_base.py
+++ b/dbt/adapters/bigquery/relation_configs/_base.py
@@ -1,7 +1,6 @@
 from dataclasses import dataclass
-from typing import Optional, Dict
+from typing import Optional, Dict, TYPE_CHECKING
 
-import agate
 from dbt.adapters.base.relation import Policy
 from dbt.adapters.relation_configs import RelationConfigBase
 from google.cloud.bigquery import Table as BigQueryTable
@@ -13,6 +12,11 @@
 )
 from dbt.adapters.contracts.relation import ComponentName, RelationConfig
 
+if TYPE_CHECKING:
+    # Indirectly imported via agate_helper, which is lazy loaded further downfile.
+    # Used by mypy for earlier type hints.
+    import agate
+
 
 @dataclass(frozen=True, eq=True, unsafe_hash=True)
 class BigQueryBaseRelationConfig(RelationConfigBase):
@@ -55,8 +59,10 @@ def _render_part(cls, component: ComponentName, value: Optional[str]) -> Optiona
         return None
 
     @classmethod
-    def _get_first_row(cls, results: agate.Table) -> agate.Row:
+    def _get_first_row(cls, results: "agate.Table") -> "agate.Row":
         try:
             return results.rows[0]
         except IndexError:
+            import agate
+
             return agate.Row(values=set())

From b828c30809676eb702d26189aa1f97ae0662f4f9 Mon Sep 17 00:00:00 2001
From: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Date: Thu, 20 Jun 2024 14:44:57 -0500
Subject: [PATCH 811/860] update user docs-issue workflow (#1260)

* update user docs-issue workflow

* add or condition trigger for workflow

* update workflow based onf feedback

* update workflow based onf feedback

* readd line 35 from previous commit
---
 .github/workflows/docs-issue.yml | 24 +++++++++++-------------
 1 file changed, 11 insertions(+), 13 deletions(-)

diff --git a/.github/workflows/docs-issue.yml b/.github/workflows/docs-issue.yml
index 00a098df8..f49cf517c 100644
--- a/.github/workflows/docs-issue.yml
+++ b/.github/workflows/docs-issue.yml
@@ -1,19 +1,18 @@
 # **what?**
-# Open an issue in docs.getdbt.com when a PR is labeled `user docs`
+# Open an issue in docs.getdbt.com when an issue is labeled `user docs` and closed as completed
 
 # **why?**
 # To reduce barriers for keeping docs up to date
 
 # **when?**
-# When a PR is labeled `user docs` and is merged.  Runs on pull_request_target to run off the workflow already merged,
-# not the workflow that existed on the PR branch.  This allows old PRs to get comments.
+# When an issue is labeled `user docs` and is closed as completed.  Can be labeled before or after the issue is closed.
 
 
-name: Open issues in docs.getdbt.com repo when a PR is labeled
-run-name: "Open an issue in docs.getdbt.com for PR #${{ github.event.pull_request.number }}"
+name: Open issues in docs.getdbt.com repo when an issue is labeled
+run-name: "Open an issue in docs.getdbt.com for issue #${{ github.event.issue.number }}"
 
 on:
-  pull_request_target:
+  issues:
     types: [labeled, closed]
 
 defaults:
@@ -21,23 +20,22 @@ defaults:
     shell: bash
 
 permissions:
-    issues: write # opens new issues
-    pull-requests: write # comments on PRs
-
+    issues: write # comments on issues
 
 jobs:
   open_issues:
-    # we only want to run this when the PR has been merged or the label in the labeled event is `user docs`.  Otherwise it runs the
+    # we only want to run this when the issue is closed as completed and the label `user docs` has been assigned.
+    # If this logic does not exist in this workflow, it runs the
     # risk of duplicaton of issues being created due to merge and label both triggering this workflow to run and neither having
     # generating the comment before the other runs.  This lives here instead of the shared workflow because this is where we
     # decide if it should run or not.
     if: |
-      (github.event.pull_request.merged == true) &&
-      ((github.event.action == 'closed' && contains( github.event.pull_request.labels.*.name, 'user docs')) ||
+      (github.event.issue.state == 'closed' && github.event.issue.state_reason == 'completed') && (
+      (github.event.action == 'closed' && contains(github.event.issue.labels.*.name, 'user docs')) ||
       (github.event.action == 'labeled' && github.event.label.name == 'user docs'))
     uses: dbt-labs/actions/.github/workflows/open-issue-in-repo.yml@main
     with:
         issue_repository: "dbt-labs/docs.getdbt.com"
-        issue_title: "Docs Changes Needed from ${{ github.event.repository.name }} PR #${{ github.event.pull_request.number }}"
+        issue_title: "Docs Changes Needed from ${{ github.event.repository.name }} Issue #${{ github.event.issue.number }}"
         issue_body: "At a minimum, update body to include a link to the page on docs.getdbt.com requiring updates and what part(s) of the page you would like to see updated."
     secrets: inherit

From 0627aa2ec5283fdc3f7df43865d37e3612f3df5b Mon Sep 17 00:00:00 2001
From: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 20 Jun 2024 15:25:03 -0700
Subject: [PATCH 812/860] Update internal release naming and arguments (#1263)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* Update internal release naming and arguments

---------

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .github/workflows/release-internal.yml | 14 ++++++++------
 1 file changed, 8 insertions(+), 6 deletions(-)

diff --git a/.github/workflows/release-internal.yml b/.github/workflows/release-internal.yml
index f29de2fb2..dbc740909 100644
--- a/.github/workflows/release-internal.yml
+++ b/.github/workflows/release-internal.yml
@@ -10,15 +10,12 @@
 #
 # Manual trigger.
 
-name: "Release internal patch"
+name: "Release to Cloud"
+run-name: "Release to Cloud off of ${{ inputs.ref }}"
 
 on:
   workflow_dispatch:
     inputs:
-      version_number:
-        description: "The release version number (i.e. 1.0.0b1)"
-        type: string
-        required: true
       ref:
         description: "The ref (sha or branch name) to use"
         type: string
@@ -29,6 +26,11 @@ on:
         type: string
         default: "python -c \"import dbt.adapters.bigquery\""
         required: true
+      skip_tests:
+        description: "Should the tests be skipped? (default to false)"
+        type: boolean
+        required: true
+        default: false
 
 defaults:
   run:
@@ -41,9 +43,9 @@ jobs:
     uses: "dbt-labs/dbt-release/.github/workflows/internal-archive-release.yml@main"
 
     with:
-      version_number: "${{ inputs.version_number }}"
       package_test_command: "${{ inputs.package_test_command }}"
       dbms_name: "bigquery"
       ref: "${{ inputs.ref }}"
+      skip_tests: "${{ inputs.skip_tests }}"
 
     secrets: "inherit"

From 2c77893d0fb7b5d76ba85cc16712b9563a60329b Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Mon, 8 Jul 2024 12:10:29 -0400
Subject: [PATCH 813/860] Base 207/add test (#1266)

* Add test for upstream change.
* Change database message.
* Adjust test case to reflect new syntax scheme.

---------

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .../functional/adapter/dbt_show/test_dbt_show.py  | 15 +++++++++++----
 1 file changed, 11 insertions(+), 4 deletions(-)

diff --git a/tests/functional/adapter/dbt_show/test_dbt_show.py b/tests/functional/adapter/dbt_show/test_dbt_show.py
index acb54cc47..6794547a5 100644
--- a/tests/functional/adapter/dbt_show/test_dbt_show.py
+++ b/tests/functional/adapter/dbt_show/test_dbt_show.py
@@ -1,5 +1,9 @@
 import pytest
-from dbt.tests.adapter.dbt_show.test_dbt_show import BaseShowSqlHeader, BaseShowLimit
+from dbt.tests.adapter.dbt_show.test_dbt_show import (
+    BaseShowSqlHeader,
+    BaseShowLimit,
+    BaseShowDoesNotHandleDoubleLimit,
+)
 
 from dbt.tests.util import run_dbt
 
@@ -20,8 +24,7 @@
     ]
   )
   as v
-    ) as model_limit_subq
-    limit 5
+    )
     """
 
 model_with_null_json_struct = """
@@ -48,7 +51,11 @@ def models(self):
         }
 
     def test_sql_header(self, project):
-        run_dbt(["show", "--select", "json_struct_model"])
+        run_dbt(["show", "--select", "json_struct_model", "-d"])
 
     def test_show_with_null_json_struct(self, project):
         run_dbt(["show", "--select", "null_json_struct_model"])
+
+
+class TestBigQueryShowDoesNotHandleDoubleLimit(BaseShowDoesNotHandleDoubleLimit):
+    DATABASE_ERROR_MESSAGE = "Syntax error: Expected end of input but got keyword LIMIT"

From 4c459ea54b0dcd4ba46cb93eac10db41a505c07c Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Wed, 17 Jul 2024 21:53:48 +0000
Subject: [PATCH 814/860] Update freezegun requirement from ~=1.3 to ~=1.4
 (#1062)

* Update freezegun requirement from ~=1.3 to ~=1.4

Updates the requirements on [freezegun](https://github.com/spulec/freezegun) to permit the latest version.
- [Release notes](https://github.com/spulec/freezegun/releases)
- [Changelog](https://github.com/spulec/freezegun/blob/master/CHANGELOG)
- [Commits](https://github.com/spulec/freezegun/compare/1.3.0...1.4.0)

---
updated-dependencies:
- dependency-name: freezegun
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231220-002130.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231220-002130.yaml

diff --git a/.changes/unreleased/Dependencies-20231220-002130.yaml b/.changes/unreleased/Dependencies-20231220-002130.yaml
new file mode 100644
index 000000000..d62e50bf2
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231220-002130.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update freezegun requirement from ~=1.3 to ~=1.4"
+time: 2023-12-20T00:21:30.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1062
diff --git a/dev-requirements.txt b/dev-requirements.txt
index cf7b1b87c..f58e717db 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -12,7 +12,7 @@ click~=8.1
 ddtrace==2.3.0
 flake8~=6.1
 flaky~=3.7
-freezegun~=1.3
+freezegun~=1.4
 ipdb~=0.13.13
 mypy==1.7.1  # patch updates have historically introduced breaking changes
 pip-tools~=7.3

From 01979c672635675a10753e29c23c4a99d1df9b59 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Wed, 17 Jul 2024 23:30:15 +0000
Subject: [PATCH 815/860] Update flake8 requirement from ~=6.1 to ~=7.0 (#1069)

* Update flake8 requirement from ~=6.1 to ~=7.0

Updates the requirements on [flake8](https://github.com/pycqa/flake8) to permit the latest version.
- [Commits](https://github.com/pycqa/flake8/compare/6.1.0...7.0.0)

---
updated-dependencies:
- dependency-name: flake8
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20240105-004800.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20240105-004800.yaml

diff --git a/.changes/unreleased/Dependencies-20240105-004800.yaml b/.changes/unreleased/Dependencies-20240105-004800.yaml
new file mode 100644
index 000000000..b0d33ceed
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240105-004800.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update flake8 requirement from ~=6.1 to ~=7.0"
+time: 2024-01-05T00:48:00.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1069
diff --git a/dev-requirements.txt b/dev-requirements.txt
index f58e717db..c78df51fe 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -10,7 +10,7 @@ black>=24.3
 bumpversion~=0.6.0
 click~=8.1
 ddtrace==2.3.0
-flake8~=6.1
+flake8~=7.0
 flaky~=3.7
 freezegun~=1.4
 ipdb~=0.13.13

From 0afeacc954eb5bcea266db8db04c84a7f3342ede Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Wed, 17 Jul 2024 20:22:52 -0400
Subject: [PATCH 816/860] Bump mypy from 1.7.1 to 1.8.0 (#1064)

* Bump mypy from 1.7.1 to 1.8.0

Bumps [mypy](https://github.com/python/mypy) from 1.7.1 to 1.8.0.
- [Changelog](https://github.com/python/mypy/blob/master/CHANGELOG.md)
- [Commits](https://github.com/python/mypy/compare/v1.7.1...v1.8.0)

---
updated-dependencies:
- dependency-name: mypy
  dependency-type: direct:development
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20231222-002351.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231222-002351.yaml

diff --git a/.changes/unreleased/Dependencies-20231222-002351.yaml b/.changes/unreleased/Dependencies-20231222-002351.yaml
new file mode 100644
index 000000000..76591de93
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231222-002351.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump mypy from 1.7.1 to 1.8.0"
+time: 2023-12-22T00:23:51.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1064
diff --git a/dev-requirements.txt b/dev-requirements.txt
index c78df51fe..de57af267 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -14,7 +14,7 @@ flake8~=7.0
 flaky~=3.7
 freezegun~=1.4
 ipdb~=0.13.13
-mypy==1.7.1  # patch updates have historically introduced breaking changes
+mypy==1.8.0  # patch updates have historically introduced breaking changes
 pip-tools~=7.3
 pre-commit~=3.5
 pre-commit-hooks~=4.5

From b4206efb27ad79312845416306df02c7ae0f008d Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 18 Jul 2024 00:58:46 +0000
Subject: [PATCH 817/860] Update pre-commit requirement from ~=3.5 to ~=3.6
 (#1052)

* Update pre-commit requirement from ~=3.5 to ~=3.6

Updates the requirements on [pre-commit](https://github.com/pre-commit/pre-commit) to permit the latest version.
- [Release notes](https://github.com/pre-commit/pre-commit/releases)
- [Changelog](https://github.com/pre-commit/pre-commit/blob/main/CHANGELOG.md)
- [Commits](https://github.com/pre-commit/pre-commit/compare/v3.5.0...v3.6.0)

---
updated-dependencies:
- dependency-name: pre-commit
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

* update pre-commit pin for python 3.8

* fix changelog

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
---
 .changes/unreleased/Dependencies-20231211-001048.yaml | 6 ++++++
 dev-requirements.txt                                  | 3 ++-
 2 files changed, 8 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20231211-001048.yaml

diff --git a/.changes/unreleased/Dependencies-20231211-001048.yaml b/.changes/unreleased/Dependencies-20231211-001048.yaml
new file mode 100644
index 000000000..6f2bfada4
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20231211-001048.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pre-commit requirement from ~=3.5 to ~=3.7"
+time: 2023-12-11T00:10:48.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1052
diff --git a/dev-requirements.txt b/dev-requirements.txt
index de57af267..5109bd93b 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -16,7 +16,8 @@ freezegun~=1.4
 ipdb~=0.13.13
 mypy==1.8.0  # patch updates have historically introduced breaking changes
 pip-tools~=7.3
-pre-commit~=3.5
+pre-commit==3.7.0;python_version >="3.9"
+pre-commit==3.5.0;python_version <"3.9"
 pre-commit-hooks~=4.5
 pytest~=7.4
 pytest-csv~=3.0

From 52cff47a712c0688363bea10cce52cc5bec25417 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 18 Jul 2024 01:45:05 +0000
Subject: [PATCH 818/860] Update pre-commit-hooks requirement from ~=4.5 to
 ~=4.6 (#1281)

* Update pre-commit-hooks requirement from ~=4.5 to ~=4.6

Updates the requirements on [pre-commit-hooks](https://github.com/pre-commit/pre-commit-hooks) to permit the latest version.
- [Release notes](https://github.com/pre-commit/pre-commit-hooks/releases)
- [Changelog](https://github.com/pre-commit/pre-commit-hooks/blob/main/CHANGELOG.md)
- [Commits](https://github.com/pre-commit/pre-commit-hooks/compare/v4.5.0...v4.6.0)

---
updated-dependencies:
- dependency-name: pre-commit-hooks
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20240718-005755.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20240718-005755.yaml

diff --git a/.changes/unreleased/Dependencies-20240718-005755.yaml b/.changes/unreleased/Dependencies-20240718-005755.yaml
new file mode 100644
index 000000000..3d2cca66c
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240718-005755.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pre-commit-hooks requirement from ~=4.5 to ~=4.6"
+time: 2024-07-18T00:57:55.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1281
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 5109bd93b..788ba6384 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -18,7 +18,7 @@ mypy==1.8.0  # patch updates have historically introduced breaking changes
 pip-tools~=7.3
 pre-commit==3.7.0;python_version >="3.9"
 pre-commit==3.5.0;python_version <"3.9"
-pre-commit-hooks~=4.5
+pre-commit-hooks~=4.6
 pytest~=7.4
 pytest-csv~=3.0
 pytest-dotenv~=0.5.2

From a03a1e810c6a039196f9781cafada239a8300005 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 18 Jul 2024 04:21:09 +0000
Subject: [PATCH 819/860] Update pytest-xdist requirement from ~=3.5 to ~=3.6
 (#1282)

* Update pytest-xdist requirement from ~=3.5 to ~=3.6

Updates the requirements on [pytest-xdist](https://github.com/pytest-dev/pytest-xdist) to permit the latest version.
- [Release notes](https://github.com/pytest-dev/pytest-xdist/releases)
- [Changelog](https://github.com/pytest-dev/pytest-xdist/blob/master/CHANGELOG.rst)
- [Commits](https://github.com/pytest-dev/pytest-xdist/compare/v3.5.0...v3.6.1)

---
updated-dependencies:
- dependency-name: pytest-xdist
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20240718-005756.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20240718-005756.yaml

diff --git a/.changes/unreleased/Dependencies-20240718-005756.yaml b/.changes/unreleased/Dependencies-20240718-005756.yaml
new file mode 100644
index 000000000..ac6b791a8
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240718-005756.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update pytest-xdist requirement from ~=3.5 to ~=3.6"
+time: 2024-07-18T00:57:56.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1282
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 788ba6384..e0d66d88c 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -23,7 +23,7 @@ pytest~=7.4
 pytest-csv~=3.0
 pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
-pytest-xdist~=3.5
+pytest-xdist~=3.6
 pytz~=2023.3
 tox~=4.11
 types-pytz~=2023.3

From f5ed1de0ae8a1bd4d05dc8570e6be39199d59967 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 18 Jul 2024 12:29:58 +0000
Subject: [PATCH 820/860] Update flaky requirement from ~=3.7 to ~=3.8 (#1283)

* Update flaky requirement from ~=3.7 to ~=3.8

Updates the requirements on [flaky](https://github.com/box/flaky) to permit the latest version.
- [Release notes](https://github.com/box/flaky/releases)
- [Changelog](https://github.com/box/flaky/blob/master/HISTORY.rst)
- [Commits](https://github.com/box/flaky/compare/v3.7.0...v3.8.1)

---
updated-dependencies:
- dependency-name: flaky
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

* update changelog name to avoid conflict with contemporaneous dependabot update

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20240718-005757.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20240718-005757.yaml

diff --git a/.changes/unreleased/Dependencies-20240718-005757.yaml b/.changes/unreleased/Dependencies-20240718-005757.yaml
new file mode 100644
index 000000000..29e12d68e
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240718-005757.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update flaky requirement from ~=3.7 to ~=3.8"
+time: 2024-07-18T00:57:57.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1283
diff --git a/dev-requirements.txt b/dev-requirements.txt
index e0d66d88c..95ed0c8ec 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -11,7 +11,7 @@ bumpversion~=0.6.0
 click~=8.1
 ddtrace==2.3.0
 flake8~=7.0
-flaky~=3.7
+flaky~=3.8
 freezegun~=1.4
 ipdb~=0.13.13
 mypy==1.8.0  # patch updates have historically introduced breaking changes

From 80d88113e69ddde59463e888e4f46f28e9cead8c Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 18 Jul 2024 10:40:43 -0400
Subject: [PATCH 821/860] Bump actions/download-artifact from 3 to 4 (#1209)

---
 .changes/unreleased/Dependencies-20240429-005158.yaml | 6 ++++++
 .github/workflows/integration.yml                     | 1 +
 .github/workflows/main.yml                            | 2 +-
 3 files changed, 8 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20240429-005158.yaml

diff --git a/.changes/unreleased/Dependencies-20240429-005158.yaml b/.changes/unreleased/Dependencies-20240429-005158.yaml
new file mode 100644
index 000000000..5d380952c
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240429-005158.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump actions/download-artifact from 3 to 4"
+time: 2024-04-29T00:51:58.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1209
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 2ae9684d9..03c1e06ed 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -97,6 +97,7 @@ jobs:
               - 'dbt/**'
               - 'tests/**'
               - 'dev-requirements.txt'
+              - '.github/**'
 
       - name: Generate integration test matrix
         id: generate-matrix
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index fdbd36dba..4cd05f9ca 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -186,7 +186,7 @@ jobs:
           python -m pip install --user --upgrade pip
           python -m pip install --upgrade wheel setuptools twine check-wheel-contents
           python -m pip --version
-      - uses: actions/download-artifact@v3
+      - uses: actions/download-artifact@v4
         with:
           name: dist
           path: dist/

From 16a7dcbdb777defe80797d3a0e52414418ffb9d4 Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Thu, 18 Jul 2024 21:36:48 +0000
Subject: [PATCH 822/860] Bump actions/upload-artifact from 3 to 4 (#1210)

* Bump actions/upload-artifact from 3 to 4

Bumps [actions/upload-artifact](https://github.com/actions/upload-artifact) from 3 to 4.
- [Release notes](https://github.com/actions/upload-artifact/releases)
- [Commits](https://github.com/actions/upload-artifact/compare/v3...v4)

---
updated-dependencies:
- dependency-name: actions/upload-artifact
  dependency-type: direct:production
  update-type: version-update:semver-major
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

* add overwrite parameter which was implicitly true in v3 but false in v4

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Mike Alfare <mike.alfare@dbtlabs.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20240429-005159.yaml | 6 ++++++
 .github/workflows/integration.yml                     | 6 ++++--
 .github/workflows/main.yml                            | 6 ++++--
 3 files changed, 14 insertions(+), 4 deletions(-)
 create mode 100644 .changes/unreleased/Dependencies-20240429-005159.yaml

diff --git a/.changes/unreleased/Dependencies-20240429-005159.yaml b/.changes/unreleased/Dependencies-20240429-005159.yaml
new file mode 100644
index 000000000..adf2cae65
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240429-005159.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Bump actions/upload-artifact from 3 to 4"
+time: 2024-04-29T00:51:59.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1210
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 03c1e06ed..a9ead6cb1 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -189,11 +189,12 @@ jobs:
           GCS_BUCKET: dbt-ci
         run: tox -- --ddtrace
 
-      - uses: actions/upload-artifact@v3
+      - uses: actions/upload-artifact@v4
         if: always()
         with:
           name: logs
           path: ./logs
+          overwrite: true
 
       - name: Get current date
         if: always()
@@ -201,11 +202,12 @@ jobs:
         run: |
           echo "date=$(date +'%Y-%m-%dT%H_%M_%S')" >> $GITHUB_OUTPUT #no colons allowed for artifacts
 
-      - uses: actions/upload-artifact@v3
+      - uses: actions/upload-artifact@v4
         if: always()
         with:
           name: integration_results_${{ matrix.python-version }}_${{ matrix.os }}_${{ matrix.adapter }}-${{ steps.date.outputs.date }}.csv
           path: integration_results.csv
+          overwrite: true
 
   # python integration tests are slow so we only run them seperately and for a single OS / python version
   test-python:
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 4cd05f9ca..631bc7652 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -104,11 +104,12 @@ jobs:
         run: |
           echo "date=$(date +'%Y-%m-%dT%H_%M_%S')" >> $GITHUB_OUTPUT #no colons allowed for artifacts
 
-      - uses: actions/upload-artifact@v3
+      - uses: actions/upload-artifact@v4
         if: always()
         with:
           name: unit_results_${{ matrix.python-version }}-${{ steps.date.outputs.date }}.csv
           path: unit_results.csv
+          overwrite: true
 
   build:
     name: build packages
@@ -156,10 +157,11 @@ jobs:
           if [[ "$(ls -lh dist/)" == *"a1"* ]]; then export is_alpha=1; fi
           echo "is_alpha=$is_alpha" >> $GITHUB_OUTPUT
 
-      - uses: actions/upload-artifact@v3
+      - uses: actions/upload-artifact@v4
         with:
           name: dist
           path: dist/
+          overwrite: true
 
   test-build:
     name: verify packages / python ${{ matrix.python-version }} / ${{ matrix.os }}

From b8bc9c0a4d4e093b7d99d05f1aedc5bf54258a2d Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 19 Jul 2024 20:31:54 +0000
Subject: [PATCH 823/860] Update twine requirement from ~=4.0 to ~=5.1 (#1293)

* Update twine requirement from ~=4.0 to ~=5.1

Updates the requirements on [twine](https://github.com/pypa/twine) to permit the latest version.
- [Release notes](https://github.com/pypa/twine/releases)
- [Changelog](https://github.com/pypa/twine/blob/main/docs/changelog.rst)
- [Commits](https://github.com/pypa/twine/compare/4.0.0...v5.1.1)

---
updated-dependencies:
- dependency-name: twine
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
---
 .changes/unreleased/Dependencies-20240719-003740.yaml | 6 ++++++
 dev-requirements.txt                                  | 2 +-
 2 files changed, 7 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Dependencies-20240719-003740.yaml

diff --git a/.changes/unreleased/Dependencies-20240719-003740.yaml b/.changes/unreleased/Dependencies-20240719-003740.yaml
new file mode 100644
index 000000000..feb483a60
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240719-003740.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update twine requirement from ~=4.0 to ~=5.1"
+time: 2024-07-19T00:37:40.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1293
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 95ed0c8ec..a81a841f0 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -29,5 +29,5 @@ tox~=4.11
 types-pytz~=2023.3
 types-protobuf~=4.24
 types-requests~=2.31
-twine~=4.0
+twine~=5.1
 wheel~=0.42

From b9018f7bbae7817b7b1f1124bb166732e905de53 Mon Sep 17 00:00:00 2001
From: Vinit Deshbhratar <deshbhratarvinit@gmail.com>
Date: Mon, 22 Jul 2024 12:25:06 -0400
Subject: [PATCH 824/860] ADAP-1051 - Temp Table Drop Fix (#1076)

* Fix for ADAP-1051

* adding changie files

* adding changie files

* Revert "adding changie files"

This reverts commit f51ca0c28829b93caa42f6a3dcb57d57ea5f3910.

* Add test cases

* Revert adapters.sql

* Update body for changie

---------

Co-authored-by: Doug Beatty <44704949+dbeatty10@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Co-authored-by: Teresa Martyny <135771777+martynydbt@users.noreply.github.com>
Co-authored-by: Mila Page <67295367+VersusFacit@users.noreply.github.com>
---
 .../unreleased/Fixes-20240120-180818.yaml     |  6 ++
 .../macros/materializations/incremental.sql   |  8 +--
 tests/functional/test_drop_temp_relation.py   | 60 +++++++++++++++++++
 3 files changed, 70 insertions(+), 4 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20240120-180818.yaml
 create mode 100644 tests/functional/test_drop_temp_relation.py

diff --git a/.changes/unreleased/Fixes-20240120-180818.yaml b/.changes/unreleased/Fixes-20240120-180818.yaml
new file mode 100644
index 000000000..0d0740361
--- /dev/null
+++ b/.changes/unreleased/Fixes-20240120-180818.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Drop intermediate objects created in BigQuery for incremental models
+time: 2024-01-20T18:08:18.817915-06:00
+custom:
+  Author: vinit2107
+  Issue: "1036"
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index 2cbb14d9b..3908bedc2 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -151,10 +151,6 @@
       {{ build_sql }}
     {% endcall %}
 
-    {%- if language == 'python' and tmp_relation -%}
-      {{ adapter.drop_relation(tmp_relation) }}
-    {%- endif -%}
-
   {% endif %}
 
   {{ run_hooks(post_hooks) }}
@@ -166,6 +162,10 @@
 
   {% do persist_docs(target_relation, model) %}
 
+  {%- if tmp_relation_exists -%}
+    {{ adapter.drop_relation(tmp_relation) }}
+  {%- endif -%}
+
   {{ return({'relations': [target_relation]}) }}
 
 {%- endmaterialization %}
diff --git a/tests/functional/test_drop_temp_relation.py b/tests/functional/test_drop_temp_relation.py
new file mode 100644
index 000000000..4cdfaedae
--- /dev/null
+++ b/tests/functional/test_drop_temp_relation.py
@@ -0,0 +1,60 @@
+import pytest
+from google.api_core.exceptions import NotFound
+from dbt.adapters.bigquery.relation import BigQueryRelation
+from dbt.tests.util import run_dbt, get_connection, relation_from_name
+
+
+_INCREMENTAL_MODEL = """
+{{
+    config(
+        materialized="incremental",
+        on_schema_change="sync_all_columns"
+    )
+}}
+    select 20 as id, cast('2020-01-01 01:00:00' as datetime) as date_hour union all
+    select 40 as id, cast('2020-01-01 02:00:00' as datetime) as date_hour
+"""
+
+_INCREMENTAL_MODEL_YAML = """version: 2
+models:
+- name: test_drop_relation
+  columns:
+  - name: id
+    type: int64
+  - name: date_hour
+    type: datetime
+"""
+
+
+class BaseIncrementalModelConfig:
+    @pytest.fixture(scope="class")
+    def models(self):
+        return {
+            "test_drop_relation.sql": _INCREMENTAL_MODEL,
+            "schema.yml": _INCREMENTAL_MODEL_YAML,
+        }
+
+
+class TestIncrementalModel(BaseIncrementalModelConfig):
+    def test_incremental_model_succeeds(self, project):
+        """
+        Steps:
+        1. Create the model
+        2. Merge into the model using __dbt_tmp table
+        3. Assert raises NotFound exception
+        """
+        results = run_dbt(["run"])
+        assert len(results) == 1
+        results = run_dbt(["run"])
+        assert len(results) == 1
+        relation: BigQueryRelation = relation_from_name(
+            project.adapter, "test_drop_relation__dbt_tmp"
+        )
+        adapter = project.adapter
+        with pytest.raises(NotFound):
+            with get_connection(project.adapter) as conn:
+                conn.handle.get_table(
+                    adapter.connections.get_bq_table(
+                        relation.database, relation.schema, relation.table
+                    )
+                )

From ea848b02b9485841997c518c9992789bea0a9d82 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 24 Jul 2024 00:05:25 -0400
Subject: [PATCH 825/860] make dependabot ignore patch releases (#1296)

---
 .github/dependabot.yml | 12 ++++++++++++
 1 file changed, 12 insertions(+)

diff --git a/.github/dependabot.yml b/.github/dependabot.yml
index ae2be43aa..746dcae22 100644
--- a/.github/dependabot.yml
+++ b/.github/dependabot.yml
@@ -5,13 +5,25 @@ updates:
     schedule:
       interval: "daily"
     rebase-strategy: "disabled"
+    ignore:
+      - dependency-name: "*"
+        update-types:
+          - version-update:semver-patch
   - package-ecosystem: "github-actions"
     directory: "/"
     schedule:
       interval: "weekly"
     rebase-strategy: "disabled"
+    ignore:
+      - dependency-name: "*"
+        update-types:
+          - version-update:semver-patch
   - package-ecosystem: "docker"
     directory: "/docker"
     schedule:
       interval: "weekly"
     rebase-strategy: "disabled"
+    ignore:
+      - dependency-name: "*"
+        update-types:
+          - version-update:semver-patch

From 8c0a192dbb7944b431a4cf6ca7b9cfb05dfc0dc7 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 24 Jul 2024 19:44:13 -0400
Subject: [PATCH 826/860] Move code quality deps to precommit (#1291)

* move linter config into .pre-commit-config.yaml
* make updates from mypy - all unneeded type ignore comments
* update contributing guide to better reflect current state
---
 .../Under the Hood-20240718-193206.yaml       |   6 +
 .flake8                                       |  16 ---
 .github/workflows/main.yml                    |   1 -
 .pre-commit-config.yaml                       | 114 ++++++++----------
 CONTRIBUTING.md                               |   2 +-
 dbt/adapters/bigquery/__init__.py             |   2 +-
 dbt/adapters/bigquery/column.py               |   6 +-
 dbt/adapters/bigquery/connections.py          |   6 +-
 dbt/adapters/bigquery/dataproc/batch.py       |  12 +-
 dbt/adapters/bigquery/impl.py                 |  32 ++---
 dbt/adapters/bigquery/python_submissions.py   |  12 +-
 dbt/adapters/bigquery/relation.py             |   2 +-
 .../bigquery/relation_configs/_base.py        |   4 +-
 .../bigquery/relation_configs/_cluster.py     |   6 +-
 .../relation_configs/_materialized_view.py    |   2 +-
 .../bigquery/relation_configs/_options.py     |  10 +-
 .../bigquery/relation_configs/_partition.py   |   2 +-
 dev-requirements.txt                          |  32 ++---
 mypy.ini                                      |   2 -
 .../python_model_tests/test_list_inference.py |   1 +
 tests/unit/utils.py                           |   1 +
 21 files changed, 121 insertions(+), 150 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20240718-193206.yaml
 delete mode 100644 .flake8

diff --git a/.changes/unreleased/Under the Hood-20240718-193206.yaml b/.changes/unreleased/Under the Hood-20240718-193206.yaml
new file mode 100644
index 000000000..32b3084f5
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20240718-193206.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Simplify linting environment and dev dependencies
+time: 2024-07-18T19:32:06.044016-04:00
+custom:
+  Author: mikealfare
+  Issue: "1291"
diff --git a/.flake8 b/.flake8
deleted file mode 100644
index da7e039fd..000000000
--- a/.flake8
+++ /dev/null
@@ -1,16 +0,0 @@
-[flake8]
-select =
-    E
-    W
-    F
-ignore =
-    # makes Flake8 work like black
-    W503,
-    W504,
-    # makes Flake8 work like black
-    E203,
-    E741,
-    E501,
-exclude = tests
-per-file-ignores =
-    */__init__.py: F401
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 631bc7652..a88793bde 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -58,7 +58,6 @@ jobs:
           python -m pip install -r dev-requirements.txt
           python -m pip --version
           pre-commit --version
-          mypy --version
           dbt --version
       - name: Run pre-comit hooks
         run: pre-commit run --all-files --show-diff-on-failure
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index 74dbdf99a..0739a7cc4 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -1,66 +1,58 @@
 # For more on configuring pre-commit hooks (see https://pre-commit.com/)
-
 default_language_version:
-  python: python3
+    python: python3
 
 repos:
-- repo: https://github.com/pre-commit/pre-commit-hooks
-  rev: v4.4.0
-  hooks:
-  - id: check-yaml
-    args: [--unsafe]
-  - id: check-json
-  - id: end-of-file-fixer
-  - id: trailing-whitespace
-  - id: check-case-conflict
-- repo: https://github.com/dbt-labs/pre-commit-hooks
-  rev: v0.1.0a1
-  hooks:
-  - id: dbt-core-in-adapters-check
-- repo: https://github.com/psf/black
-  rev: 23.1.0
-  hooks:
-  - id: black
-    additional_dependencies: ['click~=8.1']
-    args:
-    - "--line-length=99"
-    - "--target-version=py38"
-  - id: black
-    alias: black-check
-    stages: [manual]
-    additional_dependencies: ['click~=8.1']
-    args:
-    - "--line-length=99"
-    - "--target-version=py38"
-    - "--check"
-    - "--diff"
-- repo: https://github.com/pycqa/flake8
-  rev: 6.0.0
-  hooks:
-  - id: flake8
-  - id: flake8
-    alias: flake8-check
-    stages: [manual]
-- repo: https://github.com/pre-commit/mirrors-mypy
-  rev: v1.1.1
-  hooks:
-  - id: mypy
-    # N.B.: Mypy is... a bit fragile.
-    #
-    # By using `language: system` we run this hook in the local
-    # environment instead of a pre-commit isolated one.  This is needed
-    # to ensure mypy correctly parses the project.
+-   repo: https://github.com/pre-commit/pre-commit-hooks
+    rev: v4.6.0
+    hooks:
+    -   id: check-yaml
+        args: [--unsafe]
+    -   id: check-json
+    -   id: end-of-file-fixer
+    -   id: trailing-whitespace
+    -   id: check-case-conflict
+
+-   repo: https://github.com/dbt-labs/pre-commit-hooks
+    rev: v0.1.0a1
+    hooks:
+    -   id: dbt-core-in-adapters-check
+
+-   repo: https://github.com/psf/black
+    rev: 24.4.2
+    hooks:
+    -   id: black
+        args:
+        -   --line-length=99
+        -   --target-version=py38
+        -   --target-version=py39
+        -   --target-version=py310
+        -   --target-version=py311
+        additional_dependencies: [flaky]
+
+-   repo: https://github.com/pycqa/flake8
+    rev: 7.0.0
+    hooks:
+    -   id: flake8
+        exclude: tests/
+        args:
+        -   --max-line-length=99
+        -   --select=E,F,W
+        -   --ignore=E203,E501,E741,W503,W504
+        -   --per-file-ignores=*/__init__.py:F401
 
-    # It may cause trouble in that it adds environmental variables out
-    # of our control to the mix.  Unfortunately, there's nothing we can
-    # do about per pre-commit's author.
-    # See https://github.com/pre-commit/pre-commit/issues/730 for details.
-    args: [--show-error-codes, --ignore-missing-imports, --explicit-package-bases]
-    files: ^dbt/adapters/.*
-    language: system
-  - id: mypy
-    alias: mypy-check
-    stages: [manual]
-    args: [--show-error-codes, --pretty, --ignore-missing-imports, --explicit-package-bases]
-    files: ^dbt/adapters
-    language: system
+-   repo: https://github.com/pre-commit/mirrors-mypy
+    rev: v1.10.0
+    hooks:
+    -   id: mypy
+        args:
+        -   --explicit-package-bases
+        -   --ignore-missing-imports
+        -   --pretty
+        -   --show-error-codes
+        -   --warn-unused-ignores
+        files: ^dbt/adapters/bigquery
+        additional_dependencies:
+        -   types-protobuf
+        -   types-pytz
+        -   types-requests
diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md
index e9432d363..1af648741 100644
--- a/CONTRIBUTING.md
+++ b/CONTRIBUTING.md
@@ -67,7 +67,7 @@ $EDITOR test.env
 There are a few methods for running tests locally.
 
 #### `tox`
-`tox` takes care of managing Python virtualenvs and installing dependencies in order to run tests. You can also run tests in parallel, for example you can run unit tests for Python 3.8, Python 3.9, and `flake8` checks in parallel with `tox -p`. Also, you can run unit tests for specific python versions with `tox -e py38`. The configuration of these tests are located in `tox.ini`.
+`tox` takes care of managing Python virtualenvs and installing dependencies in order to run tests. You can also run tests in parallel, for example you can run unit tests for Python 3.8, Python 3.9, Python 3.10, and Python 3.11 in parallel with `tox -p`. Also, you can run unit tests for specific python versions with `tox -e py38`. The configuration of these tests are located in `tox.ini`.
 
 #### `pytest`
 Finally, you can also run a specific test or group of tests using `pytest` directly. With a Python virtualenv active and dev dependencies installed you can do things like:
diff --git a/dbt/adapters/bigquery/__init__.py b/dbt/adapters/bigquery/__init__.py
index adbd67590..5fe68e786 100644
--- a/dbt/adapters/bigquery/__init__.py
+++ b/dbt/adapters/bigquery/__init__.py
@@ -8,5 +8,5 @@
 from dbt.include import bigquery
 
 Plugin = AdapterPlugin(
-    adapter=BigQueryAdapter, credentials=BigQueryCredentials, include_path=bigquery.PACKAGE_PATH  # type: ignore[arg-type]
+    adapter=BigQueryAdapter, credentials=BigQueryCredentials, include_path=bigquery.PACKAGE_PATH
 )
diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py
index 1bdf4323d..4a12e211f 100644
--- a/dbt/adapters/bigquery/column.py
+++ b/dbt/adapters/bigquery/column.py
@@ -18,7 +18,7 @@ class BigQueryColumn(Column):
         "INTEGER": "INT64",
     }
     fields: List[Self]  # type: ignore
-    mode: str  # type: ignore
+    mode: str
 
     def __init__(
         self,
@@ -110,7 +110,7 @@ def is_numeric(self) -> bool:
     def is_float(self):
         return self.dtype.lower() == "float64"
 
-    def can_expand_to(self: Self, other_column: Self) -> bool:  # type: ignore
+    def can_expand_to(self: Self, other_column: Self) -> bool:
         """returns True if both columns are strings"""
         return self.is_string() and other_column.is_string()
 
@@ -124,7 +124,7 @@ def column_to_bq_schema(self) -> SchemaField:
             fields = [field.column_to_bq_schema() for field in self.fields]  # type: ignore[attr-defined]
             kwargs = {"fields": fields}
 
-        return SchemaField(self.name, self.dtype, self.mode, **kwargs)  # type: ignore[arg-type]
+        return SchemaField(self.name, self.dtype, self.mode, **kwargs)
 
 
 def get_nested_column_data_types(
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 14f958a05..0d57d22c3 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -116,8 +116,8 @@ class BigQueryCredentials(Credentials):
 
     # BigQuery allows an empty database / project, where it defers to the
     # environment for the project
-    database: Optional[str] = None  # type: ignore
-    schema: Optional[str] = None  # type: ignore
+    database: Optional[str] = None
+    schema: Optional[str] = None
     execution_project: Optional[str] = None
     location: Optional[str] = None
     priority: Optional[Priority] = None
@@ -568,7 +568,7 @@ def execute(
         else:
             message = f"{code}"
 
-        response = BigQueryAdapterResponse(  # type: ignore[call-arg]
+        response = BigQueryAdapterResponse(
             _message=message,
             rows_affected=num_rows,
             code=code,
diff --git a/dbt/adapters/bigquery/dataproc/batch.py b/dbt/adapters/bigquery/dataproc/batch.py
index 61dc3c18b..e7f13c913 100644
--- a/dbt/adapters/bigquery/dataproc/batch.py
+++ b/dbt/adapters/bigquery/dataproc/batch.py
@@ -20,9 +20,9 @@ def create_batch_request(
     batch: Batch, batch_id: str, project: str, region: str
 ) -> CreateBatchRequest:
     return CreateBatchRequest(
-        parent=f"projects/{project}/locations/{region}",  # type: ignore
-        batch_id=batch_id,  # type: ignore
-        batch=batch,  # type: ignore
+        parent=f"projects/{project}/locations/{region}",
+        batch_id=batch_id,
+        batch=batch,
     )
 
 
@@ -35,10 +35,10 @@ def poll_batch_job(
     run_time = 0
     while state in _BATCH_RUNNING_STATES and run_time < timeout:
         time.sleep(1)
-        response = job_client.get_batch(  # type: ignore
-            request=GetBatchRequest(name=batch_name),  # type: ignore
+        response = job_client.get_batch(
+            request=GetBatchRequest(name=batch_name),
         )
-        run_time = datetime.now().timestamp() - response.create_time.timestamp()  # type: ignore
+        run_time = datetime.now().timestamp() - response.create_time.timestamp()
         state = response.state
     if not response:
         raise ValueError("No response from Dataproc")
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index dc5cf6e17..a1aaf17eb 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -22,7 +22,7 @@
 from dbt.adapters.contracts.relation import RelationConfig
 
 import dbt_common.exceptions.base
-from dbt.adapters.base import (  # type: ignore
+from dbt.adapters.base import (
     AdapterConfig,
     BaseAdapter,
     BaseRelation,
@@ -33,11 +33,15 @@
     available,
 )
 from dbt.adapters.base.impl import FreshnessResponse
-from dbt.adapters.cache import _make_ref_key_dict  # type: ignore
+from dbt.adapters.cache import _make_ref_key_dict
 from dbt.adapters.capability import Capability, CapabilityDict, CapabilitySupport, Support
 from dbt.adapters.contracts.connection import AdapterResponse
 from dbt.adapters.contracts.macros import MacroResolverProtocol
-from dbt_common.contracts.constraints import ColumnLevelConstraint, ConstraintType, ModelLevelConstraint  # type: ignore
+from dbt_common.contracts.constraints import (
+    ColumnLevelConstraint,
+    ConstraintType,
+    ModelLevelConstraint,
+)
 from dbt_common.dataclass_schema import dbtClassMixin
 from dbt.adapters.events.logging import AdapterLogger
 from dbt_common.events.functions import fire_event
@@ -163,7 +167,7 @@ def is_cancelable(cls) -> bool:
         return False
 
     def drop_relation(self, relation: BigQueryRelation) -> None:
-        is_cached = self._schema_is_cached(relation.database, relation.schema)  # type: ignore[arg-type]
+        is_cached = self._schema_is_cached(relation.database, relation.schema)
         if is_cached:
             self.cache_dropped(relation)
 
@@ -258,7 +262,7 @@ def add_time_ingestion_partition_column(self, partition_by, columns) -> List[Big
         )
         return columns
 
-    def expand_column_types(self, goal: BigQueryRelation, current: BigQueryRelation) -> None:  # type: ignore[override]
+    def expand_column_types(self, goal: BigQueryRelation, current: BigQueryRelation) -> None:
         # This is a no-op on BigQuery
         pass
 
@@ -323,7 +327,7 @@ def get_relation(
     # TODO: the code below is copy-pasted from SQLAdapter.create_schema. Is there a better way?
     def create_schema(self, relation: BigQueryRelation) -> None:
         # use SQL 'create schema'
-        relation = relation.without_identifier()  # type: ignore
+        relation = relation.without_identifier()
 
         fire_event(SchemaCreation(relation=_make_ref_key_dict(relation)))
         kwargs = {
@@ -410,7 +414,7 @@ def _agate_to_schema(
         for idx, col_name in enumerate(agate_table.column_names):
             inferred_type = self.convert_agate_type(agate_table, idx)
             type_ = column_override.get(col_name, inferred_type)
-            bq_schema.append(SchemaField(col_name, type_))  # type: ignore[arg-type]
+            bq_schema.append(SchemaField(col_name, type_))
         return bq_schema
 
     @available.parse(lambda *a, **k: "")
@@ -736,8 +740,8 @@ def _get_catalog_schemas(self, relation_config: Iterable[RelationConfig]) -> Sch
         for candidate, schemas in candidates.items():
             database = candidate.database
             if database not in db_schemas:
-                db_schemas[database] = set(self.list_schemas(database))  # type: ignore[index]
-            if candidate.schema in db_schemas[database]:  # type: ignore[index]
+                db_schemas[database] = set(self.list_schemas(database))
+            if candidate.schema in db_schemas[database]:
                 result[candidate] = schemas
             else:
                 logger.debug(
@@ -844,7 +848,7 @@ def describe_relation(
         return None
 
     @available.parse_none
-    def grant_access_to(self, entity, entity_type, role, grant_target_dict):
+    def grant_access_to(self, entity, entity_type, role, grant_target_dict) -> None:
         """
         Given an entity, grants it access to a dataset.
         """
@@ -873,7 +877,7 @@ def get_dataset_location(self, relation):
         dataset = client.get_dataset(dataset_ref)
         return dataset.location
 
-    def get_rows_different_sql(  # type: ignore[override]
+    def get_rows_different_sql(
         self,
         relation_a: BigQueryRelation,
         relation_b: BigQueryRelation,
@@ -921,7 +925,7 @@ def run_sql_for_tests(self, sql, fetch, conn=None):
             return list(res)
 
     def generate_python_submission_response(self, submission_result) -> BigQueryAdapterResponse:
-        return BigQueryAdapterResponse(_message="OK")  # type: ignore[call-arg]
+        return BigQueryAdapterResponse(_message="OK")
 
     @property
     def default_python_submission_method(self) -> str:
@@ -961,7 +965,7 @@ def render_raw_columns_constraints(cls, raw_columns: Dict[str, Dict[str, Any]])
 
     @classmethod
     def render_column_constraint(cls, constraint: ColumnLevelConstraint) -> Optional[str]:
-        c = super().render_column_constraint(constraint)  # type: ignore
+        c = super().render_column_constraint(constraint)
         if (
             constraint.type == ConstraintType.primary_key
             or constraint.type == ConstraintType.foreign_key
@@ -971,7 +975,7 @@ def render_column_constraint(cls, constraint: ColumnLevelConstraint) -> Optional
 
     @classmethod
     def render_model_constraint(cls, constraint: ModelLevelConstraint) -> Optional[str]:
-        c = super().render_model_constraint(constraint)  # type: ignore
+        c = super().render_model_constraint(constraint)
         if (
             constraint.type == ConstraintType.primary_key
             or constraint.type == ConstraintType.foreign_key
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 065c65d8b..368ed9d07 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -9,7 +9,7 @@
 from dbt.adapters.bigquery import BigQueryConnectionManager, BigQueryCredentials
 from google.api_core import retry
 from google.api_core.client_options import ClientOptions
-from google.cloud import storage, dataproc_v1  # type: ignore
+from google.cloud import storage, dataproc_v1
 from google.cloud.dataproc_v1.types.batches import Batch
 
 from dbt.adapters.bigquery.dataproc.batch import (
@@ -89,7 +89,7 @@ def _get_job_client(self) -> dataproc_v1.JobControllerClient:
             raise ValueError(
                 "Need to supply dataproc_cluster_name in profile or config to submit python job with cluster submission method"
             )
-        return dataproc_v1.JobControllerClient(  # type: ignore
+        return dataproc_v1.JobControllerClient(
             client_options=self.client_options, credentials=self.GoogleCredentials
         )
 
@@ -105,7 +105,7 @@ def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
                 "main_python_file_uri": self.gcs_location,
             },
         }
-        operation = self.job_client.submit_job_as_operation(  # type: ignore
+        operation = self.job_client.submit_job_as_operation(
             request={
                 "project_id": self.credential.execution_project,
                 "region": self.credential.dataproc_region,
@@ -138,13 +138,13 @@ def _submit_dataproc_job(self) -> Batch:
             batch_id=batch_id,
             region=self.credential.dataproc_region,  # type: ignore
             project=self.credential.execution_project,  # type: ignore
-        )  # type: ignore
+        )
         # make the request
-        self.job_client.create_batch(request=request)  # type: ignore
+        self.job_client.create_batch(request=request)
         return poll_batch_job(
             parent=request.parent,
             batch_id=batch_id,
-            job_client=self.job_client,  # type: ignore
+            job_client=self.job_client,
             timeout=self.timeout,
         )
         # there might be useful results here that we can parse and return
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index 086b4a2aa..0e2c17670 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -78,7 +78,7 @@ def dataset(self):
     def materialized_view_from_relation_config(
         cls, relation_config: RelationConfig
     ) -> BigQueryMaterializedViewConfig:
-        return BigQueryMaterializedViewConfig.from_relation_config(relation_config)  # type: ignore
+        return BigQueryMaterializedViewConfig.from_relation_config(relation_config)
 
     @classmethod
     def materialized_view_config_changeset(
diff --git a/dbt/adapters/bigquery/relation_configs/_base.py b/dbt/adapters/bigquery/relation_configs/_base.py
index 45e29b99f..8bc861587 100644
--- a/dbt/adapters/bigquery/relation_configs/_base.py
+++ b/dbt/adapters/bigquery/relation_configs/_base.py
@@ -32,7 +32,7 @@ def quote_policy(cls) -> Policy:
     def from_relation_config(cls, relation_config: RelationConfig) -> Self:
         relation_config_dict = cls.parse_relation_config(relation_config)
         relation = cls.from_dict(relation_config_dict)
-        return relation  # type: ignore
+        return relation
 
     @classmethod
     def parse_relation_config(cls, relation_config: RelationConfig) -> Dict:
@@ -44,7 +44,7 @@ def parse_relation_config(cls, relation_config: RelationConfig) -> Dict:
     def from_bq_table(cls, table: BigQueryTable) -> Self:
         relation_config = cls.parse_bq_table(table)
         relation = cls.from_dict(relation_config)
-        return relation  # type: ignore
+        return relation
 
     @classmethod
     def parse_bq_table(cls, table: BigQueryTable) -> Dict:
diff --git a/dbt/adapters/bigquery/relation_configs/_cluster.py b/dbt/adapters/bigquery/relation_configs/_cluster.py
index 53092cb35..b3dbaf2e9 100644
--- a/dbt/adapters/bigquery/relation_configs/_cluster.py
+++ b/dbt/adapters/bigquery/relation_configs/_cluster.py
@@ -25,13 +25,13 @@ class BigQueryClusterConfig(BigQueryBaseRelationConfig):
     @classmethod
     def from_dict(cls, config_dict: Dict[str, Any]) -> Self:
         kwargs_dict = {"fields": config_dict.get("fields")}
-        return super().from_dict(kwargs_dict)  # type: ignore
+        return super().from_dict(kwargs_dict)
 
     @classmethod
     def parse_relation_config(cls, relation_config: RelationConfig) -> Dict[str, Any]:
         config_dict = {}
 
-        if cluster_by := relation_config.config.extra.get("cluster_by"):  # type: ignore
+        if cluster_by := relation_config.config.extra.get("cluster_by"):
             # users may input a single field as a string
             if isinstance(cluster_by, str):
                 cluster_by = [cluster_by]
@@ -40,7 +40,7 @@ def parse_relation_config(cls, relation_config: RelationConfig) -> Dict[str, Any
         return config_dict
 
     @classmethod
-    def parse_bq_table(cls, table: BigQueryTable) -> Dict[str, Any]:  # type: ignore
+    def parse_bq_table(cls, table: BigQueryTable) -> Dict[str, Any]:
         config_dict = {"fields": frozenset(table.clustering_fields)}
         return config_dict
 
diff --git a/dbt/adapters/bigquery/relation_configs/_materialized_view.py b/dbt/adapters/bigquery/relation_configs/_materialized_view.py
index 81ca6b3de..7c63ba3bc 100644
--- a/dbt/adapters/bigquery/relation_configs/_materialized_view.py
+++ b/dbt/adapters/bigquery/relation_configs/_materialized_view.py
@@ -61,7 +61,7 @@ def from_dict(cls, config_dict: Dict[str, Any]) -> "BigQueryMaterializedViewConf
         if cluster := config_dict.get("cluster"):
             kwargs_dict.update({"cluster": BigQueryClusterConfig.from_dict(cluster)})
 
-        materialized_view: "BigQueryMaterializedViewConfig" = super().from_dict(kwargs_dict)  # type: ignore
+        materialized_view: "BigQueryMaterializedViewConfig" = super().from_dict(kwargs_dict)
         return materialized_view
 
     @classmethod
diff --git a/dbt/adapters/bigquery/relation_configs/_options.py b/dbt/adapters/bigquery/relation_configs/_options.py
index f0272df08..7fd8797df 100644
--- a/dbt/adapters/bigquery/relation_configs/_options.py
+++ b/dbt/adapters/bigquery/relation_configs/_options.py
@@ -103,13 +103,13 @@ def formatted_setting(name: str) -> Any:
         if kwargs_dict["enable_refresh"] is False:
             kwargs_dict.update({"refresh_interval_minutes": None, "max_staleness": None})
 
-        options: Self = super().from_dict(kwargs_dict)  # type: ignore
+        options: Self = super().from_dict(kwargs_dict)
         return options
 
     @classmethod
     def parse_relation_config(cls, relation_config: RelationConfig) -> Dict[str, Any]:
         config_dict = {
-            option: relation_config.config.extra.get(option)  # type: ignore
+            option: relation_config.config.extra.get(option)
             for option in [
                 "enable_refresh",
                 "refresh_interval_minutes",
@@ -122,13 +122,11 @@ def parse_relation_config(cls, relation_config: RelationConfig) -> Dict[str, Any
         }
 
         # update dbt-specific versions of these settings
-        if hours_to_expiration := relation_config.config.extra.get(  # type: ignore
-            "hours_to_expiration"
-        ):  # type: ignore
+        if hours_to_expiration := relation_config.config.extra.get("hours_to_expiration"):
             config_dict.update(
                 {"expiration_timestamp": datetime.now() + timedelta(hours=hours_to_expiration)}
             )
-        if not relation_config.config.persist_docs:  # type: ignore
+        if not relation_config.config.persist_docs:
             del config_dict["description"]
 
         return config_dict
diff --git a/dbt/adapters/bigquery/relation_configs/_partition.py b/dbt/adapters/bigquery/relation_configs/_partition.py
index 555aa3664..e1a5ac171 100644
--- a/dbt/adapters/bigquery/relation_configs/_partition.py
+++ b/dbt/adapters/bigquery/relation_configs/_partition.py
@@ -111,7 +111,7 @@ def parse_model_node(cls, relation_config: RelationConfig) -> Dict[str, Any]:
             This doesn't currently collect `time_ingestion_partitioning` and `copy_partitions`
             because this was built for materialized views, which do not support those settings.
         """
-        config_dict: Dict[str, Any] = relation_config.config.extra.get("partition_by")  # type: ignore
+        config_dict: Dict[str, Any] = relation_config.config.extra.get("partition_by")
         if "time_ingestion_partitioning" in config_dict:
             del config_dict["time_ingestion_partitioning"]
         if "copy_partitions" in config_dict:
diff --git a/dev-requirements.txt b/dev-requirements.txt
index a81a841f0..d8033ac55 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -1,33 +1,21 @@
 # install latest changes in dbt-core
-# TODO: how to automate switching from develop to version branches?
-git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
-git+https://github.com/dbt-labs/dbt-common.git
 git+https://github.com/dbt-labs/dbt-adapters.git
 git+https://github.com/dbt-labs/dbt-adapters.git#subdirectory=dbt-tests-adapter
-# if version 1.x or greater -> pin to major version
-# if version 0.x -> pin to minor
-black>=24.3
-bumpversion~=0.6.0
-click~=8.1
+git+https://github.com/dbt-labs/dbt-common.git
+git+https://github.com/dbt-labs/dbt-core.git#subdirectory=core
+
+# dev
 ddtrace==2.3.0
-flake8~=7.0
-flaky~=3.8
-freezegun~=1.4
-ipdb~=0.13.13
-mypy==1.8.0  # patch updates have historically introduced breaking changes
-pip-tools~=7.3
-pre-commit==3.7.0;python_version >="3.9"
-pre-commit==3.5.0;python_version <"3.9"
-pre-commit-hooks~=4.6
+pre-commit~=3.7.0;python_version>="3.9"
+pre-commit~=3.5.0;python_version<"3.9"
 pytest~=7.4
 pytest-csv~=3.0
 pytest-dotenv~=0.5.2
 pytest-logbook~=1.2
 pytest-xdist~=3.6
-pytz~=2023.3
 tox~=4.11
-types-pytz~=2023.3
-types-protobuf~=4.24
-types-requests~=2.31
+
+# build
+bumpversion~=0.6.0
 twine~=5.1
-wheel~=0.42
+wheel~=0.43
diff --git a/mypy.ini b/mypy.ini
index b111482fc..247a47fec 100644
--- a/mypy.ini
+++ b/mypy.ini
@@ -1,4 +1,2 @@
 [mypy]
 mypy_path = third-party-stubs/
-namespace_packages = True
-exclude = third-party-stubs/*
diff --git a/tests/functional/python_model_tests/test_list_inference.py b/tests/functional/python_model_tests/test_list_inference.py
index 143a61e88..88b1c4fa5 100644
--- a/tests/functional/python_model_tests/test_list_inference.py
+++ b/tests/functional/python_model_tests/test_list_inference.py
@@ -9,6 +9,7 @@
 
 When the regression was first reported, `files.MULTI_RECORD` failed while the other models passed.
 """
+
 from dbt.tests.util import run_dbt_and_capture
 import pytest
 
diff --git a/tests/unit/utils.py b/tests/unit/utils.py
index 88b09ce60..633b6d565 100644
--- a/tests/unit/utils.py
+++ b/tests/unit/utils.py
@@ -2,6 +2,7 @@
 Note that all imports should be inside the functions to avoid import/mocking
 issues.
 """
+
 import string
 import os
 from unittest import mock

From 3839953a77de8b59c3830d9b2642d447de971cc9 Mon Sep 17 00:00:00 2001
From: Michelle Ark <MichelleArk@users.noreply.github.com>
Date: Wed, 31 Jul 2024 20:45:41 -0400
Subject: [PATCH 827/860] Add cancel (#1251)

* Add query cancellation.

* clean up merge + linting

* Add back mp_context

* generating a fresh job_id for every _query_and_results call

* add cancellation test

* add cancellation test

* add seed cancellation

* remove type ignore

* add changie

* use defaultdict to simplify code

---------

Co-authored-by: Daniel Cole <daniel.cole@shopify.com>
Co-authored-by: Jeremy Cohen <jeremy@dbtlabs.com>
Co-authored-by: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Co-authored-by: Colin <colin.rogers@dbtlabs.com>
---
 .../unreleased/Features-20240730-135911.yaml  |   6 +
 dbt/adapters/bigquery/connections.py          |  74 ++++++++--
 dbt/adapters/bigquery/impl.py                 |   7 +-
 tests/functional/test_cancel.py               | 127 ++++++++++++++++++
 tests/unit/test_bigquery_adapter.py           |  22 +--
 .../unit/test_bigquery_connection_manager.py  |   6 +-
 6 files changed, 213 insertions(+), 29 deletions(-)
 create mode 100644 .changes/unreleased/Features-20240730-135911.yaml
 create mode 100644 tests/functional/test_cancel.py

diff --git a/.changes/unreleased/Features-20240730-135911.yaml b/.changes/unreleased/Features-20240730-135911.yaml
new file mode 100644
index 000000000..52868c2ee
--- /dev/null
+++ b/.changes/unreleased/Features-20240730-135911.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Add support for cancelling queries on keyboard interrupt
+time: 2024-07-30T13:59:11.585452-07:00
+custom:
+  Author: d-cole MichelleArk colin-rogers-dbt
+  Issue: "917"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 0d57d22c3..cdd9d17dc 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -1,17 +1,17 @@
+from collections import defaultdict
 from concurrent.futures import TimeoutError
 import json
 import re
 from contextlib import contextmanager
 from dataclasses import dataclass, field
-
-from dbt_common.invocation import get_invocation_id
-
-from dbt_common.events.contextvars import get_node_info
+import uuid
 from mashumaro.helper import pass_through
 
 from functools import lru_cache
 from requests.exceptions import ConnectionError
-from typing import Optional, Any, Dict, Tuple, TYPE_CHECKING
+
+from multiprocessing.context import SpawnContext
+from typing import Optional, Any, Dict, Tuple, Hashable, List, TYPE_CHECKING
 
 import google.auth
 import google.auth.exceptions
@@ -24,19 +24,25 @@
     service_account as GoogleServiceAccountCredentials,
 )
 
-from dbt.adapters.bigquery import gcloud
-from dbt.adapters.contracts.connection import ConnectionState, AdapterResponse, Credentials
+from dbt_common.events.contextvars import get_node_info
+from dbt_common.events.functions import fire_event
 from dbt_common.exceptions import (
     DbtRuntimeError,
     DbtConfigError,
+    DbtDatabaseError,
+)
+from dbt_common.invocation import get_invocation_id
+from dbt.adapters.bigquery import gcloud
+from dbt.adapters.contracts.connection import (
+    ConnectionState,
+    AdapterResponse,
+    Credentials,
+    AdapterRequiredConfig,
 )
-
-from dbt_common.exceptions import DbtDatabaseError
 from dbt.adapters.exceptions.connection import FailedToConnectError
 from dbt.adapters.base import BaseConnectionManager
 from dbt.adapters.events.logging import AdapterLogger
 from dbt.adapters.events.types import SQLQuery
-from dbt_common.events.functions import fire_event
 from dbt.adapters.bigquery import __version__ as dbt_version
 from dbt.adapters.bigquery.utility import is_base64, base64_to_string
 
@@ -231,6 +237,10 @@ class BigQueryConnectionManager(BaseConnectionManager):
     DEFAULT_INITIAL_DELAY = 1.0  # Seconds
     DEFAULT_MAXIMUM_DELAY = 3.0  # Seconds
 
+    def __init__(self, profile: AdapterRequiredConfig, mp_context: SpawnContext):
+        super().__init__(profile, mp_context)
+        self.jobs_by_thread: Dict[Hashable, List[str]] = defaultdict(list)
+
     @classmethod
     def handle_error(cls, error, message):
         error_msg = "\n".join([item["message"] for item in error.errors])
@@ -284,11 +294,31 @@ def exception_handler(self, sql):
                 exc_message = exc_message.split(BQ_QUERY_JOB_SPLIT)[0].strip()
             raise DbtRuntimeError(exc_message)
 
-    def cancel_open(self) -> None:
-        pass
+    def cancel_open(self):
+        names = []
+        this_connection = self.get_if_exists()
+        with self.lock:
+            for thread_id, connection in self.thread_connections.items():
+                if connection is this_connection:
+                    continue
+                if connection.handle is not None and connection.state == ConnectionState.OPEN:
+                    client = connection.handle
+                    for job_id in self.jobs_by_thread.get(thread_id, []):
+
+                        def fn():
+                            return client.cancel_job(job_id)
+
+                        self._retry_and_handle(msg=f"Cancel job: {job_id}", conn=connection, fn=fn)
+
+                    self.close(connection)
+
+                if connection.name is not None:
+                    names.append(connection.name)
+        return names
 
     @classmethod
     def close(cls, connection):
+        connection.handle.close()
         connection.state = ConnectionState.CLOSED
 
         return connection
@@ -452,6 +482,18 @@ def get_labels_from_query_comment(cls):
 
         return {}
 
+    def generate_job_id(self) -> str:
+        # Generating a fresh job_id for every _query_and_results call to avoid job_id reuse.
+        # Generating a job id instead of persisting a BigQuery-generated one after client.query is called.
+        # Using BigQuery's job_id can lead to a race condition if a job has been started and a termination
+        # is sent before the job_id was stored, leading to a failure to cancel the job.
+        # By predetermining job_ids (uuid4), we can persist the job_id before the job has been kicked off.
+        # Doing this, the race condition only leads to attempting to cancel a job that doesn't exist.
+        job_id = str(uuid.uuid4())
+        thread_id = self.get_thread_identifier()
+        self.jobs_by_thread[thread_id].append(job_id)
+        return job_id
+
     def raw_execute(
         self,
         sql,
@@ -488,10 +530,13 @@ def raw_execute(
         job_execution_timeout = self.get_job_execution_timeout_seconds(conn)
 
         def fn():
+            job_id = self.generate_job_id()
+
             return self._query_and_results(
                 client,
                 sql,
                 job_params,
+                job_id,
                 job_creation_timeout=job_creation_timeout,
                 job_execution_timeout=job_execution_timeout,
                 limit=limit,
@@ -731,6 +776,7 @@ def _query_and_results(
         client,
         sql,
         job_params,
+        job_id,
         job_creation_timeout=None,
         job_execution_timeout=None,
         limit: Optional[int] = None,
@@ -738,7 +784,9 @@ def _query_and_results(
         """Query the client and wait for results."""
         # Cannot reuse job_config if destination is set and ddl is used
         job_config = google.cloud.bigquery.QueryJobConfig(**job_params)
-        query_job = client.query(query=sql, job_config=job_config, timeout=job_creation_timeout)
+        query_job = client.query(
+            query=sql, job_config=job_config, job_id=job_id, timeout=job_creation_timeout
+        )
         if (
             query_job.location is not None
             and query_job.job_id is not None
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index a1aaf17eb..0b49c0373 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -164,7 +164,7 @@ def date_function(cls) -> str:
 
     @classmethod
     def is_cancelable(cls) -> bool:
-        return False
+        return True
 
     def drop_relation(self, relation: BigQueryRelation) -> None:
         is_cached = self._schema_is_cached(relation.database, relation.schema)
@@ -693,8 +693,11 @@ def load_dataframe(
         load_config.skip_leading_rows = 1
         load_config.schema = bq_schema
         load_config.field_delimiter = field_delimiter
+        job_id = self.connections.generate_job_id()
         with open(agate_table.original_abspath, "rb") as f:  # type: ignore
-            job = client.load_table_from_file(f, table_ref, rewind=True, job_config=load_config)
+            job = client.load_table_from_file(
+                f, table_ref, rewind=True, job_config=load_config, job_id=job_id
+            )
 
         timeout = self.connections.get_job_execution_timeout_seconds(conn) or 300
         with self.connections.exception_handler("LOAD TABLE"):
diff --git a/tests/functional/test_cancel.py b/tests/functional/test_cancel.py
new file mode 100644
index 000000000..50306a6ae
--- /dev/null
+++ b/tests/functional/test_cancel.py
@@ -0,0 +1,127 @@
+import time
+
+import os
+import signal
+import subprocess
+
+import pytest
+
+from dbt.tests.util import get_connection
+
+_SEED_CSV = """
+id, name, astrological_sign, moral_alignment
+1, Alice, Aries, Lawful Good
+2, Bob, Taurus, Neutral Good
+3, Thaddeus, Gemini, Chaotic Neutral
+4, Zebulon, Cancer, Lawful Evil
+5, Yorick, Leo, True Neutral
+6, Xavier, Virgo, Chaotic Evil
+7, Wanda, Libra, Lawful Neutral
+"""
+
+_LONG_RUNNING_MODEL_SQL = """
+    {{ config(materialized='table') }}
+    with array_1 as (
+    select generated_ids from UNNEST(GENERATE_ARRAY(1, 200000)) AS generated_ids
+    ),
+    array_2 as (
+    select generated_ids from UNNEST(GENERATE_ARRAY(2, 200000)) AS generated_ids
+    )
+
+    SELECT array_1.generated_ids
+    FROM array_1
+    LEFT JOIN array_1 as jnd on 1=1
+    LEFT JOIN array_2 as jnd2 on 1=1
+    LEFT JOIN array_1 as jnd3 on jnd3.generated_ids >= jnd2.generated_ids
+"""
+
+
+def _get_info_schema_jobs_query(project_id, dataset_id, table_id):
+    """
+    Running this query requires roles/bigquery.resourceViewer on the project,
+    see: https://cloud.google.com/bigquery/docs/information-schema-jobs#required_role
+    :param project_id:
+    :param dataset_id:
+    :param table_id:
+    :return: a single job id that matches the model we tried to create and was cancelled
+    """
+    return f"""
+        SELECT job_id
+        FROM `region-us`.`INFORMATION_SCHEMA.JOBS_BY_PROJECT`
+        WHERE creation_time > TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 5 HOUR)
+        AND statement_type = 'CREATE_TABLE_AS_SELECT'
+        AND state = 'DONE'
+        AND job_id IS NOT NULL
+        AND project_id = '{project_id}'
+        AND error_result.reason = 'stopped'
+        AND error_result.message = 'Job execution was cancelled: User requested cancellation'
+        AND destination_table.table_id = '{table_id}'
+        AND destination_table.dataset_id = '{dataset_id}'
+    """
+
+
+def _run_dbt_in_subprocess(project, dbt_command):
+    os.chdir(project.project_root)
+    run_dbt_process = subprocess.Popen(
+        [
+            "dbt",
+            dbt_command,
+            "--profiles-dir",
+            project.profiles_dir,
+            "--project-dir",
+            project.project_root,
+        ],
+        stdout=subprocess.PIPE,
+        stderr=subprocess.PIPE,
+        shell=False,
+    )
+    std_out_log = ""
+    while True:
+        std_out_line = run_dbt_process.stdout.readline().decode("utf-8")
+        std_out_log += std_out_line
+        if std_out_line != "":
+            print(std_out_line)
+            if "1 of 1 START" in std_out_line:
+                time.sleep(1)
+                run_dbt_process.send_signal(signal.SIGINT)
+
+        if run_dbt_process.poll():
+            break
+
+    return std_out_log
+
+
+def _get_job_id(project, table_name):
+    # Because we run this in a subprocess we have to actually call Bigquery and look up the job id
+    with get_connection(project.adapter):
+        job_id = project.run_sql(
+            _get_info_schema_jobs_query(project.database, project.test_schema, table_name)
+        )
+
+    return job_id
+
+
+class TestBigqueryCancelsQueriesOnKeyboardInterrupt:
+    @pytest.fixture(scope="class", autouse=True)
+    def models(self):
+        return {
+            "model.sql": _LONG_RUNNING_MODEL_SQL,
+        }
+
+    @pytest.fixture(scope="class", autouse=True)
+    def seeds(self):
+        return {
+            "seed.csv": _SEED_CSV,
+        }
+
+    def test_bigquery_cancels_queries_for_model_on_keyboard_interrupt(self, project):
+        std_out_log = _run_dbt_in_subprocess(project, "run")
+
+        assert "CANCEL query model.test.model" in std_out_log
+        assert len(_get_job_id(project, "model")) == 1
+
+    def test_bigquery_cancels_queries_for_seed_on_keyboard_interrupt(self, project):
+        std_out_log = _run_dbt_in_subprocess(project, "seed")
+
+        assert "CANCEL query seed.test.seed" in std_out_log
+        # we can't assert the job id since we can't kill the seed process fast enough to cancel it
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index 19d9dbd08..a922525fd 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -32,6 +32,7 @@
     inject_adapter,
     TestAdapterConversions,
     load_internal_manifest_macros,
+    mock_connection,
 )
 
 
@@ -368,23 +369,22 @@ def test_acquire_connection_maximum_bytes_billed(self, mock_open_connection):
 
     def test_cancel_open_connections_empty(self):
         adapter = self.get_adapter("oauth")
-        self.assertEqual(adapter.cancel_open_connections(), None)
+        self.assertEqual(len(list(adapter.cancel_open_connections())), 0)
 
     def test_cancel_open_connections_master(self):
         adapter = self.get_adapter("oauth")
-        adapter.connections.thread_connections[0] = object()
-        self.assertEqual(adapter.cancel_open_connections(), None)
+        key = adapter.connections.get_thread_identifier()
+        adapter.connections.thread_connections[key] = mock_connection("master")
+        self.assertEqual(len(list(adapter.cancel_open_connections())), 0)
 
     def test_cancel_open_connections_single(self):
         adapter = self.get_adapter("oauth")
-        adapter.connections.thread_connections.update(
-            {
-                0: object(),
-                1: object(),
-            }
-        )
-        # actually does nothing
-        self.assertEqual(adapter.cancel_open_connections(), None)
+        master = mock_connection("master")
+        model = mock_connection("model")
+        key = adapter.connections.get_thread_identifier()
+
+        adapter.connections.thread_connections.update({key: master, 1: model})
+        self.assertEqual(len(list(adapter.cancel_open_connections())), 1)
 
     @patch("dbt.adapters.bigquery.impl.google.auth.default")
     @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
diff --git a/tests/unit/test_bigquery_connection_manager.py b/tests/unit/test_bigquery_connection_manager.py
index 9dc8fe219..d09cb1635 100644
--- a/tests/unit/test_bigquery_connection_manager.py
+++ b/tests/unit/test_bigquery_connection_manager.py
@@ -104,18 +104,18 @@ def test_drop_dataset(self):
 
     @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
     def test_query_and_results(self, mock_bq):
-        self.mock_client.query = Mock(return_value=Mock(state="DONE"))
         self.connections._query_and_results(
             self.mock_client,
             "sql",
             {"job_param_1": "blah"},
+            job_id=1,
             job_creation_timeout=15,
-            job_execution_timeout=3,
+            job_execution_timeout=100,
         )
 
         mock_bq.QueryJobConfig.assert_called_once()
         self.mock_client.query.assert_called_once_with(
-            query="sql", job_config=mock_bq.QueryJobConfig(), timeout=15
+            query="sql", job_config=mock_bq.QueryJobConfig(), job_id=1, timeout=15
         )
 
     def test_copy_bq_table_appends(self):

From d1976bbb412a5dec5ef41dc9ed0bc703cd909abd Mon Sep 17 00:00:00 2001
From: Gerda Shank <gerda@dbtlabs.com>
Date: Thu, 1 Aug 2024 09:57:55 -0400
Subject: [PATCH 828/860] Integration workflow update to support all-in-one
 adapter testing (#1309)

---
 .github/scripts/update_dbt_core_branch.sh     | 20 -------
 .../scripts/update_dev_dependency_branches.sh | 21 +++++++
 .github/workflows/integration.yml             | 60 +++++++++++++++----
 3 files changed, 71 insertions(+), 30 deletions(-)
 delete mode 100755 .github/scripts/update_dbt_core_branch.sh
 create mode 100755 .github/scripts/update_dev_dependency_branches.sh

diff --git a/.github/scripts/update_dbt_core_branch.sh b/.github/scripts/update_dbt_core_branch.sh
deleted file mode 100755
index d28a40c35..000000000
--- a/.github/scripts/update_dbt_core_branch.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/bin/bash -e
-set -e
-
-git_branch=$1
-target_req_file="dev-requirements.txt"
-core_req_sed_pattern="s|dbt-core.git.*#egg=dbt-core|dbt-core.git@${git_branch}#egg=dbt-core|g"
-postgres_req_sed_pattern="s|dbt-core.git.*#egg=dbt-postgres|dbt-core.git@${git_branch}#egg=dbt-postgres|g"
-tests_req_sed_pattern="s|dbt-core.git.*#egg=dbt-tests|dbt-core.git@${git_branch}#egg=dbt-tests|g"
-if [[ "$OSTYPE" == darwin* ]]; then
- # mac ships with a different version of sed that requires a delimiter arg
- sed -i "" "$core_req_sed_pattern" $target_req_file
- sed -i "" "$postgres_req_sed_pattern" $target_req_file
- sed -i "" "$tests_req_sed_pattern" $target_req_file
-else
- sed -i "$core_req_sed_pattern" $target_req_file
- sed -i "$postgres_req_sed_pattern" $target_req_file
- sed -i "$tests_req_sed_pattern" $target_req_file
-fi
-core_version=$(curl "https://raw.githubusercontent.com/dbt-labs/dbt-core/${git_branch}/core/dbt/version.py" | grep "__version__ = *"|cut -d'=' -f2)
-bumpversion --allow-dirty --new-version "$core_version" major
diff --git a/.github/scripts/update_dev_dependency_branches.sh b/.github/scripts/update_dev_dependency_branches.sh
new file mode 100755
index 000000000..022df6a8a
--- /dev/null
+++ b/.github/scripts/update_dev_dependency_branches.sh
@@ -0,0 +1,21 @@
+#!/bin/bash -e
+set -e
+
+
+dbt_adapters_branch=$1
+dbt_core_branch=$2
+dbt_common_branch=$3
+target_req_file="dev-requirements.txt"
+core_req_sed_pattern="s|dbt-core.git.*#egg=dbt-core|dbt-core.git@${dbt_core_branch}#egg=dbt-core|g"
+adapters_req_sed_pattern="s|dbt-adapters.git|dbt-adapters.git@${dbt_adapters_branch}|g"
+common_req_sed_pattern="s|dbt-common.git|dbt-common.git@${dbt_common_branch}|g"
+if [[ "$OSTYPE" == darwin* ]]; then
+ # mac ships with a different version of sed that requires a delimiter arg
+ sed -i "" "$adapters_req_sed_pattern" $target_req_file
+ sed -i "" "$core_req_sed_pattern" $target_req_file
+ sed -i "" "$common_req_sed_pattern" $target_req_file
+else
+ sed -i "$adapters_req_sed_pattern" $target_req_file
+ sed -i "$core_req_sed_pattern" $target_req_file
+ sed -i "$common_req_sed_pattern" $target_req_file
+fi
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index a9ead6cb1..ae75a4f2a 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -20,6 +20,8 @@
 
 name: Adapter Integration Tests
 
+run-name: "${{ (contains(github.event_name, 'workflow_') && inputs.name) || github.event_name }}: ${{ (contains(github.event_name, 'workflow_') &&  inputs.adapter_branch) || github.ref_name }} by @${{ github.actor }}"
+
 on:
   # pushes to release branches
   push:
@@ -34,10 +36,31 @@ on:
   # manual trigger
   workflow_dispatch:
     inputs:
-      dbt-core-branch:
-        description: "branch of dbt-core to use in dev-requirements.txt"
+      name:
+        description: "Name to associate with run (example: 'dbt-adapters-242')"
+        required: false
+        type: string
+        default: "Adapter Integration Tests"
+      adapter_branch:
+        description: "The branch of this adapter repository to use"
+        type: string
+        required: false
+        default: "main"
+      dbt_adapters_branch:
+        description: "The branch of dbt-adapters to use"
+        type: string
+        required: false
+        default: "main"
+      dbt_core_branch:
+        description: "The branch of dbt-core to use"
+        type: string
         required: false
+        default: "main"
+      dbt_common_branch:
+        description: "The branch of dbt-common to use"
         type: string
+        required: false
+        default: "main"
 
 # explicitly turn off permissions for `GITHUB_TOKEN`
 permissions: read-all
@@ -67,12 +90,19 @@ jobs:
       run-python-tests: ${{ steps.filter.outputs.bigquery-python }}
 
     steps:
-      - name: Check out the repository (non-PR)
-        if: github.event_name != 'pull_request_target'
+      - name: Check out the repository (push)
+        if: github.event_name == 'push'
         uses: actions/checkout@v4
         with:
           persist-credentials: false
 
+      - name: Check out the repository (workflow_dispatch)
+        if: github.event_name == 'workflow_dispatch'
+        uses: actions/checkout@v4
+        with:
+          persist-credentials: false
+          ref: ${{ inputs.adapter_branch }}
+
       - name: Check out the repository (PR)
         if: github.event_name == 'pull_request_target'
         uses: actions/checkout@v4
@@ -143,12 +173,19 @@ jobs:
 
     steps:
       - name: Check out the repository
-        if: github.event_name != 'pull_request_target'
+        if: github.event_name != 'pull_request_target' && github.event_name != 'workflow_dispatch'
         uses: actions/checkout@v4
         with:
           persist-credentials: false
 
-      # explicity checkout the branch for the PR,
+      - name: Check out the repository (workflow_dispatch)
+        if: github.event_name == 'workflow_dispatch'
+        uses: actions/checkout@v4
+        with:
+          persist-credentials: false
+          ref: ${{ inputs.adapter_branch }}
+
+      # explicitly checkout the branch for the PR,
       # this is necessary for the `pull_request_target` event
       - name: Check out the repository (PR)
         if: github.event_name == 'pull_request_target'
@@ -169,11 +206,14 @@ jobs:
           python -m pip --version
           tox --version
 
-      - name: Update dev_requirements.txt
-        if: inputs.dbt-core-branch != ''
+      - name: Update Adapters and Core branches (update dev_requirements.txt)
+        if: github.event_name == 'workflow_dispatch'
         run: |
-          pip install bumpversion
-          ./.github/scripts/update_dbt_core_branch.sh ${{ inputs.dbt-core-branch }}
+          ./.github/scripts/update_dev_dependency_branches.sh \
+            ${{ inputs.dbt_adapters_branch }} \
+            ${{ inputs.dbt_core_branch }} \
+            ${{ inputs.dbt_common_branch }}
+          cat dev-requirements.txt
 
       - name: Run tox (bigquery)
         if: matrix.adapter == 'bigquery'

From 71dd5753173fac24976b0d36586fe73a47b3a8ab Mon Sep 17 00:00:00 2001
From: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Thu, 1 Aug 2024 11:56:45 -0700
Subject: [PATCH 829/860] Skip cancel tests on windows  (#1314)

* Add query cancellation.

* clean up merge + linting

* Add back mp_context

* generating a fresh job_id for every _query_and_results call

* add cancellation test

* add cancellation test

* add seed cancellation

* remove type ignore

* add changie

* use defaultdict to simplify code

* skip test_cancel.py on windows

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

---------

Co-authored-by: Daniel Cole <daniel.cole@shopify.com>
Co-authored-by: Michelle Ark <MichelleArk@users.noreply.github.com>
Co-authored-by: Michelle Ark <michelle.ark@dbtlabs.com>
Co-authored-by: Jeremy Cohen <jeremy@dbtlabs.com>
---
 tests/functional/test_cancel.py | 5 +++++
 1 file changed, 5 insertions(+)

diff --git a/tests/functional/test_cancel.py b/tests/functional/test_cancel.py
index 50306a6ae..29737cbbb 100644
--- a/tests/functional/test_cancel.py
+++ b/tests/functional/test_cancel.py
@@ -1,3 +1,5 @@
+import platform
+
 import time
 
 import os
@@ -101,6 +103,9 @@ def _get_job_id(project, table_name):
     return job_id
 
 
+@pytest.mark.skipif(
+    platform.system() == "Windows", reason="running signt is unsupported on Windows."
+)
 class TestBigqueryCancelsQueriesOnKeyboardInterrupt:
     @pytest.fixture(scope="class", autouse=True)
     def models(self):

From 5ce38d81d80a2157686f6a38884e4e0883bd3ff2 Mon Sep 17 00:00:00 2001
From: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Mon, 5 Aug 2024 12:12:41 -0700
Subject: [PATCH 830/860] skip flaky seed test (#1315)

* Add query cancellation.

* clean up merge + linting

* Add back mp_context

* generating a fresh job_id for every _query_and_results call

* add cancellation test

* add cancellation test

* add seed cancellation

* remove type ignore

* add changie

* use defaultdict to simplify code

* skip test_cancel.py on windows

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* skip seed test

---------

Co-authored-by: Daniel Cole <daniel.cole@shopify.com>
Co-authored-by: Michelle Ark <MichelleArk@users.noreply.github.com>
Co-authored-by: Michelle Ark <michelle.ark@dbtlabs.com>
Co-authored-by: Jeremy Cohen <jeremy@dbtlabs.com>
---
 tests/functional/test_cancel.py | 1 +
 1 file changed, 1 insertion(+)

diff --git a/tests/functional/test_cancel.py b/tests/functional/test_cancel.py
index 29737cbbb..c026d76d6 100644
--- a/tests/functional/test_cancel.py
+++ b/tests/functional/test_cancel.py
@@ -125,6 +125,7 @@ def test_bigquery_cancels_queries_for_model_on_keyboard_interrupt(self, project)
         assert "CANCEL query model.test.model" in std_out_log
         assert len(_get_job_id(project, "model")) == 1
 
+    @pytest.mark.skip(reason="cannot reliably cancel seed queries in time")
     def test_bigquery_cancels_queries_for_seed_on_keyboard_interrupt(self, project):
         std_out_log = _run_dbt_in_subprocess(project, "seed")
 

From 21aeca34ff5cbe0807010d5477e9e989ffa2177a Mon Sep 17 00:00:00 2001
From: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Date: Fri, 16 Aug 2024 14:16:30 -0700
Subject: [PATCH 831/860] Update workflows and integration test matricies
 (#1320)

Co-authored-by: Mila Page <versusfacit@users.noreply.github.com>
---
 .github/scripts/integration-test-matrix.js | 2 +-
 .github/workflows/main.yml                 | 4 ++--
 .pre-commit-config.yaml                    | 1 +
 setup.py                                   | 1 +
 tox.ini                                    | 8 ++++----
 5 files changed, 9 insertions(+), 7 deletions(-)

diff --git a/.github/scripts/integration-test-matrix.js b/.github/scripts/integration-test-matrix.js
index 1a3136cf6..8e4b351fc 100644
--- a/.github/scripts/integration-test-matrix.js
+++ b/.github/scripts/integration-test-matrix.js
@@ -1,6 +1,6 @@
 module.exports = ({ context }) => {
   const defaultPythonVersion = "3.8";
-  const supportedPythonVersions = ["3.8", "3.9", "3.10", "3.11"];
+  const supportedPythonVersions = ["3.8", "3.9", "3.10", "3.11", "3.12"];
   const supportedAdapters = ["bigquery"];
 
   // if PR, generate matrix based on files changed and PR labels
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index a88793bde..d04b4307f 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -70,7 +70,7 @@ jobs:
     strategy:
       fail-fast: false
       matrix:
-        python-version: ['3.8', '3.9', '3.10', '3.11']
+        python-version: ['3.8', '3.9', '3.10', '3.11', '3.12']
 
     env:
       TOXENV: "unit"
@@ -175,7 +175,7 @@ jobs:
       fail-fast: false
       matrix:
         os: [ubuntu-latest, macos-12, windows-latest]
-        python-version: ['3.8', '3.9', '3.10', '3.11']
+        python-version: ['3.8', '3.9', '3.10', '3.11', '3.12']
 
     steps:
       - name: Set up Python ${{ matrix.python-version }}
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index 0739a7cc4..3d0b98d45 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -28,6 +28,7 @@ repos:
         -   --target-version=py39
         -   --target-version=py310
         -   --target-version=py311
+        -   --target-version=py312
         additional_dependencies: [flaky]
 
 -   repo: https://github.com/pycqa/flake8
diff --git a/setup.py b/setup.py
index 3f4bad228..97a5e96a4 100644
--- a/setup.py
+++ b/setup.py
@@ -73,6 +73,7 @@ def _dbt_bigquery_version() -> str:
         "Programming Language :: Python :: 3.9",
         "Programming Language :: Python :: 3.10",
         "Programming Language :: Python :: 3.11",
+        "Programming Language :: Python :: 3.12",
     ],
     python_requires=">=3.8",
 )
diff --git a/tox.ini b/tox.ini
index 30e3c5816..08cd20413 100644
--- a/tox.ini
+++ b/tox.ini
@@ -1,8 +1,8 @@
 [tox]
 skipsdist = True
-envlist = py38,py39,py310,py311
+envlist = py38,py39,py310,py311,py312
 
-[testenv:{unit,py38,py39,py310,py311,py}]
+[testenv:{unit,py38,py39,py310,py311,py312,py}]
 description = unit testing
 skip_install = true
 passenv =
@@ -13,7 +13,7 @@ deps =
   -rdev-requirements.txt
   -e.
 
-[testenv:{integration,py38,py39,py310,py311,py}-{bigquery}]
+[testenv:{integration,py38,py39,py310,py311,py312,py}-{bigquery}]
 description = adapter plugin integration testing
 skip_install = true
 passenv =
@@ -33,7 +33,7 @@ deps =
   -rdev-requirements.txt
   -e.
 
-[testenv:{python-tests,py38,py39,py310,py311,py}]
+[testenv:{python-tests,py38,py39,py310,py311,py312,py}]
 description = python integration testing
 skip_install = true
 passenv =

From 01cb67900cba7271c9881dba8ba0f6650eabe14e Mon Sep 17 00:00:00 2001
From: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Wed, 21 Aug 2024 12:55:18 -0700
Subject: [PATCH 832/860] fix failing test_cancel in 3.12 (#1322)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* Specify env in test_cancel subprocess call

* remove chdir from test_cancel.py

* pip install dev-requirements.txt in github actions worker

* experimental updates to debug in github actions

* experimental updates to debug in github actions

* specify dbt exec path

* experiment

* remove editable install

* remove tmate

* remove maxfail

* remove unnecessary install
---
 tests/functional/test_cancel.py | 3 ++-
 tox.ini                         | 2 +-
 2 files changed, 3 insertions(+), 2 deletions(-)

diff --git a/tests/functional/test_cancel.py b/tests/functional/test_cancel.py
index c026d76d6..823687b52 100644
--- a/tests/functional/test_cancel.py
+++ b/tests/functional/test_cancel.py
@@ -63,7 +63,7 @@ def _get_info_schema_jobs_query(project_id, dataset_id, table_id):
 
 
 def _run_dbt_in_subprocess(project, dbt_command):
-    os.chdir(project.project_root)
+
     run_dbt_process = subprocess.Popen(
         [
             "dbt",
@@ -76,6 +76,7 @@ def _run_dbt_in_subprocess(project, dbt_command):
         stdout=subprocess.PIPE,
         stderr=subprocess.PIPE,
         shell=False,
+        env=os.environ.copy(),
     )
     std_out_log = ""
     while True:
diff --git a/tox.ini b/tox.ini
index 08cd20413..b388dc5b3 100644
--- a/tox.ini
+++ b/tox.ini
@@ -31,7 +31,7 @@ commands =
   bigquery: {envpython} -m pytest -n auto {posargs} -vv tests/functional -k "not TestPython" --profile service_account
 deps =
   -rdev-requirements.txt
-  -e.
+  .
 
 [testenv:{python-tests,py38,py39,py310,py311,py312,py}]
 description = python integration testing

From 62d597a42b7a39800659c873b66af90129b524f3 Mon Sep 17 00:00:00 2001
From: Gerda Shank <gerda@dbtlabs.com>
Date: Mon, 26 Aug 2024 17:32:38 -0400
Subject: [PATCH 833/860] Make dev-requirements.txt look like other repos and
 work with update_dev_dependency_branches.sh (#1324)

---
 dev-requirements.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-requirements.txt b/dev-requirements.txt
index d8033ac55..34169172a 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -2,7 +2,7 @@
 git+https://github.com/dbt-labs/dbt-adapters.git
 git+https://github.com/dbt-labs/dbt-adapters.git#subdirectory=dbt-tests-adapter
 git+https://github.com/dbt-labs/dbt-common.git
-git+https://github.com/dbt-labs/dbt-core.git#subdirectory=core
+git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
 
 # dev
 ddtrace==2.3.0

From de2bca1c9030754f8fd981ae736206f05046f6f7 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Tue, 27 Aug 2024 19:47:56 -0400
Subject: [PATCH 834/860] Fix the logs upload step for integration tests

---
 .github/workflows/integration.yml | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index ae75a4f2a..0ee564bee 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -229,19 +229,19 @@ jobs:
           GCS_BUCKET: dbt-ci
         run: tox -- --ddtrace
 
-      - uses: actions/upload-artifact@v4
-        if: always()
-        with:
-          name: logs
-          path: ./logs
-          overwrite: true
-
       - name: Get current date
         if: always()
         id: date
         run: |
           echo "date=$(date +'%Y-%m-%dT%H_%M_%S')" >> $GITHUB_OUTPUT #no colons allowed for artifacts
 
+      - uses: actions/upload-artifact@v4
+        if: always()
+        with:
+          name: logs_${{ matrix.python-version }}_${{ matrix.os }}_${{ matrix.adapter }}-${{ steps.date.outputs.date }}
+          path: ./logs
+          overwrite: true
+
       - uses: actions/upload-artifact@v4
         if: always()
         with:

From 563633bcf22557d7b33d5c48a09b375deff27ca7 Mon Sep 17 00:00:00 2001
From: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Fri, 6 Sep 2024 11:46:21 -0700
Subject: [PATCH 835/860] add github.actor to concurrency controls in
 integration.yml (#1335)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* add github.actor to concurrency controls
---
 .github/workflows/integration.yml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 0ee564bee..fd22b1d9a 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -67,7 +67,7 @@ permissions: read-all
 
 # will cancel previous workflows triggered by the same event and for the same ref for PRs or same SHA otherwise
 concurrency:
-  group: ${{ github.workflow }}-${{ github.event_name }}-${{ contains(github.event_name, 'pull_request') && github.event.pull_request.head.ref || github.sha }}
+  group: ${{ github.workflow }}-${{ github.event_name }}-${{ contains(github.event_name, 'pull_request') && github.event.pull_request.head.ref || github.sha }}-${{ github.actor }}
   cancel-in-progress: true
 
 # sets default shell to bash, for all operating systems

From 4932b9679ae7af5dfefcca623cc5cf33a66c7ecc Mon Sep 17 00:00:00 2001
From: Michelle Ark <MichelleArk@users.noreply.github.com>
Date: Thu, 26 Sep 2024 01:51:21 +0100
Subject: [PATCH 836/860] Microbatch Strategy (#1334)

* poc: microbatch using merge

* update base tests

* use dynamic insert_overwrite under the hood for bigquery

* changelog entry

* clean up validation + add testing
---
 .../unreleased/Features-20240925-232238.yaml  |  6 ++
 .../macros/materializations/incremental.sql   | 15 ++++-
 .../incremental_strategy/microbatch.sql       | 28 ++++++++++
 .../incremental_strategy_fixtures.py          | 56 +++++++++++++++++++
 .../test_incremental_microbatch.py            | 55 ++++++++++++++++++
 5 files changed, 157 insertions(+), 3 deletions(-)
 create mode 100644 .changes/unreleased/Features-20240925-232238.yaml
 create mode 100644 dbt/include/bigquery/macros/materializations/incremental_strategy/microbatch.sql
 create mode 100644 tests/functional/adapter/incremental/test_incremental_microbatch.py

diff --git a/.changes/unreleased/Features-20240925-232238.yaml b/.changes/unreleased/Features-20240925-232238.yaml
new file mode 100644
index 000000000..903884196
--- /dev/null
+++ b/.changes/unreleased/Features-20240925-232238.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Add Microbatch Strategy to dbt-spark
+time: 2024-09-25T23:22:38.216277+01:00
+custom:
+  Author: michelleark
+  Issue: "1354"
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index 3908bedc2..935280d63 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -4,12 +4,16 @@
 
   {% set invalid_strategy_msg -%}
     Invalid incremental strategy provided: {{ strategy }}
-    Expected one of: 'merge', 'insert_overwrite'
+    Expected one of: 'merge', 'insert_overwrite', 'microbatch'
   {%- endset %}
-  {% if strategy not in ['merge', 'insert_overwrite'] %}
+  {% if strategy not in ['merge', 'insert_overwrite', 'microbatch'] %}
     {% do exceptions.raise_compiler_error(invalid_strategy_msg) %}
   {% endif %}
 
+  {% if strategy == 'microbatch' %}
+    {% do bq_validate_microbatch_config(config) %}
+  {% endif %}
+
   {% do return(strategy) %}
 {% endmacro %}
 
@@ -48,8 +52,13 @@
         tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions
     ) %}
 
-  {% else %} {# strategy == 'merge' #}
+  {% elif strategy == 'microbatch' %}
 
+    {% set build_sql = bq_generate_microbatch_build_sql(
+        tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions
+    ) %}
+
+  {% else %} {# strategy == 'merge' #}
     {% set build_sql = bq_generate_incremental_merge_build_sql(
         tmp_relation, target_relation, sql, unique_key, partition_by, dest_columns, tmp_relation_exists, incremental_predicates
     ) %}
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/microbatch.sql b/dbt/include/bigquery/macros/materializations/incremental_strategy/microbatch.sql
new file mode 100644
index 000000000..d4c4b7453
--- /dev/null
+++ b/dbt/include/bigquery/macros/materializations/incremental_strategy/microbatch.sql
@@ -0,0 +1,28 @@
+{% macro bq_validate_microbatch_config(config) %}
+  {% if config.get("partition_by") is none %}
+    {% set missing_partition_msg -%}
+    The 'microbatch' strategy requires a `partition_by` config.
+    {%- endset %}
+    {% do exceptions.raise_compiler_error(missing_partition_msg) %}
+  {% endif %}
+
+  {% if config.get("partition_by").granularity != config.get('batch_size') %}
+    {% set invalid_partition_by_granularity_msg -%}
+    The 'microbatch' strategy requires a `partition_by` config with the same granularity as its configured `batch_size`.
+    Got:
+      `batch_size`: {{ config.get('batch_size') }}
+      `partition_by.granularity`: {{ config.get("partition_by").granularity }}
+    {%- endset %}
+    {% do exceptions.raise_compiler_error(invalid_partition_by_granularity_msg) %}
+  {% endif %}
+{% endmacro %}
+
+{% macro bq_generate_microbatch_build_sql(
+      tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions
+) %}
+    {% set build_sql = bq_insert_overwrite_sql(
+        tmp_relation, target_relation, sql, unique_key, partition_by, partitions, dest_columns, tmp_relation_exists, copy_partitions
+    ) %}
+
+    {{ return(build_sql) }}
+{% endmacro %}
diff --git a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
index 17391b48d..02efbb6c2 100644
--- a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
+++ b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
@@ -555,3 +555,59 @@
 
 select * from data
 """.lstrip()
+
+microbatch_model_no_unique_id_sql = """
+{{ config(
+    materialized='incremental',
+    incremental_strategy='microbatch',
+    partition_by={
+      'field': 'event_time',
+      'data_type': 'timestamp',
+      'granularity': 'day'
+    },
+    event_time='event_time',
+    batch_size='day',
+    begin=modules.datetime.datetime(2020, 1, 1, 0, 0, 0)
+    )
+}}
+select * from {{ ref('input_model') }}
+"""
+
+microbatch_input_sql = """
+{{ config(materialized='table', event_time='event_time') }}
+select 1 as id, TIMESTAMP '2020-01-01 00:00:00-0' as event_time
+union all
+select 2 as id, TIMESTAMP '2020-01-02 00:00:00-0' as event_time
+union all
+select 3 as id, TIMESTAMP '2020-01-03 00:00:00-0' as event_time
+"""
+
+microbatch_model_no_partition_by_sql = """
+{{ config(
+    materialized='incremental',
+    incremental_strategy='microbatch',
+    event_time='event_time',
+    batch_size='day',
+    begin=modules.datetime.datetime(2020, 1, 1, 0, 0, 0)
+    )
+}}
+select * from {{ ref('input_model') }}
+"""
+
+
+microbatch_model_invalid_partition_by_sql = """
+{{ config(
+    materialized='incremental',
+    incremental_strategy='microbatch',
+    event_time='event_time',
+    batch_size='day',
+    begin=modules.datetime.datetime(2020, 1, 1, 0, 0, 0),
+    partition_by={
+      'field': 'event_time',
+      'data_type': 'timestamp',
+      'granularity': 'hour'
+    }
+    )
+}}
+select * from {{ ref('input_model') }}
+"""
diff --git a/tests/functional/adapter/incremental/test_incremental_microbatch.py b/tests/functional/adapter/incremental/test_incremental_microbatch.py
new file mode 100644
index 000000000..d1bbbcea3
--- /dev/null
+++ b/tests/functional/adapter/incremental/test_incremental_microbatch.py
@@ -0,0 +1,55 @@
+import os
+import pytest
+from unittest import mock
+
+from dbt.tests.util import run_dbt_and_capture
+from dbt.tests.adapter.incremental.test_incremental_microbatch import (
+    BaseMicrobatch,
+    patch_microbatch_end_time,
+)
+
+from tests.functional.adapter.incremental.incremental_strategy_fixtures import (
+    microbatch_model_no_unique_id_sql,
+    microbatch_input_sql,
+    microbatch_model_no_partition_by_sql,
+    microbatch_model_invalid_partition_by_sql,
+)
+
+
+class TestBigQueryMicrobatch(BaseMicrobatch):
+    @pytest.fixture(scope="class")
+    def microbatch_model_sql(self) -> str:
+        return microbatch_model_no_unique_id_sql
+
+
+class TestBigQueryMicrobatchMissingPartitionBy:
+    @pytest.fixture(scope="class")
+    def models(self) -> str:
+        return {
+            "microbatch.sql": microbatch_model_no_partition_by_sql,
+            "input_model.sql": microbatch_input_sql,
+        }
+
+    @mock.patch.dict(os.environ, {"DBT_EXPERIMENTAL_MICROBATCH": "True"})
+    def test_execution_failure_no_partition_by(self, project):
+        with patch_microbatch_end_time("2020-01-03 13:57:00"):
+            _, stdout = run_dbt_and_capture(["run"], expect_pass=False)
+        assert "The 'microbatch' strategy requires a `partition_by` config" in stdout
+
+
+class TestBigQueryMicrobatchInvalidPartitionByGranularity:
+    @pytest.fixture(scope="class")
+    def models(self) -> str:
+        return {
+            "microbatch.sql": microbatch_model_invalid_partition_by_sql,
+            "input_model.sql": microbatch_input_sql,
+        }
+
+    @mock.patch.dict(os.environ, {"DBT_EXPERIMENTAL_MICROBATCH": "True"})
+    def test_execution_failure_no_partition_by(self, project):
+        with patch_microbatch_end_time("2020-01-03 13:57:00"):
+            _, stdout = run_dbt_and_capture(["run"], expect_pass=False)
+        assert (
+            "The 'microbatch' strategy requires a `partition_by` config with the same granularity as its configured `batch_size`"
+            in stdout
+        )

From ae0f91c7a9dac303632eba2a99161251be6396f7 Mon Sep 17 00:00:00 2001
From: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Fri, 27 Sep 2024 15:33:24 -0700
Subject: [PATCH 837/860] update dbt-common dependency to 1.10 and dbt-adapters
 to 1.7 (#1356)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* update dbt-common dependency to 1.10 and dbt-adapters to 1.7

* run integration.yml on change to any *.py file
---
 .github/workflows/integration.yml | 1 +
 setup.py                          | 4 ++--
 2 files changed, 3 insertions(+), 2 deletions(-)

diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index fd22b1d9a..43ac4ecb3 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -128,6 +128,7 @@ jobs:
               - 'tests/**'
               - 'dev-requirements.txt'
               - '.github/**'
+              - '*.py'
 
       - name: Generate integration test matrix
         id: generate-matrix
diff --git a/setup.py b/setup.py
index 97a5e96a4..ab89a3c39 100644
--- a/setup.py
+++ b/setup.py
@@ -50,8 +50,8 @@ def _dbt_bigquery_version() -> str:
     packages=find_namespace_packages(include=["dbt", "dbt.*"]),
     include_package_data=True,
     install_requires=[
-        "dbt-common>=1.0.4,<2.0",
-        "dbt-adapters>=1.1.1,<2.0",
+        "dbt-common>=1.10,<2.0",
+        "dbt-adapters>=1.7,<2.0",
         # 3.20 introduced pyarrow>=3.0 under the `pandas` extra
         "google-cloud-bigquery[pandas]>=3.0,<4.0",
         "google-cloud-storage~=2.4",

From 8833c5450a5f1f59b688a5c488f27864dc02f93c Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Tue, 1 Oct 2024 21:03:14 -0400
Subject: [PATCH 838/860] Catch google `NotFound` exception as a
 `DbtDatabaseError` (#1360)

* catch google NotFound exception as a DbtDatabaseError

* changelog
---
 .changes/unreleased/Fixes-20241001-193207.yaml | 7 +++++++
 dbt/adapters/bigquery/connections.py           | 4 ++++
 2 files changed, 11 insertions(+)
 create mode 100644 .changes/unreleased/Fixes-20241001-193207.yaml

diff --git a/.changes/unreleased/Fixes-20241001-193207.yaml b/.changes/unreleased/Fixes-20241001-193207.yaml
new file mode 100644
index 000000000..584445a5b
--- /dev/null
+++ b/.changes/unreleased/Fixes-20241001-193207.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: Catch additional database error exception, NotFound, as a DbtDatabaseError instead
+  of defaulting to a DbtRuntimeError
+time: 2024-10-01T19:32:07.304353-04:00
+custom:
+  Author: mikealfare
+  Issue: "1360"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index cdd9d17dc..d3eee3ef3 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -268,6 +268,10 @@ def exception_handler(self, sql):
             message = "Access denied while running query"
             self.handle_error(e, message)
 
+        except google.cloud.exceptions.NotFound as e:
+            message = "Not found while running query"
+            self.handle_error(e, message)
+
         except google.auth.exceptions.RefreshError as e:
             message = (
                 "Unable to generate access token, if you're using "

From bc93052b81a674cfa5ead0b72427d60b60f79d8b Mon Sep 17 00:00:00 2001
From: Github Build Bot <buildbot@fishtownanalytics.com>
Date: Wed, 2 Oct 2024 01:06:42 +0000
Subject: [PATCH 839/860] Bumping version to 1.9.0b1 and generate changelog

---
 .bumpversion.cfg                              |  2 +-
 .changes/1.9.0-b1.md                          | 44 +++++++++++++++++
 .../Dependencies-20231211-001048.yaml         |  0
 .../Dependencies-20231220-002130.yaml         |  0
 .../Dependencies-20231222-002351.yaml         |  0
 .../Dependencies-20240105-004800.yaml         |  0
 .../Dependencies-20240429-005158.yaml         |  0
 .../Dependencies-20240429-005159.yaml         |  0
 .../Dependencies-20240520-230208.yaml         |  0
 .../Dependencies-20240718-005755.yaml         |  0
 .../Dependencies-20240718-005756.yaml         |  0
 .../Dependencies-20240718-005757.yaml         |  0
 .../Dependencies-20240719-003740.yaml         |  0
 .../Features-20240426-105319.yaml             |  0
 .../Features-20240430-185650.yaml             |  0
 .../Features-20240501-151902.yaml             |  0
 .../Features-20240516-125735.yaml             |  0
 .../Features-20240730-135911.yaml             |  0
 .../Features-20240925-232238.yaml             |  0
 .../Fixes-20240120-180818.yaml                |  0
 .../Fixes-20240201-145323.yaml                |  0
 .../Fixes-20240226-233024.yaml                |  0
 .../Fixes-20240426-105224.yaml                |  0
 .../Fixes-20241001-193207.yaml                |  0
 .../Under the Hood-20240331-101418.yaml       |  0
 .../Under the Hood-20240718-193206.yaml       |  0
 CHANGELOG.md                                  | 48 ++++++++++++++++++-
 dbt/adapters/bigquery/__version__.py          |  2 +-
 28 files changed, 92 insertions(+), 4 deletions(-)
 create mode 100644 .changes/1.9.0-b1.md
 rename .changes/{unreleased => 1.9.0}/Dependencies-20231211-001048.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Dependencies-20231220-002130.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Dependencies-20231222-002351.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Dependencies-20240105-004800.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Dependencies-20240429-005158.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Dependencies-20240429-005159.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Dependencies-20240520-230208.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Dependencies-20240718-005755.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Dependencies-20240718-005756.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Dependencies-20240718-005757.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Dependencies-20240719-003740.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Features-20240426-105319.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Features-20240430-185650.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Features-20240501-151902.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Features-20240516-125735.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Features-20240730-135911.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Features-20240925-232238.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Fixes-20240120-180818.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Fixes-20240201-145323.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Fixes-20240226-233024.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Fixes-20240426-105224.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Fixes-20241001-193207.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Under the Hood-20240331-101418.yaml (100%)
 rename .changes/{unreleased => 1.9.0}/Under the Hood-20240718-193206.yaml (100%)

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index 24e904ac0..bd9430cbe 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.9.0a1
+current_version = 1.9.0b1
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.9.0-b1.md b/.changes/1.9.0-b1.md
new file mode 100644
index 000000000..7d0dd2c8f
--- /dev/null
+++ b/.changes/1.9.0-b1.md
@@ -0,0 +1,44 @@
+## dbt-bigquery 1.9.0-b1 - October 02, 2024
+
+### Features
+
+- Add configuration options `enable_list_inference` and `intermediate_format` for python models ([#1047](https://github.com/dbt-labs/dbt-bigquery/issues/1047), [#1114](https://github.com/dbt-labs/dbt-bigquery/issues/1114))
+- Add tests for cross-database `cast` macro ([#1214](https://github.com/dbt-labs/dbt-bigquery/issues/1214))
+- Cross-database `date` macro ([#1221](https://github.com/dbt-labs/dbt-bigquery/issues/1221))
+- Add support for base 64 encoded json keyfile credentials ([#923](https://github.com/dbt-labs/dbt-bigquery/issues/923))
+- Add support for cancelling queries on keyboard interrupt ([#917](https://github.com/dbt-labs/dbt-bigquery/issues/917))
+- Add Microbatch Strategy to dbt-spark ([#1354](https://github.com/dbt-labs/dbt-bigquery/issues/1354))
+
+### Fixes
+
+- Drop intermediate objects created in BigQuery for incremental models ([#1036](https://github.com/dbt-labs/dbt-bigquery/issues/1036))
+- Fix null column index issue during `dbt docs generate` for external tables ([#1079](https://github.com/dbt-labs/dbt-bigquery/issues/1079))
+- make seed delimiter configurable via `field_delimeter` in model config ([#1119](https://github.com/dbt-labs/dbt-bigquery/issues/1119))
+- Default `enableListInference` to `True` for python models to support nested lists ([#1047](https://github.com/dbt-labs/dbt-bigquery/issues/1047), [#1114](https://github.com/dbt-labs/dbt-bigquery/issues/1114))
+- Catch additional database error exception, NotFound, as a DbtDatabaseError instead of defaulting to a DbtRuntimeError ([#1360](https://github.com/dbt-labs/dbt-bigquery/issues/1360))
+
+### Under the Hood
+
+- Lazy load `agate` ([#1162](https://github.com/dbt-labs/dbt-bigquery/issues/1162))
+- Simplify linting environment and dev dependencies ([#1291](https://github.com/dbt-labs/dbt-bigquery/issues/1291))
+
+### Dependencies
+
+- Update pre-commit requirement from ~=3.5 to ~=3.7 ([#1052](https://github.com/dbt-labs/dbt-bigquery/pull/1052))
+- Update freezegun requirement from ~=1.3 to ~=1.4 ([#1062](https://github.com/dbt-labs/dbt-bigquery/pull/1062))
+- Bump mypy from 1.7.1 to 1.8.0 ([#1064](https://github.com/dbt-labs/dbt-bigquery/pull/1064))
+- Update flake8 requirement from ~=6.1 to ~=7.0 ([#1069](https://github.com/dbt-labs/dbt-bigquery/pull/1069))
+- Bump actions/download-artifact from 3 to 4 ([#1209](https://github.com/dbt-labs/dbt-bigquery/pull/1209))
+- Bump actions/upload-artifact from 3 to 4 ([#1210](https://github.com/dbt-labs/dbt-bigquery/pull/1210))
+- Bump ubuntu from 22.04 to 24.04 in /docker ([#1247](https://github.com/dbt-labs/dbt-bigquery/pull/1247))
+- Update pre-commit-hooks requirement from ~=4.5 to ~=4.6 ([#1281](https://github.com/dbt-labs/dbt-bigquery/pull/1281))
+- Update pytest-xdist requirement from ~=3.5 to ~=3.6 ([#1282](https://github.com/dbt-labs/dbt-bigquery/pull/1282))
+- Update flaky requirement from ~=3.7 to ~=3.8 ([#1283](https://github.com/dbt-labs/dbt-bigquery/pull/1283))
+- Update twine requirement from ~=4.0 to ~=5.1 ([#1293](https://github.com/dbt-labs/dbt-bigquery/pull/1293))
+
+### Contributors
+- [@d-cole](https://github.com/d-cole) ([#917](https://github.com/dbt-labs/dbt-bigquery/issues/917))
+- [@dwreeves](https://github.com/dwreeves) ([#1162](https://github.com/dbt-labs/dbt-bigquery/issues/1162))
+- [@robeleb1](https://github.com/robeleb1) ([#923](https://github.com/dbt-labs/dbt-bigquery/issues/923))
+- [@salimmoulouel](https://github.com/salimmoulouel) ([#1119](https://github.com/dbt-labs/dbt-bigquery/issues/1119))
+- [@vinit2107](https://github.com/vinit2107) ([#1036](https://github.com/dbt-labs/dbt-bigquery/issues/1036))
diff --git a/.changes/unreleased/Dependencies-20231211-001048.yaml b/.changes/1.9.0/Dependencies-20231211-001048.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231211-001048.yaml
rename to .changes/1.9.0/Dependencies-20231211-001048.yaml
diff --git a/.changes/unreleased/Dependencies-20231220-002130.yaml b/.changes/1.9.0/Dependencies-20231220-002130.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231220-002130.yaml
rename to .changes/1.9.0/Dependencies-20231220-002130.yaml
diff --git a/.changes/unreleased/Dependencies-20231222-002351.yaml b/.changes/1.9.0/Dependencies-20231222-002351.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20231222-002351.yaml
rename to .changes/1.9.0/Dependencies-20231222-002351.yaml
diff --git a/.changes/unreleased/Dependencies-20240105-004800.yaml b/.changes/1.9.0/Dependencies-20240105-004800.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20240105-004800.yaml
rename to .changes/1.9.0/Dependencies-20240105-004800.yaml
diff --git a/.changes/unreleased/Dependencies-20240429-005158.yaml b/.changes/1.9.0/Dependencies-20240429-005158.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20240429-005158.yaml
rename to .changes/1.9.0/Dependencies-20240429-005158.yaml
diff --git a/.changes/unreleased/Dependencies-20240429-005159.yaml b/.changes/1.9.0/Dependencies-20240429-005159.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20240429-005159.yaml
rename to .changes/1.9.0/Dependencies-20240429-005159.yaml
diff --git a/.changes/unreleased/Dependencies-20240520-230208.yaml b/.changes/1.9.0/Dependencies-20240520-230208.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20240520-230208.yaml
rename to .changes/1.9.0/Dependencies-20240520-230208.yaml
diff --git a/.changes/unreleased/Dependencies-20240718-005755.yaml b/.changes/1.9.0/Dependencies-20240718-005755.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20240718-005755.yaml
rename to .changes/1.9.0/Dependencies-20240718-005755.yaml
diff --git a/.changes/unreleased/Dependencies-20240718-005756.yaml b/.changes/1.9.0/Dependencies-20240718-005756.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20240718-005756.yaml
rename to .changes/1.9.0/Dependencies-20240718-005756.yaml
diff --git a/.changes/unreleased/Dependencies-20240718-005757.yaml b/.changes/1.9.0/Dependencies-20240718-005757.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20240718-005757.yaml
rename to .changes/1.9.0/Dependencies-20240718-005757.yaml
diff --git a/.changes/unreleased/Dependencies-20240719-003740.yaml b/.changes/1.9.0/Dependencies-20240719-003740.yaml
similarity index 100%
rename from .changes/unreleased/Dependencies-20240719-003740.yaml
rename to .changes/1.9.0/Dependencies-20240719-003740.yaml
diff --git a/.changes/unreleased/Features-20240426-105319.yaml b/.changes/1.9.0/Features-20240426-105319.yaml
similarity index 100%
rename from .changes/unreleased/Features-20240426-105319.yaml
rename to .changes/1.9.0/Features-20240426-105319.yaml
diff --git a/.changes/unreleased/Features-20240430-185650.yaml b/.changes/1.9.0/Features-20240430-185650.yaml
similarity index 100%
rename from .changes/unreleased/Features-20240430-185650.yaml
rename to .changes/1.9.0/Features-20240430-185650.yaml
diff --git a/.changes/unreleased/Features-20240501-151902.yaml b/.changes/1.9.0/Features-20240501-151902.yaml
similarity index 100%
rename from .changes/unreleased/Features-20240501-151902.yaml
rename to .changes/1.9.0/Features-20240501-151902.yaml
diff --git a/.changes/unreleased/Features-20240516-125735.yaml b/.changes/1.9.0/Features-20240516-125735.yaml
similarity index 100%
rename from .changes/unreleased/Features-20240516-125735.yaml
rename to .changes/1.9.0/Features-20240516-125735.yaml
diff --git a/.changes/unreleased/Features-20240730-135911.yaml b/.changes/1.9.0/Features-20240730-135911.yaml
similarity index 100%
rename from .changes/unreleased/Features-20240730-135911.yaml
rename to .changes/1.9.0/Features-20240730-135911.yaml
diff --git a/.changes/unreleased/Features-20240925-232238.yaml b/.changes/1.9.0/Features-20240925-232238.yaml
similarity index 100%
rename from .changes/unreleased/Features-20240925-232238.yaml
rename to .changes/1.9.0/Features-20240925-232238.yaml
diff --git a/.changes/unreleased/Fixes-20240120-180818.yaml b/.changes/1.9.0/Fixes-20240120-180818.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20240120-180818.yaml
rename to .changes/1.9.0/Fixes-20240120-180818.yaml
diff --git a/.changes/unreleased/Fixes-20240201-145323.yaml b/.changes/1.9.0/Fixes-20240201-145323.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20240201-145323.yaml
rename to .changes/1.9.0/Fixes-20240201-145323.yaml
diff --git a/.changes/unreleased/Fixes-20240226-233024.yaml b/.changes/1.9.0/Fixes-20240226-233024.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20240226-233024.yaml
rename to .changes/1.9.0/Fixes-20240226-233024.yaml
diff --git a/.changes/unreleased/Fixes-20240426-105224.yaml b/.changes/1.9.0/Fixes-20240426-105224.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20240426-105224.yaml
rename to .changes/1.9.0/Fixes-20240426-105224.yaml
diff --git a/.changes/unreleased/Fixes-20241001-193207.yaml b/.changes/1.9.0/Fixes-20241001-193207.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20241001-193207.yaml
rename to .changes/1.9.0/Fixes-20241001-193207.yaml
diff --git a/.changes/unreleased/Under the Hood-20240331-101418.yaml b/.changes/1.9.0/Under the Hood-20240331-101418.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20240331-101418.yaml
rename to .changes/1.9.0/Under the Hood-20240331-101418.yaml
diff --git a/.changes/unreleased/Under the Hood-20240718-193206.yaml b/.changes/1.9.0/Under the Hood-20240718-193206.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20240718-193206.yaml
rename to .changes/1.9.0/Under the Hood-20240718-193206.yaml
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 4a408c580..b9bda350a 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,10 +5,54 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
+## dbt-bigquery 1.9.0-b1 - October 02, 2024
+
+### Features
+
+- Add configuration options `enable_list_inference` and `intermediate_format` for python models ([#1047](https://github.com/dbt-labs/dbt-bigquery/issues/1047), [#1114](https://github.com/dbt-labs/dbt-bigquery/issues/1114))
+- Add tests for cross-database `cast` macro ([#1214](https://github.com/dbt-labs/dbt-bigquery/issues/1214))
+- Cross-database `date` macro ([#1221](https://github.com/dbt-labs/dbt-bigquery/issues/1221))
+- Add support for base 64 encoded json keyfile credentials ([#923](https://github.com/dbt-labs/dbt-bigquery/issues/923))
+- Add support for cancelling queries on keyboard interrupt ([#917](https://github.com/dbt-labs/dbt-bigquery/issues/917))
+- Add Microbatch Strategy to dbt-spark ([#1354](https://github.com/dbt-labs/dbt-bigquery/issues/1354))
+
+### Fixes
+
+- Drop intermediate objects created in BigQuery for incremental models ([#1036](https://github.com/dbt-labs/dbt-bigquery/issues/1036))
+- Fix null column index issue during `dbt docs generate` for external tables ([#1079](https://github.com/dbt-labs/dbt-bigquery/issues/1079))
+- make seed delimiter configurable via `field_delimeter` in model config ([#1119](https://github.com/dbt-labs/dbt-bigquery/issues/1119))
+- Default `enableListInference` to `True` for python models to support nested lists ([#1047](https://github.com/dbt-labs/dbt-bigquery/issues/1047), [#1114](https://github.com/dbt-labs/dbt-bigquery/issues/1114))
+- Catch additional database error exception, NotFound, as a DbtDatabaseError instead of defaulting to a DbtRuntimeError ([#1360](https://github.com/dbt-labs/dbt-bigquery/issues/1360))
+
+### Under the Hood
+
+- Lazy load `agate` ([#1162](https://github.com/dbt-labs/dbt-bigquery/issues/1162))
+- Simplify linting environment and dev dependencies ([#1291](https://github.com/dbt-labs/dbt-bigquery/issues/1291))
+
+### Dependencies
+
+- Update pre-commit requirement from ~=3.5 to ~=3.7 ([#1052](https://github.com/dbt-labs/dbt-bigquery/pull/1052))
+- Update freezegun requirement from ~=1.3 to ~=1.4 ([#1062](https://github.com/dbt-labs/dbt-bigquery/pull/1062))
+- Bump mypy from 1.7.1 to 1.8.0 ([#1064](https://github.com/dbt-labs/dbt-bigquery/pull/1064))
+- Update flake8 requirement from ~=6.1 to ~=7.0 ([#1069](https://github.com/dbt-labs/dbt-bigquery/pull/1069))
+- Bump actions/download-artifact from 3 to 4 ([#1209](https://github.com/dbt-labs/dbt-bigquery/pull/1209))
+- Bump actions/upload-artifact from 3 to 4 ([#1210](https://github.com/dbt-labs/dbt-bigquery/pull/1210))
+- Bump ubuntu from 22.04 to 24.04 in /docker ([#1247](https://github.com/dbt-labs/dbt-bigquery/pull/1247))
+- Update pre-commit-hooks requirement from ~=4.5 to ~=4.6 ([#1281](https://github.com/dbt-labs/dbt-bigquery/pull/1281))
+- Update pytest-xdist requirement from ~=3.5 to ~=3.6 ([#1282](https://github.com/dbt-labs/dbt-bigquery/pull/1282))
+- Update flaky requirement from ~=3.7 to ~=3.8 ([#1283](https://github.com/dbt-labs/dbt-bigquery/pull/1283))
+- Update twine requirement from ~=4.0 to ~=5.1 ([#1293](https://github.com/dbt-labs/dbt-bigquery/pull/1293))
+
+### Contributors
+- [@d-cole](https://github.com/d-cole) ([#917](https://github.com/dbt-labs/dbt-bigquery/issues/917))
+- [@dwreeves](https://github.com/dwreeves) ([#1162](https://github.com/dbt-labs/dbt-bigquery/issues/1162))
+- [@robeleb1](https://github.com/robeleb1) ([#923](https://github.com/dbt-labs/dbt-bigquery/issues/923))
+- [@salimmoulouel](https://github.com/salimmoulouel) ([#1119](https://github.com/dbt-labs/dbt-bigquery/issues/1119))
+- [@vinit2107](https://github.com/vinit2107) ([#1036](https://github.com/dbt-labs/dbt-bigquery/issues/1036))
+
+
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
-- [1.8](https://github.com/dbt-labs/dbt-bigquery/blob/1.8.latest/CHANGELOG.md)
-- [1.7](https://github.com/dbt-labs/dbt-bigquery/blob/1.7.latest/CHANGELOG.md)
 - [1.6](https://github.com/dbt-labs/dbt-bigquery/blob/1.6.latest/CHANGELOG.md)
 - [1.5](https://github.com/dbt-labs/dbt-bigquery/blob/1.5.latest/CHANGELOG.md)
 - [1.4](https://github.com/dbt-labs/dbt-bigquery/blob/1.4.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index 6698ed64c..a4077fff2 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.9.0a1"
+version = "1.9.0b1"

From 1d61844b9bea3e3d43ffb544a3573d49954fd550 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Thu, 17 Oct 2024 15:07:27 -0400
Subject: [PATCH 840/860] Drop support for Python 3.8 (#1373)

* drop support for python 3.8
---
 .../Breaking Changes-20241016-185117.yaml     |  6 +++
 .github/scripts/integration-test-matrix.js    |  4 +-
 .github/workflows/integration.yml             |  5 +-
 .github/workflows/main.yml                    |  8 +--
 .pre-commit-config.yaml                       |  1 -
 CONTRIBUTING.md                               |  2 +-
 dev-requirements.txt                          |  3 +-
 docker/Dockerfile                             |  4 +-
 docker/dev.Dockerfile                         | 50 +++++++++----------
 setup.py                                      |  7 ++-
 tox.ini                                       |  8 +--
 11 files changed, 51 insertions(+), 47 deletions(-)
 create mode 100644 .changes/unreleased/Breaking Changes-20241016-185117.yaml

diff --git a/.changes/unreleased/Breaking Changes-20241016-185117.yaml b/.changes/unreleased/Breaking Changes-20241016-185117.yaml
new file mode 100644
index 000000000..55bb37461
--- /dev/null
+++ b/.changes/unreleased/Breaking Changes-20241016-185117.yaml	
@@ -0,0 +1,6 @@
+kind: Breaking Changes
+body: Drop support for Python 3.8
+time: 2024-10-16T18:51:17.581547-04:00
+custom:
+  Author: mikealfare
+  Issue: "1373"
diff --git a/.github/scripts/integration-test-matrix.js b/.github/scripts/integration-test-matrix.js
index 8e4b351fc..49db45575 100644
--- a/.github/scripts/integration-test-matrix.js
+++ b/.github/scripts/integration-test-matrix.js
@@ -1,6 +1,6 @@
 module.exports = ({ context }) => {
-  const defaultPythonVersion = "3.8";
-  const supportedPythonVersions = ["3.8", "3.9", "3.10", "3.11", "3.12"];
+  const defaultPythonVersion = "3.9";
+  const supportedPythonVersions = ["3.9", "3.10", "3.11", "3.12"];
   const supportedAdapters = ["bigquery"];
 
   // if PR, generate matrix based on files changed and PR labels
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 43ac4ecb3..a9179f9ce 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -33,6 +33,7 @@ on:
   # all PRs, important to note that `pull_request_target` workflows
   # will run in the context of the target branch of a PR
   pull_request_target:
+    types: [opened, reopened, synchronize, labeled]
   # manual trigger
   workflow_dispatch:
     inputs:
@@ -280,10 +281,10 @@ jobs:
           persist-credentials: false
           ref: ${{ github.event.pull_request.head.sha }}
 
-      - name: Set up Python 3.8
+      - name: Set up Python 3.9
         uses: actions/setup-python@v5
         with:
-          python-version: "3.8"
+          python-version: "3.9"
 
       - name: Install python dependencies
         run: |
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index d04b4307f..2b0ce2c25 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -50,7 +50,7 @@ jobs:
       - name: Set up Python
         uses: actions/setup-python@v5
         with:
-          python-version: '3.8'
+          python-version: '3.9'
 
       - name: Install python dependencies
         run: |
@@ -70,7 +70,7 @@ jobs:
     strategy:
       fail-fast: false
       matrix:
-        python-version: ['3.8', '3.9', '3.10', '3.11', '3.12']
+        python-version: ['3.9', '3.10', '3.11', '3.12']
 
     env:
       TOXENV: "unit"
@@ -127,7 +127,7 @@ jobs:
       - name: Set up Python
         uses: actions/setup-python@v5
         with:
-          python-version: '3.8'
+          python-version: '3.9'
 
       - name: Install python dependencies
         run: |
@@ -175,7 +175,7 @@ jobs:
       fail-fast: false
       matrix:
         os: [ubuntu-latest, macos-12, windows-latest]
-        python-version: ['3.8', '3.9', '3.10', '3.11', '3.12']
+        python-version: ['3.9', '3.10', '3.11', '3.12']
 
     steps:
       - name: Set up Python ${{ matrix.python-version }}
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index 3d0b98d45..16760bf07 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -24,7 +24,6 @@ repos:
     -   id: black
         args:
         -   --line-length=99
-        -   --target-version=py38
         -   --target-version=py39
         -   --target-version=py310
         -   --target-version=py311
diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md
index 1af648741..bef8d67da 100644
--- a/CONTRIBUTING.md
+++ b/CONTRIBUTING.md
@@ -67,7 +67,7 @@ $EDITOR test.env
 There are a few methods for running tests locally.
 
 #### `tox`
-`tox` takes care of managing Python virtualenvs and installing dependencies in order to run tests. You can also run tests in parallel, for example you can run unit tests for Python 3.8, Python 3.9, Python 3.10, and Python 3.11 in parallel with `tox -p`. Also, you can run unit tests for specific python versions with `tox -e py38`. The configuration of these tests are located in `tox.ini`.
+`tox` takes care of managing Python virtualenvs and installing dependencies in order to run tests. You can also run tests in parallel, for example you can run unit tests for Python 3.9, Python 3.10, and Python 3.11 in parallel with `tox -p`. Also, you can run unit tests for specific python versions with `tox -e py39`. The configuration of these tests are located in `tox.ini`.
 
 #### `pytest`
 Finally, you can also run a specific test or group of tests using `pytest` directly. With a Python virtualenv active and dev dependencies installed you can do things like:
diff --git a/dev-requirements.txt b/dev-requirements.txt
index 34169172a..2c0134110 100644
--- a/dev-requirements.txt
+++ b/dev-requirements.txt
@@ -6,8 +6,7 @@ git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
 
 # dev
 ddtrace==2.3.0
-pre-commit~=3.7.0;python_version>="3.9"
-pre-commit~=3.5.0;python_version<"3.9"
+pre-commit~=3.7.0
 pytest~=7.4
 pytest-csv~=3.0
 pytest-dotenv~=0.5.2
diff --git a/docker/Dockerfile b/docker/Dockerfile
index 3b9431fd1..bda507dc5 100644
--- a/docker/Dockerfile
+++ b/docker/Dockerfile
@@ -1,7 +1,7 @@
 # this image gets published to GHCR for production use
 ARG py_version=3.11.2
 
-FROM python:$py_version-slim-bullseye as base
+FROM python:$py_version-slim-bullseye AS base
 
 RUN apt-get update \
   && apt-get dist-upgrade -y \
@@ -25,7 +25,7 @@ ENV LANG=C.UTF-8
 RUN python -m pip install --upgrade "pip==24.0" "setuptools==69.2.0" "wheel==0.43.0" --no-cache-dir
 
 
-FROM base as dbt-bigquery
+FROM base AS dbt-bigquery
 
 ARG commit_ref=main
 
diff --git a/docker/dev.Dockerfile b/docker/dev.Dockerfile
index 2afad0a95..f122f5343 100644
--- a/docker/dev.Dockerfile
+++ b/docker/dev.Dockerfile
@@ -1,43 +1,43 @@
 # this image does not get published, it is intended for local development only, see `Makefile` for usage
-FROM ubuntu:24.04 as base
+FROM ubuntu:24.04 AS base
 
 # prevent python installation from asking for time zone region
 ARG DEBIAN_FRONTEND=noninteractive
 
 # add python repository
 RUN apt-get update \
-  && apt-get install -y software-properties-common=0.99.22.9 \
-  && add-apt-repository -y ppa:deadsnakes/ppa \
-  && apt-get clean \
-  && rm -rf \
-    /var/lib/apt/lists/* \
-    /tmp/* \
-    /var/tmp/*
+    && apt-get install -y software-properties-common=0.99.48 \
+    && add-apt-repository -y ppa:deadsnakes/ppa \
+    && apt-get clean \
+    && rm -rf \
+        /var/lib/apt/lists/* \
+        /tmp/* \
+        /var/tmp/*
 
 # install python
 RUN apt-get update \
-  && apt-get install -y --no-install-recommends \
-    build-essential=12.9ubuntu3 \
-    git-all=1:2.34.1-1ubuntu1.10 \
-    python3.8=3.8.19-1+jammy1 \
-    python3.8-dev=3.8.19-1+jammy1 \
-    python3.8-distutils=3.8.19-1+jammy1 \
-    python3.8-venv=3.8.19-1+jammy1 \
-    python3-pip=22.0.2+dfsg-1ubuntu0.4 \
-    python3-wheel=0.37.1-2ubuntu0.22.04.1 \
-  && apt-get clean \
-  && rm -rf \
-    /var/lib/apt/lists/* \
-    /tmp/* \
-    /var/tmp/*
+    && apt-get install -y --no-install-recommends \
+        build-essential=12.10ubuntu1 \
+        git-all=1:2.43.0-1ubuntu7.1 \
+        python3.9=3.9.20-1+noble1 \
+        python3.9-dev=3.9.20-1+noble1 \
+        python3.9-distutils=3.9.20-1+noble1 \
+        python3.9-venv=3.9.20-1+noble1 \
+        python3-pip=24.0+dfsg-1ubuntu1 \
+        python3-wheel=0.42.0-2 \
+    && apt-get clean \
+    && rm -rf \
+        /var/lib/apt/lists/* \
+        /tmp/* \
+        /var/tmp/*
 
 # update the default system interpreter to the newly installed version
-RUN update-alternatives --install /usr/bin/python3 python3 /usr/bin/python3.8 1
+RUN update-alternatives --install /usr/bin/python python /usr/bin/python3.9 1
 
 
-FROM base as dbt-bigquery-dev
+FROM base AS dbt-bigquery-dev
 
-HEALTHCHECK CMD python3 --version || exit 1
+HEALTHCHECK CMD python --version || exit 1
 
 # send stdout/stderr to terminal
 ENV PYTHONUNBUFFERED=1
diff --git a/setup.py b/setup.py
index ab89a3c39..79f6025ea 100644
--- a/setup.py
+++ b/setup.py
@@ -2,9 +2,9 @@
 import sys
 
 # require a supported version of Python
-if sys.version_info < (3, 8):
+if sys.version_info < (3, 9):
     print("Error: dbt does not support this version of Python.")
-    print("Please upgrade to Python 3.8 or higher.")
+    print("Please upgrade to Python 3.9 or higher.")
     sys.exit(1)
 
 try:
@@ -69,11 +69,10 @@ def _dbt_bigquery_version() -> str:
         "Operating System :: Microsoft :: Windows",
         "Operating System :: MacOS :: MacOS X",
         "Operating System :: POSIX :: Linux",
-        "Programming Language :: Python :: 3.8",
         "Programming Language :: Python :: 3.9",
         "Programming Language :: Python :: 3.10",
         "Programming Language :: Python :: 3.11",
         "Programming Language :: Python :: 3.12",
     ],
-    python_requires=">=3.8",
+    python_requires=">=3.9",
 )
diff --git a/tox.ini b/tox.ini
index b388dc5b3..240d85e34 100644
--- a/tox.ini
+++ b/tox.ini
@@ -1,8 +1,8 @@
 [tox]
 skipsdist = True
-envlist = py38,py39,py310,py311,py312
+envlist = py39,py310,py311,py312
 
-[testenv:{unit,py38,py39,py310,py311,py312,py}]
+[testenv:{unit,py39,py310,py311,py312,py}]
 description = unit testing
 skip_install = true
 passenv =
@@ -13,7 +13,7 @@ deps =
   -rdev-requirements.txt
   -e.
 
-[testenv:{integration,py38,py39,py310,py311,py312,py}-{bigquery}]
+[testenv:{integration,py39,py310,py311,py312,py}-{bigquery}]
 description = adapter plugin integration testing
 skip_install = true
 passenv =
@@ -33,7 +33,7 @@ deps =
   -rdev-requirements.txt
   .
 
-[testenv:{python-tests,py38,py39,py310,py311,py312,py}]
+[testenv:{python-tests,py39,py310,py311,py312,py}]
 description = python integration testing
 skip_install = true
 passenv =

From 0c96c9575452ff33e15f8bdb53d1bde9be55637c Mon Sep 17 00:00:00 2001
From: leahwicz <60146280+leahwicz@users.noreply.github.com>
Date: Thu, 17 Oct 2024 17:26:08 -0400
Subject: [PATCH 841/860] Isolating distribution testing (#1346)

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .../Under the Hood-20240910-212052.yaml       |  6 ++++++
 .github/workflows/main.yml                    | 19 ++++++++-----------
 2 files changed, 14 insertions(+), 11 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20240910-212052.yaml

diff --git a/.changes/unreleased/Under the Hood-20240910-212052.yaml b/.changes/unreleased/Under the Hood-20240910-212052.yaml
new file mode 100644
index 000000000..3e4885dcd
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20240910-212052.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Isolating distribution testing
+time: 2024-09-10T21:20:52.574204-04:00
+custom:
+    Author: leahwicz
+    Issue: "1290"
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 2b0ce2c25..0e4d476b7 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -163,7 +163,7 @@ jobs:
           overwrite: true
 
   test-build:
-    name: verify packages / python ${{ matrix.python-version }} / ${{ matrix.os }}
+    name: verify packages / python ${{ matrix.python-version }} / ${{ matrix.os }} / ${{ matrix.dist-type }}
 
     if: needs.build.outputs.is_alpha == 0
 
@@ -182,11 +182,13 @@ jobs:
         uses: actions/setup-python@v5
         with:
           python-version: ${{ matrix.python-version }}
+
       - name: Install python dependencies
         run: |
           python -m pip install --user --upgrade pip
-          python -m pip install --upgrade wheel setuptools twine check-wheel-contents
+          python -m pip install --upgrade wheel
           python -m pip --version
+
       - uses: actions/download-artifact@v4
         with:
           name: dist
@@ -195,15 +197,10 @@ jobs:
       - name: Show distributions
         run: ls -lh dist/
 
-      - name: Install wheel distributions
-        run: |
-          find ./dist/*.whl -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/
-      - name: Check wheel distributions
+      - name: Install ${{ matrix.dist-type }} distributions
         run: |
-          python -c "import dbt.adapters.bigquery"
-      - name: Install source distributions
-        run: |
-          find ./dist/*.gz -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/
-      - name: Check source distributions
+          find ./dist/*.${{ matrix.dist-type }} -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/
+
+      - name: Check ${{ matrix.dist-type }} distributions
         run: |
           python -c "import dbt.adapters.bigquery"

From 4e24d6a8c78ee7f5b74f6edd7d7bb63fce28296c Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Fri, 18 Oct 2024 08:52:08 -0400
Subject: [PATCH 842/860] add matrix entry back that was inadvertently removed
 during a sync with main (#1375)

---
 .github/workflows/main.yml | 1 +
 1 file changed, 1 insertion(+)

diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 0e4d476b7..6bfed5df6 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -176,6 +176,7 @@ jobs:
       matrix:
         os: [ubuntu-latest, macos-12, windows-latest]
         python-version: ['3.9', '3.10', '3.11', '3.12']
+        dist-type: ["whl", "gz"]
 
     steps:
       - name: Set up Python ${{ matrix.python-version }}

From d4be89a3840c400a963c8d24e82cbbac608290a2 Mon Sep 17 00:00:00 2001
From: Yasuhisa Yoshida <syou6162@gmail.com>
Date: Sat, 19 Oct 2024 04:34:44 +0900
Subject: [PATCH 843/860] Replace 'integration' with 'functional' in
 CONTRIBUTING.md (#1357)

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 CONTRIBUTING.md | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md
index bef8d67da..f915af713 100644
--- a/CONTRIBUTING.md
+++ b/CONTRIBUTING.md
@@ -54,7 +54,7 @@ To confirm you have the correct version of `dbt-core` installed please run `dbt
 
 ### Initial Setup
 
-`dbt-bigquery` contains [unit](https://github.com/dbt-labs/dbt-bigquery/tree/main/tests/unit) and [integration](https://github.com/dbt-labs/dbt-bigquery/tree/main/tests/integration) tests. Integration tests require testing against an actual BigQuery warehouse. We have CI set up to test against a BigQuery warehouse. In order to run integration tests locally, you will need a `test.env` file in the root of the repository that contains credentials for BigQuery.
+`dbt-bigquery` contains [unit](https://github.com/dbt-labs/dbt-bigquery/tree/main/tests/unit) and [functional](https://github.com/dbt-labs/dbt-bigquery/tree/main/tests/functional) tests. functional tests require testing against an actual BigQuery warehouse. We have CI set up to test against a BigQuery warehouse. In order to run functional tests locally, you will need a `test.env` file in the root of the repository that contains credentials for BigQuery.
 
 Note: This `test.env` file is git-ignored, but please be _extra_ careful to never check in credentials or other sensitive information when developing. To create your `test.env` file, copy the provided example file, then supply your relevant credentials.
 
@@ -104,6 +104,6 @@ You don't need to worry about which `dbt-bigquery` version your change will go i
 
 dbt Labs provides a CI environment to test changes to the `dbt-bigquery` adapter and periodic checks against the development version of `dbt-core` through Github Actions.
 
-A `dbt-bigquery` maintainer will review your PR. They may suggest code revision for style or clarity, or request that you add unit or integration test(s). These are good things! We believe that, with a little bit of help, anyone can contribute high-quality code.
+A `dbt-bigquery` maintainer will review your PR. They may suggest code revision for style or clarity, or request that you add unit or functional test(s). These are good things! We believe that, with a little bit of help, anyone can contribute high-quality code.
 
 Once all tests are passing, you have updated the changelog to reflect and tag your issue/pr for reference with a small description of the change, and your PR has been approved, a `dbt-bigquery` maintainer will merge your changes into the active development branch. And that's it! Happy developing :tada:

From 79fbd9062c2d927fce2b105902466453e1d8b55f Mon Sep 17 00:00:00 2001
From: jaejun <63435794+jx2lee@users.noreply.github.com>
Date: Fri, 25 Oct 2024 05:50:37 +0900
Subject: [PATCH 844/860] add test case when raise ServiceUnavailable in
 is_retryable (#1224)

* raise ServiceUnavailable and test that it is retry-able

---------

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Features-20240505-011838.yaml | 6 ++++++
 tests/unit/test_bigquery_connection_manager.py    | 2 ++
 2 files changed, 8 insertions(+)
 create mode 100644 .changes/unreleased/Features-20240505-011838.yaml

diff --git a/.changes/unreleased/Features-20240505-011838.yaml b/.changes/unreleased/Features-20240505-011838.yaml
new file mode 100644
index 000000000..66411853f
--- /dev/null
+++ b/.changes/unreleased/Features-20240505-011838.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: add is_retryable test case when raise ServiceUnavailable
+time: 2024-05-05T01:18:38.737882+09:00
+custom:
+  Author: jx2lee
+  Issue: "682"
diff --git a/tests/unit/test_bigquery_connection_manager.py b/tests/unit/test_bigquery_connection_manager.py
index d09cb1635..1c14100f6 100644
--- a/tests/unit/test_bigquery_connection_manager.py
+++ b/tests/unit/test_bigquery_connection_manager.py
@@ -84,12 +84,14 @@ def test_is_retryable(self):
         rate_limit_error = exceptions.Forbidden(
             "code broke", errors=[{"reason": "rateLimitExceeded"}]
         )
+        service_unavailable_error = exceptions.ServiceUnavailable("service is unavailable")
 
         self.assertTrue(_is_retryable(internal_server_error))
         self.assertTrue(_is_retryable(bad_request_error))
         self.assertTrue(_is_retryable(connection_error))
         self.assertFalse(_is_retryable(client_error))
         self.assertTrue(_is_retryable(rate_limit_error))
+        self.assertTrue(_is_retryable(service_unavailable_error))
 
     def test_drop_dataset(self):
         mock_table = Mock()

From a09a8faefe247a4793918a0b857dc27f78cf8602 Mon Sep 17 00:00:00 2001
From: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Tue, 29 Oct 2024 14:41:02 -0700
Subject: [PATCH 845/860] use "direct" write for non-partitioned python model
 materializations (#1388)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* use "direct" write for non-partitioned python model materializations

* add changie log

* add code comment

* make code comment inline

* make code comment inline

* remove code comment

* use set write_method instead of inline conditional

* use set write_method instead of inline conditional
---
 .changes/unreleased/Fixes-20241028-172719.yaml        |  6 ++++++
 .../bigquery/macros/materializations/table.sql        | 11 ++++++++++-
 2 files changed, 16 insertions(+), 1 deletion(-)
 create mode 100644 .changes/unreleased/Fixes-20241028-172719.yaml

diff --git a/.changes/unreleased/Fixes-20241028-172719.yaml b/.changes/unreleased/Fixes-20241028-172719.yaml
new file mode 100644
index 000000000..87ee2c25d
--- /dev/null
+++ b/.changes/unreleased/Fixes-20241028-172719.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: use "direct" write for non-partitioned python model materializations
+time: 2024-10-28T17:27:19.306348-07:00
+custom:
+  Author: colin-rogers-dbt
+  Issue: "1318"
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt/include/bigquery/macros/materializations/table.sql
index e3c5b3598..41bb69770 100644
--- a/dbt/include/bigquery/macros/materializations/table.sql
+++ b/dbt/include/bigquery/macros/materializations/table.sql
@@ -113,10 +113,19 @@ else:
   msg = f"{type(df)} is not a supported type for dbt Python materialization"
   raise Exception(msg)
 
+# For writeMethod we need to use "indirect" if materializing a partitioned table
+# otherwise we can use "direct". Note that indirect will fail if the GCS bucket has a retention policy set on it.
+{%- if partition_config %}
+      {%- set write_method = 'indirect' -%}
+{%- else %}
+      {% set write_method = 'direct' -%}
+{%- endif %}
+
 df.write \
   .mode("overwrite") \
   .format("bigquery") \
-  .option("writeMethod", "indirect").option("writeDisposition", 'WRITE_TRUNCATE') \
+  .option("writeMethod", "{{ write_method }}") \
+  .option("writeDisposition", 'WRITE_TRUNCATE') \
   {%- if partition_config is not none %}
   {%- if partition_config.data_type | lower in ('date','timestamp','datetime') %}
   .option("partitionField", "{{- partition_config.field -}}") \

From 4e3f86e02a98e7c0ae9b17c7e14ee38de08314e3 Mon Sep 17 00:00:00 2001
From: Jack <j.carpenter_12@live.com>
Date: Fri, 1 Nov 2024 20:05:23 +0000
Subject: [PATCH 846/860] Feature/add quota project option (#1345)

* add ability to specify quota project in profile

---------

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .../unreleased/Features-20240911-234859.yaml  |  6 +++++
 dbt/adapters/bigquery/connections.py          |  6 ++++-
 tests/functional/test_quota_project.py        | 27 +++++++++++++++++++
 tests/unit/test_bigquery_adapter.py           |  5 +++-
 4 files changed, 42 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Features-20240911-234859.yaml
 create mode 100644 tests/functional/test_quota_project.py

diff --git a/.changes/unreleased/Features-20240911-234859.yaml b/.changes/unreleased/Features-20240911-234859.yaml
new file mode 100644
index 000000000..5351c3315
--- /dev/null
+++ b/.changes/unreleased/Features-20240911-234859.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Adds the ability to set optional `quota_project` in profile
+time: 2024-09-11T23:48:59.767649+01:00
+custom:
+    Author: jcarpenter12
+    Issue: 1343 1344
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index d3eee3ef3..58b3dbe41 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -17,7 +17,7 @@
 import google.auth.exceptions
 import google.cloud.bigquery
 import google.cloud.exceptions
-from google.api_core import retry, client_info
+from google.api_core import retry, client_info, client_options
 from google.auth import impersonated_credentials
 from google.oauth2 import (
     credentials as GoogleCredentials,
@@ -125,6 +125,7 @@ class BigQueryCredentials(Credentials):
     database: Optional[str] = None
     schema: Optional[str] = None
     execution_project: Optional[str] = None
+    quota_project: Optional[str] = None
     location: Optional[str] = None
     priority: Optional[Priority] = None
     maximum_bytes_billed: Optional[int] = None
@@ -408,14 +409,17 @@ def get_credentials(cls, profile_credentials):
     def get_bigquery_client(cls, profile_credentials):
         creds = cls.get_credentials(profile_credentials)
         execution_project = profile_credentials.execution_project
+        quota_project = profile_credentials.quota_project
         location = getattr(profile_credentials, "location", None)
 
         info = client_info.ClientInfo(user_agent=f"dbt-bigquery-{dbt_version.version}")
+        options = client_options.ClientOptions(quota_project_id=quota_project)
         return google.cloud.bigquery.Client(
             execution_project,
             creds,
             location=location,
             client_info=info,
+            client_options=options,
         )
 
     @classmethod
diff --git a/tests/functional/test_quota_project.py b/tests/functional/test_quota_project.py
new file mode 100644
index 000000000..0b4bb90c4
--- /dev/null
+++ b/tests/functional/test_quota_project.py
@@ -0,0 +1,27 @@
+import os
+
+import pytest
+
+from dbt.tests.util import run_dbt
+
+_QUOTA_PROJECT = os.getenv("BIGQUERY_TEST_ALT_DATABASE")
+
+
+class TestNoQuotaProject:
+    def test_no_quota_project(self, project):
+        results = run_dbt()
+        for result in results:
+            assert None == result.adapter_response["quota_project"]
+
+
+class TestQuotaProjectOption:
+    @pytest.fixture(scope="class")
+    def profiles_config_update(self, dbt_profile_target):
+        outputs = {"default": dbt_profile_target}
+        outputs["default"]["quota_project"] = _QUOTA_PROJECT
+        yield
+
+    def test_quota_project_option(self, project):
+        results = run_dbt()
+        for result in results:
+            assert _QUOTA_PROJECT == result.adapter_response["quota_project"]
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index a922525fd..da499d266 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -386,15 +386,17 @@ def test_cancel_open_connections_single(self):
         adapter.connections.thread_connections.update({key: master, 1: model})
         self.assertEqual(len(list(adapter.cancel_open_connections())), 1)
 
+    @patch("dbt.adapters.bigquery.impl.google.api_core.client_options.ClientOptions")
     @patch("dbt.adapters.bigquery.impl.google.auth.default")
     @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
-    def test_location_user_agent(self, mock_bq, mock_auth_default):
+    def test_location_user_agent(self, mock_bq, mock_auth_default, MockClientOptions):
         creds = MagicMock()
         mock_auth_default.return_value = (creds, MagicMock())
         adapter = self.get_adapter("loc")
 
         connection = adapter.acquire_connection("dummy")
         mock_client = mock_bq.Client
+        mock_client_options = MockClientOptions.return_value
 
         mock_client.assert_not_called()
         connection.handle
@@ -403,6 +405,7 @@ def test_location_user_agent(self, mock_bq, mock_auth_default):
             creds,
             location="Luna Station",
             client_info=HasUserAgent(),
+            client_options=mock_client_options,
         )
 
 

From 23e8020801526787a72407d9dea3d42016e49ad0 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Mon, 4 Nov 2024 19:25:40 -0500
Subject: [PATCH 847/860] update macos runners from macos-12 to macos-14
 (#1394)

---
 .github/scripts/integration-test-matrix.js | 4 ++--
 .github/workflows/main.yml                 | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/.github/scripts/integration-test-matrix.js b/.github/scripts/integration-test-matrix.js
index 49db45575..bebe08569 100644
--- a/.github/scripts/integration-test-matrix.js
+++ b/.github/scripts/integration-test-matrix.js
@@ -44,7 +44,7 @@ module.exports = ({ context }) => {
 
             if (labels.includes("test macos") || testAllLabel) {
               include.push({
-                os: "macos-12",
+                os: "macos-14",
                 adapter,
                 "python-version": pythonVersion,
               });
@@ -78,7 +78,7 @@ module.exports = ({ context }) => {
   // additionally include runs for all adapters, on macos and windows,
   // but only for the default python version
   for (const adapter of supportedAdapters) {
-    for (const operatingSystem of ["windows-latest", "macos-12"]) {
+    for (const operatingSystem of ["windows-latest", "macos-14"]) {
       include.push({
         os: operatingSystem,
         adapter: adapter,
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 6bfed5df6..7b82f3e0f 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -174,7 +174,7 @@ jobs:
     strategy:
       fail-fast: false
       matrix:
-        os: [ubuntu-latest, macos-12, windows-latest]
+        os: [ubuntu-latest, macos-14, windows-latest]
         python-version: ['3.9', '3.10', '3.11', '3.12']
         dist-type: ["whl", "gz"]
 

From 35c32f181b97f69d2b20a374c7a0dca765455c81 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Tue, 5 Nov 2024 14:03:44 -0500
Subject: [PATCH 848/860] Break out credentials as a separate module (#1391)

* break out credentials into its own module
* fix imports
* update unit test mocks for new location of get_bigquery_defaults
---
 .../Under the Hood-20241104-173815.yaml       |   7 +
 dbt/adapters/bigquery/__init__.py             |  10 +-
 dbt/adapters/bigquery/column.py               |   5 +-
 dbt/adapters/bigquery/connections.py          | 197 ++----------------
 dbt/adapters/bigquery/credentials.py          | 187 +++++++++++++++++
 dbt/adapters/bigquery/dataproc/batch.py       |  13 +-
 dbt/adapters/bigquery/dataset.py              |   4 +-
 dbt/adapters/bigquery/gcloud.py               |  29 ---
 dbt/adapters/bigquery/impl.py                 |  51 +++--
 dbt/adapters/bigquery/python_submissions.py   |  14 +-
 dbt/adapters/bigquery/relation.py             |   9 +-
 tests/unit/test_bigquery_adapter.py           |   4 +-
 tests/unit/test_configure_dataproc_batch.py   |   4 +-
 13 files changed, 275 insertions(+), 259 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20241104-173815.yaml
 create mode 100644 dbt/adapters/bigquery/credentials.py
 delete mode 100644 dbt/adapters/bigquery/gcloud.py

diff --git a/.changes/unreleased/Under the Hood-20241104-173815.yaml b/.changes/unreleased/Under the Hood-20241104-173815.yaml
new file mode 100644
index 000000000..e3e81dec1
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20241104-173815.yaml	
@@ -0,0 +1,7 @@
+kind: Under the Hood
+body: Separate credentials functionality into its own module for reuse in retry and
+  python submissions
+time: 2024-11-04T17:38:15.940962-05:00
+custom:
+  Author: mikealfare
+  Issue: "1391"
diff --git a/dbt/adapters/bigquery/__init__.py b/dbt/adapters/bigquery/__init__.py
index 5fe68e786..74fa17cda 100644
--- a/dbt/adapters/bigquery/__init__.py
+++ b/dbt/adapters/bigquery/__init__.py
@@ -1,8 +1,8 @@
-from dbt.adapters.bigquery.connections import BigQueryConnectionManager  # noqa
-from dbt.adapters.bigquery.connections import BigQueryCredentials
-from dbt.adapters.bigquery.relation import BigQueryRelation  # noqa
-from dbt.adapters.bigquery.column import BigQueryColumn  # noqa
-from dbt.adapters.bigquery.impl import BigQueryAdapter, GrantTarget, PartitionConfig  # noqa
+from dbt.adapters.bigquery.column import BigQueryColumn
+from dbt.adapters.bigquery.connections import BigQueryConnectionManager
+from dbt.adapters.bigquery.credentials import BigQueryCredentials
+from dbt.adapters.bigquery.impl import BigQueryAdapter, GrantTarget, PartitionConfig
+from dbt.adapters.bigquery.relation import BigQueryRelation
 
 from dbt.adapters.base import AdapterPlugin
 from dbt.include import bigquery
diff --git a/dbt/adapters/bigquery/column.py b/dbt/adapters/bigquery/column.py
index 4a12e211f..a676fef4b 100644
--- a/dbt/adapters/bigquery/column.py
+++ b/dbt/adapters/bigquery/column.py
@@ -1,9 +1,10 @@
 from dataclasses import dataclass
-from typing import Optional, List, TypeVar, Iterable, Type, Any, Dict, Union
+from typing import Any, Dict, Iterable, List, Optional, Type, TypeVar, Union
+
+from google.cloud.bigquery import SchemaField
 
 from dbt.adapters.base.column import Column
 
-from google.cloud.bigquery import SchemaField
 
 _PARENT_DATA_TYPE_KEY = "__parent_data_type"
 
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 58b3dbe41..bda54080b 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -1,58 +1,54 @@
 from collections import defaultdict
 from concurrent.futures import TimeoutError
+from contextlib import contextmanager
+from dataclasses import dataclass
 import json
+from multiprocessing.context import SpawnContext
 import re
-from contextlib import contextmanager
-from dataclasses import dataclass, field
+from typing import Dict, Hashable, List, Optional, Tuple, TYPE_CHECKING
 import uuid
-from mashumaro.helper import pass_through
-
-from functools import lru_cache
-from requests.exceptions import ConnectionError
-
-from multiprocessing.context import SpawnContext
-from typing import Optional, Any, Dict, Tuple, Hashable, List, TYPE_CHECKING
 
+from google.api_core import client_info, client_options, retry
 import google.auth
+from google.auth import impersonated_credentials
 import google.auth.exceptions
 import google.cloud.bigquery
 import google.cloud.exceptions
-from google.api_core import retry, client_info, client_options
-from google.auth import impersonated_credentials
 from google.oauth2 import (
     credentials as GoogleCredentials,
     service_account as GoogleServiceAccountCredentials,
 )
+from requests.exceptions import ConnectionError
 
 from dbt_common.events.contextvars import get_node_info
 from dbt_common.events.functions import fire_event
-from dbt_common.exceptions import (
-    DbtRuntimeError,
-    DbtConfigError,
-    DbtDatabaseError,
-)
+from dbt_common.exceptions import DbtDatabaseError, DbtRuntimeError
 from dbt_common.invocation import get_invocation_id
-from dbt.adapters.bigquery import gcloud
+from dbt.adapters.base import BaseConnectionManager
 from dbt.adapters.contracts.connection import (
-    ConnectionState,
-    AdapterResponse,
-    Credentials,
     AdapterRequiredConfig,
+    AdapterResponse,
+    ConnectionState,
 )
-from dbt.adapters.exceptions.connection import FailedToConnectError
-from dbt.adapters.base import BaseConnectionManager
 from dbt.adapters.events.logging import AdapterLogger
 from dbt.adapters.events.types import SQLQuery
-from dbt.adapters.bigquery import __version__ as dbt_version
-from dbt.adapters.bigquery.utility import is_base64, base64_to_string
+from dbt.adapters.exceptions.connection import FailedToConnectError
 
-from dbt_common.dataclass_schema import ExtensibleDbtClassMixin, StrEnum
+import dbt.adapters.bigquery.__version__ as dbt_version
+from dbt.adapters.bigquery.credentials import (
+    BigQueryConnectionMethod,
+    Priority,
+    get_bigquery_defaults,
+    setup_default_credentials,
+)
+from dbt.adapters.bigquery.utility import is_base64, base64_to_string
 
 if TYPE_CHECKING:
     # Indirectly imported via agate_helper, which is lazy loaded further downfile.
     # Used by mypy for earlier type hints.
     import agate
 
+
 logger = AdapterLogger("BigQuery")
 
 BQ_QUERY_JOB_SPLIT = "-----Query Job SQL Follows-----"
@@ -73,33 +69,6 @@
 )
 
 
-@lru_cache()
-def get_bigquery_defaults(scopes=None) -> Tuple[Any, Optional[str]]:
-    """
-    Returns (credentials, project_id)
-
-    project_id is returned available from the environment; otherwise None
-    """
-    # Cached, because the underlying implementation shells out, taking ~1s
-    try:
-        credentials, _ = google.auth.default(scopes=scopes)
-        return credentials, _
-    except google.auth.exceptions.DefaultCredentialsError as e:
-        raise DbtConfigError(f"Failed to authenticate with supplied credentials\nerror:\n{e}")
-
-
-class Priority(StrEnum):
-    Interactive = "interactive"
-    Batch = "batch"
-
-
-class BigQueryConnectionMethod(StrEnum):
-    OAUTH = "oauth"
-    SERVICE_ACCOUNT = "service-account"
-    SERVICE_ACCOUNT_JSON = "service-account-json"
-    OAUTH_SECRETS = "oauth-secrets"
-
-
 @dataclass
 class BigQueryAdapterResponse(AdapterResponse):
     bytes_processed: Optional[int] = None
@@ -110,128 +79,6 @@ class BigQueryAdapterResponse(AdapterResponse):
     slot_ms: Optional[int] = None
 
 
-@dataclass
-class DataprocBatchConfig(ExtensibleDbtClassMixin):
-    def __init__(self, batch_config):
-        self.batch_config = batch_config
-
-
-@dataclass
-class BigQueryCredentials(Credentials):
-    method: BigQueryConnectionMethod = None  # type: ignore
-
-    # BigQuery allows an empty database / project, where it defers to the
-    # environment for the project
-    database: Optional[str] = None
-    schema: Optional[str] = None
-    execution_project: Optional[str] = None
-    quota_project: Optional[str] = None
-    location: Optional[str] = None
-    priority: Optional[Priority] = None
-    maximum_bytes_billed: Optional[int] = None
-    impersonate_service_account: Optional[str] = None
-
-    job_retry_deadline_seconds: Optional[int] = None
-    job_retries: Optional[int] = 1
-    job_creation_timeout_seconds: Optional[int] = None
-    job_execution_timeout_seconds: Optional[int] = None
-
-    # Keyfile json creds (unicode or base 64 encoded)
-    keyfile: Optional[str] = None
-    keyfile_json: Optional[Dict[str, Any]] = None
-
-    # oauth-secrets
-    token: Optional[str] = None
-    refresh_token: Optional[str] = None
-    client_id: Optional[str] = None
-    client_secret: Optional[str] = None
-    token_uri: Optional[str] = None
-
-    dataproc_region: Optional[str] = None
-    dataproc_cluster_name: Optional[str] = None
-    gcs_bucket: Optional[str] = None
-
-    dataproc_batch: Optional[DataprocBatchConfig] = field(
-        metadata={
-            "serialization_strategy": pass_through,
-        },
-        default=None,
-    )
-
-    scopes: Optional[Tuple[str, ...]] = (
-        "https://www.googleapis.com/auth/bigquery",
-        "https://www.googleapis.com/auth/cloud-platform",
-        "https://www.googleapis.com/auth/drive",
-    )
-
-    _ALIASES = {
-        # 'legacy_name': 'current_name'
-        "project": "database",
-        "dataset": "schema",
-        "target_project": "target_database",
-        "target_dataset": "target_schema",
-        "retries": "job_retries",
-        "timeout_seconds": "job_execution_timeout_seconds",
-    }
-
-    def __post_init__(self):
-        if self.keyfile_json and "private_key" in self.keyfile_json:
-            self.keyfile_json["private_key"] = self.keyfile_json["private_key"].replace(
-                "\\n", "\n"
-            )
-        if not self.method:
-            raise DbtRuntimeError("Must specify authentication method")
-
-        if not self.schema:
-            raise DbtRuntimeError("Must specify schema")
-
-    @property
-    def type(self):
-        return "bigquery"
-
-    @property
-    def unique_field(self):
-        return self.database
-
-    def _connection_keys(self):
-        return (
-            "method",
-            "database",
-            "execution_project",
-            "schema",
-            "location",
-            "priority",
-            "maximum_bytes_billed",
-            "impersonate_service_account",
-            "job_retry_deadline_seconds",
-            "job_retries",
-            "job_creation_timeout_seconds",
-            "job_execution_timeout_seconds",
-            "timeout_seconds",
-            "client_id",
-            "token_uri",
-            "dataproc_region",
-            "dataproc_cluster_name",
-            "gcs_bucket",
-            "dataproc_batch",
-        )
-
-    @classmethod
-    def __pre_deserialize__(cls, d: Dict[Any, Any]) -> Dict[Any, Any]:
-        # We need to inject the correct value of the database (aka project) at
-        # this stage, ref
-        # https://github.com/dbt-labs/dbt/pull/2908#discussion_r532927436.
-
-        # `database` is an alias of `project` in BigQuery
-        if "database" not in d:
-            _, database = get_bigquery_defaults()
-            d["database"] = database
-        # `execution_project` default to dataset/project
-        if "execution_project" not in d:
-            d["execution_project"] = d["database"]
-        return d
-
-
 class BigQueryConnectionManager(BaseConnectionManager):
     TYPE = "bigquery"
 
@@ -433,7 +280,7 @@ def open(cls, connection):
 
         except google.auth.exceptions.DefaultCredentialsError:
             logger.info("Please log into GCP to continue")
-            gcloud.setup_default_credentials()
+            setup_default_credentials()
 
             handle = cls.get_bigquery_client(connection.credentials)
 
diff --git a/dbt/adapters/bigquery/credentials.py b/dbt/adapters/bigquery/credentials.py
new file mode 100644
index 000000000..32f172dac
--- /dev/null
+++ b/dbt/adapters/bigquery/credentials.py
@@ -0,0 +1,187 @@
+from dataclasses import dataclass, field
+from functools import lru_cache
+from typing import Any, Dict, Optional, Tuple
+
+import google.auth
+from google.auth.exceptions import DefaultCredentialsError
+from mashumaro import pass_through
+
+from dbt_common.clients.system import run_cmd
+from dbt_common.dataclass_schema import ExtensibleDbtClassMixin, StrEnum
+from dbt_common.exceptions import DbtConfigError, DbtRuntimeError
+from dbt.adapters.contracts.connection import Credentials
+from dbt.adapters.events.logging import AdapterLogger
+
+
+_logger = AdapterLogger("BigQuery")
+
+
+class Priority(StrEnum):
+    Interactive = "interactive"
+    Batch = "batch"
+
+
+class BigQueryConnectionMethod(StrEnum):
+    OAUTH = "oauth"
+    SERVICE_ACCOUNT = "service-account"
+    SERVICE_ACCOUNT_JSON = "service-account-json"
+    OAUTH_SECRETS = "oauth-secrets"
+
+
+@dataclass
+class DataprocBatchConfig(ExtensibleDbtClassMixin):
+    def __init__(self, batch_config):
+        self.batch_config = batch_config
+
+
+@lru_cache()
+def get_bigquery_defaults(scopes=None) -> Tuple[Any, Optional[str]]:
+    """
+    Returns (credentials, project_id)
+
+    project_id is returned available from the environment; otherwise None
+    """
+    # Cached, because the underlying implementation shells out, taking ~1s
+    try:
+        credentials, _ = google.auth.default(scopes=scopes)
+        return credentials, _
+    except DefaultCredentialsError as e:
+        raise DbtConfigError(f"Failed to authenticate with supplied credentials\nerror:\n{e}")
+
+
+def setup_default_credentials():
+    if _gcloud_installed():
+        run_cmd(".", ["gcloud", "auth", "application-default", "login"])
+    else:
+        msg = """
+        dbt requires the gcloud SDK to be installed to authenticate with BigQuery.
+        Please download and install the SDK, or use a Service Account instead.
+
+        https://cloud.google.com/sdk/
+        """
+        raise DbtRuntimeError(msg)
+
+
+def _gcloud_installed():
+    try:
+        run_cmd(".", ["gcloud", "--version"])
+        return True
+    except OSError as e:
+        _logger.debug(e)
+        return False
+
+
+@dataclass
+class BigQueryCredentials(Credentials):
+    method: BigQueryConnectionMethod = None  # type: ignore
+
+    # BigQuery allows an empty database / project, where it defers to the
+    # environment for the project
+    database: Optional[str] = None
+    schema: Optional[str] = None
+    execution_project: Optional[str] = None
+    quota_project: Optional[str] = None
+    location: Optional[str] = None
+    priority: Optional[Priority] = None
+    maximum_bytes_billed: Optional[int] = None
+    impersonate_service_account: Optional[str] = None
+
+    job_retry_deadline_seconds: Optional[int] = None
+    job_retries: Optional[int] = 1
+    job_creation_timeout_seconds: Optional[int] = None
+    job_execution_timeout_seconds: Optional[int] = None
+
+    # Keyfile json creds (unicode or base 64 encoded)
+    keyfile: Optional[str] = None
+    keyfile_json: Optional[Dict[str, Any]] = None
+
+    # oauth-secrets
+    token: Optional[str] = None
+    refresh_token: Optional[str] = None
+    client_id: Optional[str] = None
+    client_secret: Optional[str] = None
+    token_uri: Optional[str] = None
+
+    dataproc_region: Optional[str] = None
+    dataproc_cluster_name: Optional[str] = None
+    gcs_bucket: Optional[str] = None
+
+    dataproc_batch: Optional[DataprocBatchConfig] = field(
+        metadata={
+            "serialization_strategy": pass_through,
+        },
+        default=None,
+    )
+
+    scopes: Optional[Tuple[str, ...]] = (
+        "https://www.googleapis.com/auth/bigquery",
+        "https://www.googleapis.com/auth/cloud-platform",
+        "https://www.googleapis.com/auth/drive",
+    )
+
+    _ALIASES = {
+        # 'legacy_name': 'current_name'
+        "project": "database",
+        "dataset": "schema",
+        "target_project": "target_database",
+        "target_dataset": "target_schema",
+        "retries": "job_retries",
+        "timeout_seconds": "job_execution_timeout_seconds",
+    }
+
+    def __post_init__(self):
+        if self.keyfile_json and "private_key" in self.keyfile_json:
+            self.keyfile_json["private_key"] = self.keyfile_json["private_key"].replace(
+                "\\n", "\n"
+            )
+        if not self.method:
+            raise DbtRuntimeError("Must specify authentication method")
+
+        if not self.schema:
+            raise DbtRuntimeError("Must specify schema")
+
+    @property
+    def type(self):
+        return "bigquery"
+
+    @property
+    def unique_field(self):
+        return self.database
+
+    def _connection_keys(self):
+        return (
+            "method",
+            "database",
+            "execution_project",
+            "schema",
+            "location",
+            "priority",
+            "maximum_bytes_billed",
+            "impersonate_service_account",
+            "job_retry_deadline_seconds",
+            "job_retries",
+            "job_creation_timeout_seconds",
+            "job_execution_timeout_seconds",
+            "timeout_seconds",
+            "client_id",
+            "token_uri",
+            "dataproc_region",
+            "dataproc_cluster_name",
+            "gcs_bucket",
+            "dataproc_batch",
+        )
+
+    @classmethod
+    def __pre_deserialize__(cls, d: Dict[Any, Any]) -> Dict[Any, Any]:
+        # We need to inject the correct value of the database (aka project) at
+        # this stage, ref
+        # https://github.com/dbt-labs/dbt/pull/2908#discussion_r532927436.
+
+        # `database` is an alias of `project` in BigQuery
+        if "database" not in d:
+            _, database = get_bigquery_defaults()
+            d["database"] = database
+        # `execution_project` default to dataset/project
+        if "execution_project" not in d:
+            d["execution_project"] = d["database"]
+        return d
diff --git a/dbt/adapters/bigquery/dataproc/batch.py b/dbt/adapters/bigquery/dataproc/batch.py
index e7f13c913..59f40d246 100644
--- a/dbt/adapters/bigquery/dataproc/batch.py
+++ b/dbt/adapters/bigquery/dataproc/batch.py
@@ -1,16 +1,17 @@
-from typing import Union, Dict
-
-import time
 from datetime import datetime
+import time
+from typing import Dict, Union
+
 from google.cloud.dataproc_v1 import (
-    CreateBatchRequest,
-    BatchControllerClient,
     Batch,
+    BatchControllerClient,
+    CreateBatchRequest,
     GetBatchRequest,
 )
 from google.protobuf.json_format import ParseDict
 
-from dbt.adapters.bigquery.connections import DataprocBatchConfig
+from dbt.adapters.bigquery.credentials import DataprocBatchConfig
+
 
 _BATCH_RUNNING_STATES = [Batch.State.PENDING, Batch.State.RUNNING]
 DEFAULT_JAR_FILE_URI = "gs://spark-lib/bigquery/spark-bigquery-with-dependencies_2.13-0.34.0.jar"
diff --git a/dbt/adapters/bigquery/dataset.py b/dbt/adapters/bigquery/dataset.py
index 4ecd6daa5..a4504294a 100644
--- a/dbt/adapters/bigquery/dataset.py
+++ b/dbt/adapters/bigquery/dataset.py
@@ -1,8 +1,10 @@
 from typing import List
-from google.cloud.bigquery import Dataset, AccessEntry
+
+from google.cloud.bigquery import AccessEntry, Dataset
 
 from dbt.adapters.events.logging import AdapterLogger
 
+
 logger = AdapterLogger("BigQuery")
 
 
diff --git a/dbt/adapters/bigquery/gcloud.py b/dbt/adapters/bigquery/gcloud.py
deleted file mode 100644
index ea1f644ba..000000000
--- a/dbt/adapters/bigquery/gcloud.py
+++ /dev/null
@@ -1,29 +0,0 @@
-from dbt_common.exceptions import DbtRuntimeError
-
-from dbt.adapters.events.logging import AdapterLogger
-from dbt_common.clients.system import run_cmd
-
-NOT_INSTALLED_MSG = """
-dbt requires the gcloud SDK to be installed to authenticate with BigQuery.
-Please download and install the SDK, or use a Service Account instead.
-
-https://cloud.google.com/sdk/
-"""
-
-logger = AdapterLogger("BigQuery")
-
-
-def gcloud_installed():
-    try:
-        run_cmd(".", ["gcloud", "--version"])
-        return True
-    except OSError as e:
-        logger.debug(e)
-        return False
-
-
-def setup_default_credentials():
-    if gcloud_installed():
-        run_cmd(".", ["gcloud", "auth", "application-default", "login"])
-    else:
-        raise DbtRuntimeError(NOT_INSTALLED_MSG)
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index 0b49c0373..cf5800fd3 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -1,27 +1,41 @@
 from dataclasses import dataclass
 from datetime import datetime
 import json
-import threading
 from multiprocessing.context import SpawnContext
-
+import threading
 import time
 from typing import (
     Any,
     Dict,
+    FrozenSet,
+    Iterable,
     List,
     Optional,
+    Tuple,
+    TYPE_CHECKING,
     Type,
     Set,
     Union,
-    FrozenSet,
-    Tuple,
-    Iterable,
-    TYPE_CHECKING,
 )
 
-from dbt.adapters.contracts.relation import RelationConfig
+import google.api_core
+import google.auth
+import google.oauth2
+import google.cloud.bigquery
+from google.cloud.bigquery import AccessEntry, SchemaField, Table as BigQueryTable
+import google.cloud.exceptions
+import pytz
 
+from dbt_common.contracts.constraints import (
+    ColumnLevelConstraint,
+    ConstraintType,
+    ModelLevelConstraint,
+)
+from dbt_common.dataclass_schema import dbtClassMixin
+from dbt_common.events.functions import fire_event
+import dbt_common.exceptions
 import dbt_common.exceptions.base
+from dbt_common.utils import filter_null_values
 from dbt.adapters.base import (
     AdapterConfig,
     BaseAdapter,
@@ -37,28 +51,12 @@
 from dbt.adapters.capability import Capability, CapabilityDict, CapabilitySupport, Support
 from dbt.adapters.contracts.connection import AdapterResponse
 from dbt.adapters.contracts.macros import MacroResolverProtocol
-from dbt_common.contracts.constraints import (
-    ColumnLevelConstraint,
-    ConstraintType,
-    ModelLevelConstraint,
-)
-from dbt_common.dataclass_schema import dbtClassMixin
+from dbt.adapters.contracts.relation import RelationConfig
 from dbt.adapters.events.logging import AdapterLogger
-from dbt_common.events.functions import fire_event
 from dbt.adapters.events.types import SchemaCreation, SchemaDrop
-import dbt_common.exceptions
-from dbt_common.utils import filter_null_values
-import google.api_core
-import google.auth
-import google.oauth2
-import google.cloud.bigquery
-from google.cloud.bigquery import AccessEntry, SchemaField, Table as BigQueryTable
-import google.cloud.exceptions
-import pytz
 
-from dbt.adapters.bigquery import BigQueryColumn, BigQueryConnectionManager
-from dbt.adapters.bigquery.column import get_nested_column_data_types
-from dbt.adapters.bigquery.connections import BigQueryAdapterResponse
+from dbt.adapters.bigquery.column import BigQueryColumn, get_nested_column_data_types
+from dbt.adapters.bigquery.connections import BigQueryAdapterResponse, BigQueryConnectionManager
 from dbt.adapters.bigquery.dataset import add_access_entry_to_dataset, is_access_entry_in_dataset
 from dbt.adapters.bigquery.python_submissions import (
     ClusterDataprocHelper,
@@ -77,6 +75,7 @@
     # Used by mypy for earlier type hints.
     import agate
 
+
 logger = AdapterLogger("BigQuery")
 
 # Write dispositions for bigquery.
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 368ed9d07..93c82ca92 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -1,21 +1,21 @@
 import uuid
 from typing import Dict, Union
 
-from dbt.adapters.events.logging import AdapterLogger
-
-from dbt.adapters.base import PythonJobHelper
-from google.api_core.future.polling import POLLING_PREDICATE
-
-from dbt.adapters.bigquery import BigQueryConnectionManager, BigQueryCredentials
 from google.api_core import retry
 from google.api_core.client_options import ClientOptions
+from google.api_core.future.polling import POLLING_PREDICATE
 from google.cloud import storage, dataproc_v1
 from google.cloud.dataproc_v1.types.batches import Batch
 
+from dbt.adapters.base import PythonJobHelper
+from dbt.adapters.events.logging import AdapterLogger
+
+from dbt.adapters.bigquery.connections import BigQueryConnectionManager
+from dbt.adapters.bigquery.credentials import BigQueryCredentials
 from dbt.adapters.bigquery.dataproc.batch import (
+    DEFAULT_JAR_FILE_URI,
     create_batch_request,
     poll_batch_job,
-    DEFAULT_JAR_FILE_URI,
     update_batch_from_config,
 )
 
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index 0e2c17670..4edc8d7ac 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -1,9 +1,13 @@
 from dataclasses import dataclass, field
+from itertools import chain, islice
 from typing import FrozenSet, Optional, TypeVar
 
-from itertools import chain, islice
+from dbt_common.exceptions import CompilationError
+from dbt_common.utils.dict import filter_null_values
 from dbt.adapters.base.relation import BaseRelation, ComponentName, InformationSchema
+from dbt.adapters.contracts.relation import RelationConfig, RelationType
 from dbt.adapters.relation_configs import RelationConfigChangeAction
+
 from dbt.adapters.bigquery.relation_configs import (
     BigQueryClusterConfigChange,
     BigQueryMaterializedViewConfig,
@@ -11,9 +15,6 @@
     BigQueryOptionsConfigChange,
     BigQueryPartitionConfigChange,
 )
-from dbt.adapters.contracts.relation import RelationType, RelationConfig
-from dbt_common.exceptions import CompilationError
-from dbt_common.utils.dict import filter_null_values
 
 
 Self = TypeVar("Self", bound="BigQueryRelation")
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index da499d266..ca3bfc24c 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -203,7 +203,7 @@ def get_adapter(self, target) -> BigQueryAdapter:
 
 class TestBigQueryAdapterAcquire(BaseTestBigQueryAdapter):
     @patch(
-        "dbt.adapters.bigquery.connections.get_bigquery_defaults",
+        "dbt.adapters.bigquery.credentials.get_bigquery_defaults",
         return_value=("credentials", "project_id"),
     )
     @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
@@ -244,7 +244,7 @@ def test_acquire_connection_oauth_validations(self, mock_open_connection):
         mock_open_connection.assert_called_once()
 
     @patch(
-        "dbt.adapters.bigquery.connections.get_bigquery_defaults",
+        "dbt.adapters.bigquery.credentials.get_bigquery_defaults",
         return_value=("credentials", "project_id"),
     )
     @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
diff --git a/tests/unit/test_configure_dataproc_batch.py b/tests/unit/test_configure_dataproc_batch.py
index 94cb28efb..f56aee129 100644
--- a/tests/unit/test_configure_dataproc_batch.py
+++ b/tests/unit/test_configure_dataproc_batch.py
@@ -12,7 +12,7 @@
 # parsed credentials
 class TestConfigureDataprocBatch(BaseTestBigQueryAdapter):
     @patch(
-        "dbt.adapters.bigquery.connections.get_bigquery_defaults",
+        "dbt.adapters.bigquery.credentials.get_bigquery_defaults",
         return_value=("credentials", "project_id"),
     )
     def test_update_dataproc_serverless_batch(self, mock_get_bigquery_defaults):
@@ -64,7 +64,7 @@ def to_str_values(d):
         )
 
     @patch(
-        "dbt.adapters.bigquery.connections.get_bigquery_defaults",
+        "dbt.adapters.bigquery.credentials.get_bigquery_defaults",
         return_value=("credentials", "project_id"),
     )
     def test_default_dataproc_serverless_batch(self, mock_get_bigquery_defaults):

From 1fe273a9de73a950429f00b90a48b95cf91d328f Mon Sep 17 00:00:00 2001
From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com>
Date: Fri, 8 Nov 2024 20:09:29 -0500
Subject: [PATCH 849/860] Update wheel requirement from ~=0.42 to ~=0.43
 (#1304)

* Update wheel requirement from ~=0.42 to ~=0.43

Updates the requirements on [wheel](https://github.com/pypa/wheel) to permit the latest version.
- [Release notes](https://github.com/pypa/wheel/releases)
- [Changelog](https://github.com/pypa/wheel/blob/main/docs/news.rst)
- [Commits](https://github.com/pypa/wheel/compare/0.42.0...0.43.0)

---
updated-dependencies:
- dependency-name: wheel
  dependency-type: direct:development
...

Signed-off-by: dependabot[bot] <support@github.com>

* Add automated changelog yaml from template for bot PR

---------

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: Github Build Bot <buildbot@fishtownanalytics.com>
Co-authored-by: Matthew McKnight <91097623+McKnight-42@users.noreply.github.com>
Co-authored-by: Mila Page <67295367+VersusFacit@users.noreply.github.com>
Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 .changes/unreleased/Dependencies-20240724-040744.yaml | 6 ++++++
 1 file changed, 6 insertions(+)
 create mode 100644 .changes/unreleased/Dependencies-20240724-040744.yaml

diff --git a/.changes/unreleased/Dependencies-20240724-040744.yaml b/.changes/unreleased/Dependencies-20240724-040744.yaml
new file mode 100644
index 000000000..fd713788e
--- /dev/null
+++ b/.changes/unreleased/Dependencies-20240724-040744.yaml
@@ -0,0 +1,6 @@
+kind: "Dependencies"
+body: "Update wheel requirement from ~=0.42 to ~=0.43"
+time: 2024-07-24T04:07:44.00000Z
+custom:
+  Author: dependabot[bot]
+  PR: 1304

From 75142ac7f18ae94b6171461ec379b97ebf79a0e4 Mon Sep 17 00:00:00 2001
From: Yasuhisa Yoshida <syou6162@gmail.com>
Date: Tue, 19 Nov 2024 03:46:14 +0900
Subject: [PATCH 850/860] Remove unused stub function (#1393)

Co-authored-by: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
---
 dbt/adapters/bigquery/impl.py | 6 ------
 1 file changed, 6 deletions(-)

diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index cf5800fd3..f6470e7f7 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -95,12 +95,6 @@ def render(self):
         return f"{self.project}.{self.dataset}"
 
 
-def _stub_relation(*args, **kwargs):
-    return BigQueryRelation.create(
-        database="", schema="", identifier="", quote_policy={}, type=BigQueryRelation.Table
-    )
-
-
 @dataclass
 class BigqueryConfig(AdapterConfig):
     cluster_by: Optional[Union[List[str], str]] = None

From 83bb413d770fa1adb68cdb738dde0351347811ad Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Wed, 20 Nov 2024 00:41:32 -0500
Subject: [PATCH 851/860] Add retry factory to consolidate retry strategies
 across dbt-bigquery (#1395)

* fix imports
* create a retry factory and move relevant objects from connections
* add on_error method for deadline retries
* remove dependency on retry_and_handle from cancel_open
* remove dependencies on retry_and_handle
* remove timeout methods from connection manager
* add retry to get_bq_table
* move client factory to credentials module so that on_error can be moved to the retry factory in the retry module
* move on_error factory to retry module
* move client factories from python_submissions module to credentials module
* create a clients module
* retry all client factories by default
* move polling from manual check in python_submissions module into retry_factory
* move load_dataframe logic from adapter to connection manager, use the built-in timeout argument instead of a manual polling method
* move upload_file logic from adapter to connection manager, use the built-in timeout argument instead of a manual polling method, remove the manual polling method
* move the retry to polling for done instead of create
* align new retries with original methods, simplify retry factory
* create a method for the dataproc endpoint
* make imports explicit, remove unused constant
* update names in clients.py to follow the naming convention
* update names in connections.py to follow the naming convention
* update names in credentials.py to follow the naming convention
* update names in python_submissions.py to follow the naming convention
* update names in retry.py to follow the naming convention

---------

Co-authored-by: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .../Under the Hood-20241107-143856.yaml       |   6 +
 dbt/adapters/bigquery/clients.py              |  69 +++
 dbt/adapters/bigquery/connections.py          | 434 +++++++-----------
 dbt/adapters/bigquery/credentials.py          | 174 +++++--
 dbt/adapters/bigquery/dataproc/__init__.py    |   0
 dbt/adapters/bigquery/dataproc/batch.py       |  68 ---
 dbt/adapters/bigquery/impl.py                 | 105 ++---
 dbt/adapters/bigquery/python_submissions.py   | 250 +++++-----
 dbt/adapters/bigquery/retry.py                | 128 ++++++
 dbt/adapters/bigquery/utility.py              |  40 +-
 tests/conftest.py                             |   8 +-
 tests/functional/adapter/test_json_keyfile.py |  13 +-
 tests/unit/test_bigquery_adapter.py           |  21 +-
 .../unit/test_bigquery_connection_manager.py  | 104 ++---
 tests/unit/test_configure_dataproc_batch.py   |   8 +-
 15 files changed, 713 insertions(+), 715 deletions(-)
 create mode 100644 .changes/unreleased/Under the Hood-20241107-143856.yaml
 create mode 100644 dbt/adapters/bigquery/clients.py
 delete mode 100644 dbt/adapters/bigquery/dataproc/__init__.py
 delete mode 100644 dbt/adapters/bigquery/dataproc/batch.py
 create mode 100644 dbt/adapters/bigquery/retry.py

diff --git a/.changes/unreleased/Under the Hood-20241107-143856.yaml b/.changes/unreleased/Under the Hood-20241107-143856.yaml
new file mode 100644
index 000000000..db8557bf0
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20241107-143856.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Create a retry factory to simplify retry strategies across dbt-bigquery
+time: 2024-11-07T14:38:56.210445-05:00
+custom:
+  Author: mikealfare osalama
+  Issue: "1395"
diff --git a/dbt/adapters/bigquery/clients.py b/dbt/adapters/bigquery/clients.py
new file mode 100644
index 000000000..18c59fc12
--- /dev/null
+++ b/dbt/adapters/bigquery/clients.py
@@ -0,0 +1,69 @@
+from google.api_core.client_info import ClientInfo
+from google.api_core.client_options import ClientOptions
+from google.api_core.retry import Retry
+from google.auth.exceptions import DefaultCredentialsError
+from google.cloud.bigquery import Client as BigQueryClient
+from google.cloud.dataproc_v1 import BatchControllerClient, JobControllerClient
+from google.cloud.storage import Client as StorageClient
+
+from dbt.adapters.events.logging import AdapterLogger
+
+import dbt.adapters.bigquery.__version__ as dbt_version
+from dbt.adapters.bigquery.credentials import (
+    BigQueryCredentials,
+    create_google_credentials,
+    set_default_credentials,
+)
+
+
+_logger = AdapterLogger("BigQuery")
+
+
+def create_bigquery_client(credentials: BigQueryCredentials) -> BigQueryClient:
+    try:
+        return _create_bigquery_client(credentials)
+    except DefaultCredentialsError:
+        _logger.info("Please log into GCP to continue")
+        set_default_credentials()
+        return _create_bigquery_client(credentials)
+
+
+@Retry()  # google decorator. retries on transient errors with exponential backoff
+def create_gcs_client(credentials: BigQueryCredentials) -> StorageClient:
+    return StorageClient(
+        project=credentials.execution_project,
+        credentials=create_google_credentials(credentials),
+    )
+
+
+@Retry()  # google decorator. retries on transient errors with exponential backoff
+def create_dataproc_job_controller_client(credentials: BigQueryCredentials) -> JobControllerClient:
+    return JobControllerClient(
+        credentials=create_google_credentials(credentials),
+        client_options=ClientOptions(api_endpoint=_dataproc_endpoint(credentials)),
+    )
+
+
+@Retry()  # google decorator. retries on transient errors with exponential backoff
+def create_dataproc_batch_controller_client(
+    credentials: BigQueryCredentials,
+) -> BatchControllerClient:
+    return BatchControllerClient(
+        credentials=create_google_credentials(credentials),
+        client_options=ClientOptions(api_endpoint=_dataproc_endpoint(credentials)),
+    )
+
+
+@Retry()  # google decorator. retries on transient errors with exponential backoff
+def _create_bigquery_client(credentials: BigQueryCredentials) -> BigQueryClient:
+    return BigQueryClient(
+        credentials.execution_project,
+        create_google_credentials(credentials),
+        location=getattr(credentials, "location", None),
+        client_info=ClientInfo(user_agent=f"dbt-bigquery-{dbt_version.version}"),
+        client_options=ClientOptions(quota_project_id=credentials.quota_project),
+    )
+
+
+def _dataproc_endpoint(credentials: BigQueryCredentials) -> str:
+    return f"{credentials.dataproc_region}-dataproc.googleapis.com:443"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index bda54080b..61fa87d40 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -8,17 +8,20 @@
 from typing import Dict, Hashable, List, Optional, Tuple, TYPE_CHECKING
 import uuid
 
-from google.api_core import client_info, client_options, retry
-import google.auth
-from google.auth import impersonated_credentials
-import google.auth.exceptions
-import google.cloud.bigquery
-import google.cloud.exceptions
-from google.oauth2 import (
-    credentials as GoogleCredentials,
-    service_account as GoogleServiceAccountCredentials,
+from google.auth.exceptions import RefreshError
+from google.cloud.bigquery import (
+    Client,
+    CopyJobConfig,
+    Dataset,
+    DatasetReference,
+    LoadJobConfig,
+    QueryJobConfig,
+    QueryPriority,
+    SchemaField,
+    Table,
+    TableReference,
 )
-from requests.exceptions import ConnectionError
+from google.cloud.exceptions import BadRequest, Forbidden, NotFound
 
 from dbt_common.events.contextvars import get_node_info
 from dbt_common.events.functions import fire_event
@@ -34,14 +37,9 @@
 from dbt.adapters.events.types import SQLQuery
 from dbt.adapters.exceptions.connection import FailedToConnectError
 
-import dbt.adapters.bigquery.__version__ as dbt_version
-from dbt.adapters.bigquery.credentials import (
-    BigQueryConnectionMethod,
-    Priority,
-    get_bigquery_defaults,
-    setup_default_credentials,
-)
-from dbt.adapters.bigquery.utility import is_base64, base64_to_string
+from dbt.adapters.bigquery.clients import create_bigquery_client
+from dbt.adapters.bigquery.credentials import Priority
+from dbt.adapters.bigquery.retry import RetryFactory
 
 if TYPE_CHECKING:
     # Indirectly imported via agate_helper, which is lazy loaded further downfile.
@@ -51,22 +49,8 @@
 
 logger = AdapterLogger("BigQuery")
 
-BQ_QUERY_JOB_SPLIT = "-----Query Job SQL Follows-----"
-
-WRITE_TRUNCATE = google.cloud.bigquery.job.WriteDisposition.WRITE_TRUNCATE
 
-REOPENABLE_ERRORS = (
-    ConnectionResetError,
-    ConnectionError,
-)
-
-RETRYABLE_ERRORS = (
-    google.cloud.exceptions.ServerError,
-    google.cloud.exceptions.BadRequest,
-    google.cloud.exceptions.BadGateway,
-    ConnectionResetError,
-    ConnectionError,
-)
+BQ_QUERY_JOB_SPLIT = "-----Query Job SQL Follows-----"
 
 
 @dataclass
@@ -82,12 +66,10 @@ class BigQueryAdapterResponse(AdapterResponse):
 class BigQueryConnectionManager(BaseConnectionManager):
     TYPE = "bigquery"
 
-    DEFAULT_INITIAL_DELAY = 1.0  # Seconds
-    DEFAULT_MAXIMUM_DELAY = 3.0  # Seconds
-
     def __init__(self, profile: AdapterRequiredConfig, mp_context: SpawnContext):
         super().__init__(profile, mp_context)
         self.jobs_by_thread: Dict[Hashable, List[str]] = defaultdict(list)
+        self._retry = RetryFactory(profile.credentials)
 
     @classmethod
     def handle_error(cls, error, message):
@@ -108,19 +90,19 @@ def exception_handler(self, sql):
         try:
             yield
 
-        except google.cloud.exceptions.BadRequest as e:
+        except BadRequest as e:
             message = "Bad request while running query"
             self.handle_error(e, message)
 
-        except google.cloud.exceptions.Forbidden as e:
+        except Forbidden as e:
             message = "Access denied while running query"
             self.handle_error(e, message)
 
-        except google.cloud.exceptions.NotFound as e:
+        except NotFound as e:
             message = "Not found while running query"
             self.handle_error(e, message)
 
-        except google.auth.exceptions.RefreshError as e:
+        except RefreshError as e:
             message = (
                 "Unable to generate access token, if you're using "
                 "impersonate_service_account, make sure your "
@@ -153,15 +135,15 @@ def cancel_open(self):
             for thread_id, connection in self.thread_connections.items():
                 if connection is this_connection:
                     continue
+
                 if connection.handle is not None and connection.state == ConnectionState.OPEN:
-                    client = connection.handle
+                    client: Client = connection.handle
                     for job_id in self.jobs_by_thread.get(thread_id, []):
-
-                        def fn():
-                            return client.cancel_job(job_id)
-
-                        self._retry_and_handle(msg=f"Cancel job: {job_id}", conn=connection, fn=fn)
-
+                        with self.exception_handler(f"Cancel job: {job_id}"):
+                            client.cancel_job(
+                                job_id,
+                                retry=self._retry.create_reopen_with_deadline(connection),
+                            )
                     self.close(connection)
 
                 if connection.name is not None:
@@ -203,121 +185,23 @@ def format_rows_number(self, rows_number):
         rows_number *= 1000.0
         return f"{rows_number:3.1f}{unit}".strip()
 
-    @classmethod
-    def get_google_credentials(cls, profile_credentials) -> GoogleCredentials:
-        method = profile_credentials.method
-        creds = GoogleServiceAccountCredentials.Credentials
-
-        if method == BigQueryConnectionMethod.OAUTH:
-            credentials, _ = get_bigquery_defaults(scopes=profile_credentials.scopes)
-            return credentials
-
-        elif method == BigQueryConnectionMethod.SERVICE_ACCOUNT:
-            keyfile = profile_credentials.keyfile
-            return creds.from_service_account_file(keyfile, scopes=profile_credentials.scopes)
-
-        elif method == BigQueryConnectionMethod.SERVICE_ACCOUNT_JSON:
-            details = profile_credentials.keyfile_json
-            if is_base64(profile_credentials.keyfile_json):
-                details = base64_to_string(details)
-            return creds.from_service_account_info(details, scopes=profile_credentials.scopes)
-
-        elif method == BigQueryConnectionMethod.OAUTH_SECRETS:
-            return GoogleCredentials.Credentials(
-                token=profile_credentials.token,
-                refresh_token=profile_credentials.refresh_token,
-                client_id=profile_credentials.client_id,
-                client_secret=profile_credentials.client_secret,
-                token_uri=profile_credentials.token_uri,
-                scopes=profile_credentials.scopes,
-            )
-
-        error = 'Invalid `method` in profile: "{}"'.format(method)
-        raise FailedToConnectError(error)
-
-    @classmethod
-    def get_impersonated_credentials(cls, profile_credentials):
-        source_credentials = cls.get_google_credentials(profile_credentials)
-        return impersonated_credentials.Credentials(
-            source_credentials=source_credentials,
-            target_principal=profile_credentials.impersonate_service_account,
-            target_scopes=list(profile_credentials.scopes),
-        )
-
-    @classmethod
-    def get_credentials(cls, profile_credentials):
-        if profile_credentials.impersonate_service_account:
-            return cls.get_impersonated_credentials(profile_credentials)
-        else:
-            return cls.get_google_credentials(profile_credentials)
-
-    @classmethod
-    @retry.Retry()  # google decorator. retries on transient errors with exponential backoff
-    def get_bigquery_client(cls, profile_credentials):
-        creds = cls.get_credentials(profile_credentials)
-        execution_project = profile_credentials.execution_project
-        quota_project = profile_credentials.quota_project
-        location = getattr(profile_credentials, "location", None)
-
-        info = client_info.ClientInfo(user_agent=f"dbt-bigquery-{dbt_version.version}")
-        options = client_options.ClientOptions(quota_project_id=quota_project)
-        return google.cloud.bigquery.Client(
-            execution_project,
-            creds,
-            location=location,
-            client_info=info,
-            client_options=options,
-        )
-
     @classmethod
     def open(cls, connection):
-        if connection.state == "open":
+        if connection.state == ConnectionState.OPEN:
             logger.debug("Connection is already open, skipping open.")
             return connection
 
         try:
-            handle = cls.get_bigquery_client(connection.credentials)
-
-        except google.auth.exceptions.DefaultCredentialsError:
-            logger.info("Please log into GCP to continue")
-            setup_default_credentials()
-
-            handle = cls.get_bigquery_client(connection.credentials)
+            connection.handle = create_bigquery_client(connection.credentials)
+            connection.state = ConnectionState.OPEN
+            return connection
 
         except Exception as e:
-            logger.debug(
-                "Got an error when attempting to create a bigquery " "client: '{}'".format(e)
-            )
-
+            logger.debug(f"""Got an error when attempting to create a bigquery " "client: '{e}'""")
             connection.handle = None
-            connection.state = "fail"
-
+            connection.state = ConnectionState.FAIL
             raise FailedToConnectError(str(e))
 
-        connection.handle = handle
-        connection.state = "open"
-        return connection
-
-    @classmethod
-    def get_job_execution_timeout_seconds(cls, conn):
-        credentials = conn.credentials
-        return credentials.job_execution_timeout_seconds
-
-    @classmethod
-    def get_job_retries(cls, conn) -> int:
-        credentials = conn.credentials
-        return credentials.job_retries
-
-    @classmethod
-    def get_job_creation_timeout_seconds(cls, conn):
-        credentials = conn.credentials
-        return credentials.job_creation_timeout_seconds
-
-    @classmethod
-    def get_job_retry_deadline_seconds(cls, conn):
-        credentials = conn.credentials
-        return credentials.job_retry_deadline_seconds
-
     @classmethod
     def get_table_from_response(cls, resp) -> "agate.Table":
         from dbt_common.clients import agate_helper
@@ -357,7 +241,6 @@ def raw_execute(
         dry_run: bool = False,
     ):
         conn = self.get_thread_connection()
-        client = conn.handle
 
         fire_event(SQLQuery(conn_name=conn.name, sql=sql, node_info=get_node_info()))
 
@@ -373,34 +256,25 @@ def raw_execute(
 
         priority = conn.credentials.priority
         if priority == Priority.Batch:
-            job_params["priority"] = google.cloud.bigquery.QueryPriority.BATCH
+            job_params["priority"] = QueryPriority.BATCH
         else:
-            job_params["priority"] = google.cloud.bigquery.QueryPriority.INTERACTIVE
+            job_params["priority"] = QueryPriority.INTERACTIVE
 
         maximum_bytes_billed = conn.credentials.maximum_bytes_billed
         if maximum_bytes_billed is not None and maximum_bytes_billed != 0:
             job_params["maximum_bytes_billed"] = maximum_bytes_billed
 
-        job_creation_timeout = self.get_job_creation_timeout_seconds(conn)
-        job_execution_timeout = self.get_job_execution_timeout_seconds(conn)
-
-        def fn():
+        with self.exception_handler(sql):
             job_id = self.generate_job_id()
 
             return self._query_and_results(
-                client,
+                conn,
                 sql,
                 job_params,
                 job_id,
-                job_creation_timeout=job_creation_timeout,
-                job_execution_timeout=job_execution_timeout,
                 limit=limit,
             )
 
-        query_job, iterator = self._retry_and_handle(msg=sql, conn=conn, fn=fn)
-
-        return query_job, iterator
-
     def execute(
         self, sql, auto_begin=False, fetch=None, limit: Optional[int] = None
     ) -> Tuple[BigQueryAdapterResponse, "agate.Table"]:
@@ -528,9 +402,9 @@ def standard_to_legacy(table):
         _, iterator = self.raw_execute(sql, use_legacy_sql=True)
         return self.get_table_from_response(iterator)
 
-    def copy_bq_table(self, source, destination, write_disposition):
+    def copy_bq_table(self, source, destination, write_disposition) -> None:
         conn = self.get_thread_connection()
-        client = conn.handle
+        client: Client = conn.handle
 
         # -------------------------------------------------------------------------------
         #  BigQuery allows to use copy API using two different formats:
@@ -558,89 +432,149 @@ def copy_bq_table(self, source, destination, write_disposition):
             write_disposition,
         )
 
-        def copy_and_results():
-            job_config = google.cloud.bigquery.CopyJobConfig(write_disposition=write_disposition)
-            copy_job = client.copy_table(source_ref_array, destination_ref, job_config=job_config)
-            timeout = self.get_job_execution_timeout_seconds(conn) or 300
-            iterator = copy_job.result(timeout=timeout)
-            return copy_job, iterator
-
-        self._retry_and_handle(
-            msg='copy table "{}" to "{}"'.format(
-                ", ".join(source_ref.path for source_ref in source_ref_array),
-                destination_ref.path,
-            ),
-            conn=conn,
-            fn=copy_and_results,
+        msg = 'copy table "{}" to "{}"'.format(
+            ", ".join(source_ref.path for source_ref in source_ref_array),
+            destination_ref.path,
+        )
+        with self.exception_handler(msg):
+            copy_job = client.copy_table(
+                source_ref_array,
+                destination_ref,
+                job_config=CopyJobConfig(write_disposition=write_disposition),
+                retry=self._retry.create_reopen_with_deadline(conn),
+            )
+            copy_job.result(timeout=self._retry.create_job_execution_timeout(fallback=300))
+
+    def write_dataframe_to_table(
+        self,
+        client: Client,
+        file_path: str,
+        database: str,
+        schema: str,
+        identifier: str,
+        table_schema: List[SchemaField],
+        field_delimiter: str,
+        fallback_timeout: Optional[float] = None,
+    ) -> None:
+        load_config = LoadJobConfig(
+            skip_leading_rows=1,
+            schema=table_schema,
+            field_delimiter=field_delimiter,
         )
+        table = self.table_ref(database, schema, identifier)
+        self._write_file_to_table(client, file_path, table, load_config, fallback_timeout)
+
+    def write_file_to_table(
+        self,
+        client: Client,
+        file_path: str,
+        database: str,
+        schema: str,
+        identifier: str,
+        fallback_timeout: Optional[float] = None,
+        **kwargs,
+    ) -> None:
+        config = kwargs["kwargs"]
+        if "schema" in config:
+            config["schema"] = json.load(config["schema"])
+        load_config = LoadJobConfig(**config)
+        table = self.table_ref(database, schema, identifier)
+        self._write_file_to_table(client, file_path, table, load_config, fallback_timeout)
+
+    def _write_file_to_table(
+        self,
+        client: Client,
+        file_path: str,
+        table: TableReference,
+        config: LoadJobConfig,
+        fallback_timeout: Optional[float] = None,
+    ) -> None:
+
+        with self.exception_handler("LOAD TABLE"):
+            with open(file_path, "rb") as f:
+                job = client.load_table_from_file(f, table, rewind=True, job_config=config)
+
+        response = job.result(retry=self._retry.create_retry(fallback=fallback_timeout))
+
+        if response.state != "DONE":
+            raise DbtRuntimeError("BigQuery Timeout Exceeded")
+
+        elif response.error_result:
+            message = "\n".join(error["message"].strip() for error in response.errors)
+            raise DbtRuntimeError(message)
 
     @staticmethod
     def dataset_ref(database, schema):
-        return google.cloud.bigquery.DatasetReference(project=database, dataset_id=schema)
+        return DatasetReference(project=database, dataset_id=schema)
 
     @staticmethod
     def table_ref(database, schema, table_name):
-        dataset_ref = google.cloud.bigquery.DatasetReference(database, schema)
-        return google.cloud.bigquery.TableReference(dataset_ref, table_name)
+        dataset_ref = DatasetReference(database, schema)
+        return TableReference(dataset_ref, table_name)
 
-    def get_bq_table(self, database, schema, identifier):
+    def get_bq_table(self, database, schema, identifier) -> Table:
         """Get a bigquery table for a schema/model."""
         conn = self.get_thread_connection()
+        client: Client = conn.handle
         # backwards compatibility: fill in with defaults if not specified
         database = database or conn.credentials.database
         schema = schema or conn.credentials.schema
-        table_ref = self.table_ref(database, schema, identifier)
-        return conn.handle.get_table(table_ref)
+        return client.get_table(
+            table=self.table_ref(database, schema, identifier),
+            retry=self._retry.create_reopen_with_deadline(conn),
+        )
 
-    def drop_dataset(self, database, schema):
+    def drop_dataset(self, database, schema) -> None:
         conn = self.get_thread_connection()
-        dataset_ref = self.dataset_ref(database, schema)
-        client = conn.handle
-
-        def fn():
-            return client.delete_dataset(dataset_ref, delete_contents=True, not_found_ok=True)
-
-        self._retry_and_handle(msg="drop dataset", conn=conn, fn=fn)
+        client: Client = conn.handle
+        with self.exception_handler("drop dataset"):
+            client.delete_dataset(
+                dataset=self.dataset_ref(database, schema),
+                delete_contents=True,
+                not_found_ok=True,
+                retry=self._retry.create_reopen_with_deadline(conn),
+            )
 
-    def create_dataset(self, database, schema):
+    def create_dataset(self, database, schema) -> Dataset:
         conn = self.get_thread_connection()
-        client = conn.handle
-        dataset_ref = self.dataset_ref(database, schema)
-
-        def fn():
-            return client.create_dataset(dataset_ref, exists_ok=True)
-
-        self._retry_and_handle(msg="create dataset", conn=conn, fn=fn)
+        client: Client = conn.handle
+        with self.exception_handler("create dataset"):
+            return client.create_dataset(
+                dataset=self.dataset_ref(database, schema),
+                exists_ok=True,
+                retry=self._retry.create_reopen_with_deadline(conn),
+            )
 
     def list_dataset(self, database: str):
-        # the database string we get here is potentially quoted. Strip that off
-        # for the API call.
-        database = database.strip("`")
+        # The database string we get here is potentially quoted.
+        # Strip that off for the API call.
         conn = self.get_thread_connection()
-        client = conn.handle
-
-        def query_schemas():
+        client: Client = conn.handle
+        with self.exception_handler("list dataset"):
             # this is similar to how we have to deal with listing tables
-            all_datasets = client.list_datasets(project=database, max_results=10000)
+            all_datasets = client.list_datasets(
+                project=database.strip("`"),
+                max_results=10000,
+                retry=self._retry.create_reopen_with_deadline(conn),
+            )
             return [ds.dataset_id for ds in all_datasets]
 
-        return self._retry_and_handle(msg="list dataset", conn=conn, fn=query_schemas)
-
     def _query_and_results(
         self,
-        client,
+        conn,
         sql,
         job_params,
         job_id,
-        job_creation_timeout=None,
-        job_execution_timeout=None,
         limit: Optional[int] = None,
     ):
+        client: Client = conn.handle
         """Query the client and wait for results."""
         # Cannot reuse job_config if destination is set and ddl is used
-        job_config = google.cloud.bigquery.QueryJobConfig(**job_params)
         query_job = client.query(
-            query=sql, job_config=job_config, job_id=job_id, timeout=job_creation_timeout
+            query=sql,
+            job_config=QueryJobConfig(**job_params),
+            job_id=job_id,  # note, this disables retry since the job_id will have been used
+            timeout=self._retry.create_job_creation_timeout(),
         )
         if (
             query_job.location is not None
@@ -650,37 +584,14 @@ def _query_and_results(
             logger.debug(
                 self._bq_job_link(query_job.location, query_job.project, query_job.job_id)
             )
+
+        timeout = self._retry.create_job_execution_timeout()
         try:
-            iterator = query_job.result(max_results=limit, timeout=job_execution_timeout)
-            return query_job, iterator
+            iterator = query_job.result(max_results=limit, timeout=timeout)
         except TimeoutError:
-            exc = f"Operation did not complete within the designated timeout of {job_execution_timeout} seconds."
+            exc = f"Operation did not complete within the designated timeout of {timeout} seconds."
             raise TimeoutError(exc)
-
-    def _retry_and_handle(self, msg, conn, fn):
-        """retry a function call within the context of exception_handler."""
-
-        def reopen_conn_on_error(error):
-            if isinstance(error, REOPENABLE_ERRORS):
-                logger.warning("Reopening connection after {!r}".format(error))
-                self.close(conn)
-                self.open(conn)
-                return
-
-        with self.exception_handler(msg):
-            return retry.retry_target(
-                target=fn,
-                predicate=_ErrorCounter(self.get_job_retries(conn)).count_error,
-                sleep_generator=self._retry_generator(),
-                deadline=self.get_job_retry_deadline_seconds(conn),
-                on_error=reopen_conn_on_error,
-            )
-
-    def _retry_generator(self):
-        """Generates retry intervals that exponentially back off."""
-        return retry.exponential_sleep_generator(
-            initial=self.DEFAULT_INITIAL_DELAY, maximum=self.DEFAULT_MAXIMUM_DELAY
-        )
+        return query_job, iterator
 
     def _labels_from_query_comment(self, comment: str) -> Dict:
         try:
@@ -693,39 +604,6 @@ def _labels_from_query_comment(self, comment: str) -> Dict:
         }
 
 
-class _ErrorCounter(object):
-    """Counts errors seen up to a threshold then raises the next error."""
-
-    def __init__(self, retries):
-        self.retries = retries
-        self.error_count = 0
-
-    def count_error(self, error):
-        if self.retries == 0:
-            return False  # Don't log
-        self.error_count += 1
-        if _is_retryable(error) and self.error_count <= self.retries:
-            logger.debug(
-                "Retry attempt {} of {} after error: {}".format(
-                    self.error_count, self.retries, repr(error)
-                )
-            )
-            return True
-        else:
-            return False
-
-
-def _is_retryable(error):
-    """Return true for errors that are unlikely to occur again if retried."""
-    if isinstance(error, RETRYABLE_ERRORS):
-        return True
-    elif isinstance(error, google.api_core.exceptions.Forbidden) and any(
-        e["reason"] == "rateLimitExceeded" for e in error.errors
-    ):
-        return True
-    return False
-
-
 _SANITIZE_LABEL_PATTERN = re.compile(r"[^a-z0-9_-]")
 
 _VALIDATE_LABEL_LENGTH_LIMIT = 63
diff --git a/dbt/adapters/bigquery/credentials.py b/dbt/adapters/bigquery/credentials.py
index 32f172dac..94d70a931 100644
--- a/dbt/adapters/bigquery/credentials.py
+++ b/dbt/adapters/bigquery/credentials.py
@@ -1,9 +1,14 @@
+import base64
+import binascii
 from dataclasses import dataclass, field
 from functools import lru_cache
-from typing import Any, Dict, Optional, Tuple
+from typing import Any, Dict, Iterable, Optional, Tuple, Union
 
-import google.auth
+from google.auth import default
 from google.auth.exceptions import DefaultCredentialsError
+from google.auth.impersonated_credentials import Credentials as ImpersonatedCredentials
+from google.oauth2.credentials import Credentials as GoogleCredentials
+from google.oauth2.service_account import Credentials as ServiceAccountCredentials
 from mashumaro import pass_through
 
 from dbt_common.clients.system import run_cmd
@@ -11,6 +16,7 @@
 from dbt_common.exceptions import DbtConfigError, DbtRuntimeError
 from dbt.adapters.contracts.connection import Credentials
 from dbt.adapters.events.logging import AdapterLogger
+from dbt.adapters.exceptions.connection import FailedToConnectError
 
 
 _logger = AdapterLogger("BigQuery")
@@ -21,59 +27,22 @@ class Priority(StrEnum):
     Batch = "batch"
 
 
-class BigQueryConnectionMethod(StrEnum):
-    OAUTH = "oauth"
-    SERVICE_ACCOUNT = "service-account"
-    SERVICE_ACCOUNT_JSON = "service-account-json"
-    OAUTH_SECRETS = "oauth-secrets"
-
-
 @dataclass
 class DataprocBatchConfig(ExtensibleDbtClassMixin):
     def __init__(self, batch_config):
         self.batch_config = batch_config
 
 
-@lru_cache()
-def get_bigquery_defaults(scopes=None) -> Tuple[Any, Optional[str]]:
-    """
-    Returns (credentials, project_id)
-
-    project_id is returned available from the environment; otherwise None
-    """
-    # Cached, because the underlying implementation shells out, taking ~1s
-    try:
-        credentials, _ = google.auth.default(scopes=scopes)
-        return credentials, _
-    except DefaultCredentialsError as e:
-        raise DbtConfigError(f"Failed to authenticate with supplied credentials\nerror:\n{e}")
-
-
-def setup_default_credentials():
-    if _gcloud_installed():
-        run_cmd(".", ["gcloud", "auth", "application-default", "login"])
-    else:
-        msg = """
-        dbt requires the gcloud SDK to be installed to authenticate with BigQuery.
-        Please download and install the SDK, or use a Service Account instead.
-
-        https://cloud.google.com/sdk/
-        """
-        raise DbtRuntimeError(msg)
-
-
-def _gcloud_installed():
-    try:
-        run_cmd(".", ["gcloud", "--version"])
-        return True
-    except OSError as e:
-        _logger.debug(e)
-        return False
+class _BigQueryConnectionMethod(StrEnum):
+    OAUTH = "oauth"
+    OAUTH_SECRETS = "oauth-secrets"
+    SERVICE_ACCOUNT = "service-account"
+    SERVICE_ACCOUNT_JSON = "service-account-json"
 
 
 @dataclass
 class BigQueryCredentials(Credentials):
-    method: BigQueryConnectionMethod = None  # type: ignore
+    method: _BigQueryConnectionMethod = None  # type: ignore
 
     # BigQuery allows an empty database / project, where it defers to the
     # environment for the project
@@ -179,9 +148,122 @@ def __pre_deserialize__(cls, d: Dict[Any, Any]) -> Dict[Any, Any]:
 
         # `database` is an alias of `project` in BigQuery
         if "database" not in d:
-            _, database = get_bigquery_defaults()
+            _, database = _create_bigquery_defaults()
             d["database"] = database
         # `execution_project` default to dataset/project
         if "execution_project" not in d:
             d["execution_project"] = d["database"]
         return d
+
+
+def set_default_credentials() -> None:
+    try:
+        run_cmd(".", ["gcloud", "--version"])
+    except OSError as e:
+        _logger.debug(e)
+        msg = """
+        dbt requires the gcloud SDK to be installed to authenticate with BigQuery.
+        Please download and install the SDK, or use a Service Account instead.
+
+        https://cloud.google.com/sdk/
+        """
+        raise DbtRuntimeError(msg)
+
+    run_cmd(".", ["gcloud", "auth", "application-default", "login"])
+
+
+def create_google_credentials(credentials: BigQueryCredentials) -> GoogleCredentials:
+    if credentials.impersonate_service_account:
+        return _create_impersonated_credentials(credentials)
+    return _create_google_credentials(credentials)
+
+
+def _create_impersonated_credentials(credentials: BigQueryCredentials) -> ImpersonatedCredentials:
+    if credentials.scopes and isinstance(credentials.scopes, Iterable):
+        target_scopes = list(credentials.scopes)
+    else:
+        target_scopes = []
+
+    return ImpersonatedCredentials(
+        source_credentials=_create_google_credentials(credentials),
+        target_principal=credentials.impersonate_service_account,
+        target_scopes=target_scopes,
+    )
+
+
+def _create_google_credentials(credentials: BigQueryCredentials) -> GoogleCredentials:
+
+    if credentials.method == _BigQueryConnectionMethod.OAUTH:
+        creds, _ = _create_bigquery_defaults(scopes=credentials.scopes)
+
+    elif credentials.method == _BigQueryConnectionMethod.SERVICE_ACCOUNT:
+        creds = ServiceAccountCredentials.from_service_account_file(
+            credentials.keyfile, scopes=credentials.scopes
+        )
+
+    elif credentials.method == _BigQueryConnectionMethod.SERVICE_ACCOUNT_JSON:
+        details = credentials.keyfile_json
+        if _is_base64(details):  # type:ignore
+            details = _base64_to_string(details)
+        creds = ServiceAccountCredentials.from_service_account_info(
+            details, scopes=credentials.scopes
+        )
+
+    elif credentials.method == _BigQueryConnectionMethod.OAUTH_SECRETS:
+        creds = GoogleCredentials(
+            token=credentials.token,
+            refresh_token=credentials.refresh_token,
+            client_id=credentials.client_id,
+            client_secret=credentials.client_secret,
+            token_uri=credentials.token_uri,
+            scopes=credentials.scopes,
+        )
+
+    else:
+        raise FailedToConnectError(f"Invalid `method` in profile: '{credentials.method}'")
+
+    return creds
+
+
+@lru_cache()
+def _create_bigquery_defaults(scopes=None) -> Tuple[Any, Optional[str]]:
+    """
+    Returns (credentials, project_id)
+
+    project_id is returned available from the environment; otherwise None
+    """
+    # Cached, because the underlying implementation shells out, taking ~1s
+    try:
+        return default(scopes=scopes)
+    except DefaultCredentialsError as e:
+        raise DbtConfigError(f"Failed to authenticate with supplied credentials\nerror:\n{e}")
+
+
+def _is_base64(s: Union[str, bytes]) -> bool:
+    """
+    Checks if the given string or bytes object is valid Base64 encoded.
+
+    Args:
+        s: The string or bytes object to check.
+
+    Returns:
+        True if the input is valid Base64, False otherwise.
+    """
+
+    if isinstance(s, str):
+        # For strings, ensure they consist only of valid Base64 characters
+        if not s.isascii():
+            return False
+        # Convert to bytes for decoding
+        s = s.encode("ascii")
+
+    try:
+        # Use the 'validate' parameter to enforce strict Base64 decoding rules
+        base64.b64decode(s, validate=True)
+        return True
+    except (TypeError, binascii.Error):
+        return False
+
+
+def _base64_to_string(b):
+    return base64.b64decode(b).decode("utf-8")
diff --git a/dbt/adapters/bigquery/dataproc/__init__.py b/dbt/adapters/bigquery/dataproc/__init__.py
deleted file mode 100644
index e69de29bb..000000000
diff --git a/dbt/adapters/bigquery/dataproc/batch.py b/dbt/adapters/bigquery/dataproc/batch.py
deleted file mode 100644
index 59f40d246..000000000
--- a/dbt/adapters/bigquery/dataproc/batch.py
+++ /dev/null
@@ -1,68 +0,0 @@
-from datetime import datetime
-import time
-from typing import Dict, Union
-
-from google.cloud.dataproc_v1 import (
-    Batch,
-    BatchControllerClient,
-    CreateBatchRequest,
-    GetBatchRequest,
-)
-from google.protobuf.json_format import ParseDict
-
-from dbt.adapters.bigquery.credentials import DataprocBatchConfig
-
-
-_BATCH_RUNNING_STATES = [Batch.State.PENDING, Batch.State.RUNNING]
-DEFAULT_JAR_FILE_URI = "gs://spark-lib/bigquery/spark-bigquery-with-dependencies_2.13-0.34.0.jar"
-
-
-def create_batch_request(
-    batch: Batch, batch_id: str, project: str, region: str
-) -> CreateBatchRequest:
-    return CreateBatchRequest(
-        parent=f"projects/{project}/locations/{region}",
-        batch_id=batch_id,
-        batch=batch,
-    )
-
-
-def poll_batch_job(
-    parent: str, batch_id: str, job_client: BatchControllerClient, timeout: int
-) -> Batch:
-    batch_name = "".join([parent, "/batches/", batch_id])
-    state = Batch.State.PENDING
-    response = None
-    run_time = 0
-    while state in _BATCH_RUNNING_STATES and run_time < timeout:
-        time.sleep(1)
-        response = job_client.get_batch(
-            request=GetBatchRequest(name=batch_name),
-        )
-        run_time = datetime.now().timestamp() - response.create_time.timestamp()
-        state = response.state
-    if not response:
-        raise ValueError("No response from Dataproc")
-    if state != Batch.State.SUCCEEDED:
-        if run_time >= timeout:
-            raise ValueError(
-                f"Operation did not complete within the designated timeout of {timeout} seconds."
-            )
-        else:
-            raise ValueError(response.state_message)
-    return response
-
-
-def update_batch_from_config(config_dict: Union[Dict, DataprocBatchConfig], target: Batch):
-    try:
-        # updates in place
-        ParseDict(config_dict, target._pb)
-    except Exception as e:
-        docurl = (
-            "https://cloud.google.com/dataproc-serverless/docs/reference/rpc/google.cloud.dataproc.v1"
-            "#google.cloud.dataproc.v1.Batch"
-        )
-        raise ValueError(
-            f"Unable to parse dataproc_batch as valid batch specification. See {docurl}. {str(e)}"
-        ) from e
-    return target
diff --git a/dbt/adapters/bigquery/impl.py b/dbt/adapters/bigquery/impl.py
index f6470e7f7..51c457129 100644
--- a/dbt/adapters/bigquery/impl.py
+++ b/dbt/adapters/bigquery/impl.py
@@ -1,9 +1,7 @@
 from dataclasses import dataclass
 from datetime import datetime
-import json
 from multiprocessing.context import SpawnContext
 import threading
-import time
 from typing import (
     Any,
     Dict,
@@ -22,7 +20,7 @@
 import google.auth
 import google.oauth2
 import google.cloud.bigquery
-from google.cloud.bigquery import AccessEntry, SchemaField, Table as BigQueryTable
+from google.cloud.bigquery import AccessEntry, Client, SchemaField, Table as BigQueryTable
 import google.cloud.exceptions
 import pytz
 
@@ -454,22 +452,6 @@ def get_columns_in_select_sql(self, select_sql: str) -> List[BigQueryColumn]:
             logger.debug("get_columns_in_select_sql error: {}".format(e))
             return []
 
-    @classmethod
-    def poll_until_job_completes(cls, job, timeout):
-        retry_count = timeout
-
-        while retry_count > 0 and job.state != "DONE":
-            retry_count -= 1
-            time.sleep(1)
-            job.reload()
-
-        if job.state != "DONE":
-            raise dbt_common.exceptions.DbtRuntimeError("BigQuery Timeout Exceeded")
-
-        elif job.error_result:
-            message = "\n".join(error["message"].strip() for error in job.errors)
-            raise dbt_common.exceptions.DbtRuntimeError(message)
-
     def _bq_table_to_relation(self, bq_table) -> Union[BigQueryRelation, None]:
         if bq_table is None:
             return None
@@ -669,55 +651,50 @@ def alter_table_add_columns(self, relation, columns):
     @available.parse_none
     def load_dataframe(
         self,
-        database,
-        schema,
-        table_name,
+        database: str,
+        schema: str,
+        table_name: str,
         agate_table: "agate.Table",
-        column_override,
-        field_delimiter,
-    ):
-        bq_schema = self._agate_to_schema(agate_table, column_override)
-        conn = self.connections.get_thread_connection()
-        client = conn.handle
-
-        table_ref = self.connections.table_ref(database, schema, table_name)
-
-        load_config = google.cloud.bigquery.LoadJobConfig()
-        load_config.skip_leading_rows = 1
-        load_config.schema = bq_schema
-        load_config.field_delimiter = field_delimiter
-        job_id = self.connections.generate_job_id()
-        with open(agate_table.original_abspath, "rb") as f:  # type: ignore
-            job = client.load_table_from_file(
-                f, table_ref, rewind=True, job_config=load_config, job_id=job_id
-            )
-
-        timeout = self.connections.get_job_execution_timeout_seconds(conn) or 300
-        with self.connections.exception_handler("LOAD TABLE"):
-            self.poll_until_job_completes(job, timeout)
+        column_override: Dict[str, str],
+        field_delimiter: str,
+    ) -> None:
+        connection = self.connections.get_thread_connection()
+        client: Client = connection.handle
+        table_schema = self._agate_to_schema(agate_table, column_override)
+        file_path = agate_table.original_abspath  # type: ignore
+
+        self.connections.write_dataframe_to_table(
+            client,
+            file_path,
+            database,
+            schema,
+            table_name,
+            table_schema,
+            field_delimiter,
+            fallback_timeout=300,
+        )
 
     @available.parse_none
     def upload_file(
-        self, local_file_path: str, database: str, table_schema: str, table_name: str, **kwargs
+        self,
+        local_file_path: str,
+        database: str,
+        table_schema: str,
+        table_name: str,
+        **kwargs,
     ) -> None:
-        conn = self.connections.get_thread_connection()
-        client = conn.handle
-
-        table_ref = self.connections.table_ref(database, table_schema, table_name)
-
-        load_config = google.cloud.bigquery.LoadJobConfig()
-        for k, v in kwargs["kwargs"].items():
-            if k == "schema":
-                setattr(load_config, k, json.loads(v))
-            else:
-                setattr(load_config, k, v)
-
-        with open(local_file_path, "rb") as f:
-            job = client.load_table_from_file(f, table_ref, rewind=True, job_config=load_config)
-
-        timeout = self.connections.get_job_execution_timeout_seconds(conn) or 300
-        with self.connections.exception_handler("LOAD TABLE"):
-            self.poll_until_job_completes(job, timeout)
+        connection = self.connections.get_thread_connection()
+        client: Client = connection.handle
+
+        self.connections.write_file_to_table(
+            client,
+            local_file_path,
+            database,
+            table_schema,
+            table_name,
+            fallback_timeout=300,
+            **kwargs,
+        )
 
     @classmethod
     def _catalog_filter_table(
@@ -753,7 +730,7 @@ def calculate_freshness_from_metadata(
         macro_resolver: Optional[MacroResolverProtocol] = None,
     ) -> Tuple[Optional[AdapterResponse], FreshnessResponse]:
         conn = self.connections.get_thread_connection()
-        client: google.cloud.bigquery.Client = conn.handle
+        client: Client = conn.handle
 
         table_ref = self.get_table_ref_from_relation(source)
         table = client.get_table(table_ref)
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt/adapters/bigquery/python_submissions.py
index 93c82ca92..cd7f7d86f 100644
--- a/dbt/adapters/bigquery/python_submissions.py
+++ b/dbt/adapters/bigquery/python_submissions.py
@@ -1,187 +1,165 @@
-import uuid
 from typing import Dict, Union
+import uuid
 
-from google.api_core import retry
-from google.api_core.client_options import ClientOptions
-from google.api_core.future.polling import POLLING_PREDICATE
-from google.cloud import storage, dataproc_v1
-from google.cloud.dataproc_v1.types.batches import Batch
+from google.cloud.dataproc_v1 import Batch, CreateBatchRequest, Job, RuntimeConfig
 
 from dbt.adapters.base import PythonJobHelper
 from dbt.adapters.events.logging import AdapterLogger
+from google.protobuf.json_format import ParseDict
 
-from dbt.adapters.bigquery.connections import BigQueryConnectionManager
-from dbt.adapters.bigquery.credentials import BigQueryCredentials
-from dbt.adapters.bigquery.dataproc.batch import (
-    DEFAULT_JAR_FILE_URI,
-    create_batch_request,
-    poll_batch_job,
-    update_batch_from_config,
+from dbt.adapters.bigquery.credentials import BigQueryCredentials, DataprocBatchConfig
+from dbt.adapters.bigquery.clients import (
+    create_dataproc_batch_controller_client,
+    create_dataproc_job_controller_client,
+    create_gcs_client,
 )
+from dbt.adapters.bigquery.retry import RetryFactory
+
 
-OPERATION_RETRY_TIME = 10
-logger = AdapterLogger("BigQuery")
+_logger = AdapterLogger("BigQuery")
 
 
-class BaseDataProcHelper(PythonJobHelper):
-    def __init__(self, parsed_model: Dict, credential: BigQueryCredentials) -> None:
-        """_summary_
+_DEFAULT_JAR_FILE_URI = "gs://spark-lib/bigquery/spark-bigquery-with-dependencies_2.13-0.34.0.jar"
 
-        Args:
-            credential (_type_): _description_
-        """
+
+class _BaseDataProcHelper(PythonJobHelper):
+    def __init__(self, parsed_model: Dict, credentials: BigQueryCredentials) -> None:
         # validate all additional stuff for python is set
-        schema = parsed_model["schema"]
-        identifier = parsed_model["alias"]
-        self.parsed_model = parsed_model
-        python_required_configs = [
-            "dataproc_region",
-            "gcs_bucket",
-        ]
-        for required_config in python_required_configs:
-            if not getattr(credential, required_config):
+        for required_config in ["dataproc_region", "gcs_bucket"]:
+            if not getattr(credentials, required_config):
                 raise ValueError(
                     f"Need to supply {required_config} in profile to submit python job"
                 )
-        self.model_file_name = f"{schema}/{identifier}.py"
-        self.credential = credential
-        self.GoogleCredentials = BigQueryConnectionManager.get_credentials(credential)
-        self.storage_client = storage.Client(
-            project=self.credential.execution_project, credentials=self.GoogleCredentials
-        )
-        self.gcs_location = "gs://{}/{}".format(self.credential.gcs_bucket, self.model_file_name)
+
+        self._storage_client = create_gcs_client(credentials)
+        self._project = credentials.execution_project
+        self._region = credentials.dataproc_region
+
+        schema = parsed_model["schema"]
+        identifier = parsed_model["alias"]
+        self._model_file_name = f"{schema}/{identifier}.py"
+        self._gcs_bucket = credentials.gcs_bucket
+        self._gcs_path = f"gs://{credentials.gcs_bucket}/{self._model_file_name}"
 
         # set retry policy, default to timeout after 24 hours
-        self.timeout = self.parsed_model["config"].get(
-            "timeout", self.credential.job_execution_timeout_seconds or 60 * 60 * 24
-        )
-        self.result_polling_policy = retry.Retry(
-            predicate=POLLING_PREDICATE, maximum=10.0, timeout=self.timeout
-        )
-        self.client_options = ClientOptions(
-            api_endpoint="{}-dataproc.googleapis.com:443".format(self.credential.dataproc_region)
+        retry = RetryFactory(credentials)
+        self._polling_retry = retry.create_polling(
+            model_timeout=parsed_model["config"].get("timeout")
         )
-        self.job_client = self._get_job_client()
 
-    def _upload_to_gcs(self, filename: str, compiled_code: str) -> None:
-        bucket = self.storage_client.get_bucket(self.credential.gcs_bucket)
-        blob = bucket.blob(filename)
+    def _write_to_gcs(self, compiled_code: str) -> None:
+        bucket = self._storage_client.get_bucket(self._gcs_bucket)
+        blob = bucket.blob(self._model_file_name)
         blob.upload_from_string(compiled_code)
 
-    def submit(self, compiled_code: str) -> dataproc_v1.types.jobs.Job:
-        # upload python file to GCS
-        self._upload_to_gcs(self.model_file_name, compiled_code)
-        # submit dataproc job
-        return self._submit_dataproc_job()
-
-    def _get_job_client(
-        self,
-    ) -> Union[dataproc_v1.JobControllerClient, dataproc_v1.BatchControllerClient]:
-        raise NotImplementedError("_get_job_client not implemented")
-
-    def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
-        raise NotImplementedError("_submit_dataproc_job not implemented")
 
+class ClusterDataprocHelper(_BaseDataProcHelper):
+    def __init__(self, parsed_model: Dict, credentials: BigQueryCredentials) -> None:
+        super().__init__(parsed_model, credentials)
+        self._job_controller_client = create_dataproc_job_controller_client(credentials)
+        self._cluster_name = parsed_model["config"].get(
+            "dataproc_cluster_name", credentials.dataproc_cluster_name
+        )
 
-class ClusterDataprocHelper(BaseDataProcHelper):
-    def _get_job_client(self) -> dataproc_v1.JobControllerClient:
-        if not self._get_cluster_name():
+        if not self._cluster_name:
             raise ValueError(
                 "Need to supply dataproc_cluster_name in profile or config to submit python job with cluster submission method"
             )
-        return dataproc_v1.JobControllerClient(
-            client_options=self.client_options, credentials=self.GoogleCredentials
-        )
 
-    def _get_cluster_name(self) -> str:
-        return self.parsed_model["config"].get(
-            "dataproc_cluster_name", self.credential.dataproc_cluster_name
-        )
+    def submit(self, compiled_code: str) -> Job:
+        _logger.debug(f"Submitting cluster job to: {self._cluster_name}")
 
-    def _submit_dataproc_job(self) -> dataproc_v1.types.jobs.Job:
-        job = {
-            "placement": {"cluster_name": self._get_cluster_name()},
-            "pyspark_job": {
-                "main_python_file_uri": self.gcs_location,
+        self._write_to_gcs(compiled_code)
+
+        request = {
+            "project_id": self._project,
+            "region": self._region,
+            "job": {
+                "placement": {"cluster_name": self._cluster_name},
+                "pyspark_job": {
+                    "main_python_file_uri": self._gcs_path,
+                },
             },
         }
-        operation = self.job_client.submit_job_as_operation(
-            request={
-                "project_id": self.credential.execution_project,
-                "region": self.credential.dataproc_region,
-                "job": job,
-            }
-        )
-        # check if job failed
-        response = operation.result(polling=self.result_polling_policy)
+
+        # submit the job
+        operation = self._job_controller_client.submit_job_as_operation(request)
+
+        # wait for the job to complete
+        response: Job = operation.result(polling=self._polling_retry)
+
         if response.status.state == 6:
             raise ValueError(response.status.details)
+
         return response
 
 
-class ServerlessDataProcHelper(BaseDataProcHelper):
-    def _get_job_client(self) -> dataproc_v1.BatchControllerClient:
-        return dataproc_v1.BatchControllerClient(
-            client_options=self.client_options, credentials=self.GoogleCredentials
-        )
+class ServerlessDataProcHelper(_BaseDataProcHelper):
+    def __init__(self, parsed_model: Dict, credentials: BigQueryCredentials) -> None:
+        super().__init__(parsed_model, credentials)
+        self._batch_controller_client = create_dataproc_batch_controller_client(credentials)
+        self._batch_id = parsed_model["config"].get("batch_id", str(uuid.uuid4()))
+        self._jar_file_uri = parsed_model["config"].get("jar_file_uri", _DEFAULT_JAR_FILE_URI)
+        self._dataproc_batch = credentials.dataproc_batch
 
-    def _get_batch_id(self) -> str:
-        model = self.parsed_model
-        default_batch_id = str(uuid.uuid4())
-        return model["config"].get("batch_id", default_batch_id)
-
-    def _submit_dataproc_job(self) -> Batch:
-        batch_id = self._get_batch_id()
-        logger.info(f"Submitting batch job with id: {batch_id}")
-        request = create_batch_request(
-            batch=self._configure_batch(),
-            batch_id=batch_id,
-            region=self.credential.dataproc_region,  # type: ignore
-            project=self.credential.execution_project,  # type: ignore
-        )
-        # make the request
-        self.job_client.create_batch(request=request)
-        return poll_batch_job(
-            parent=request.parent,
-            batch_id=batch_id,
-            job_client=self.job_client,
-            timeout=self.timeout,
+    def submit(self, compiled_code: str) -> Batch:
+        _logger.debug(f"Submitting batch job with id: {self._batch_id}")
+
+        self._write_to_gcs(compiled_code)
+
+        request = CreateBatchRequest(
+            parent=f"projects/{self._project}/locations/{self._region}",
+            batch=self._create_batch(),
+            batch_id=self._batch_id,
         )
-        # there might be useful results here that we can parse and return
-        # Dataproc job output is saved to the Cloud Storage bucket
-        # allocated to the job. Use regex to obtain the bucket and blob info.
-        # matches = re.match("gs://(.*?)/(.*)", response.driver_output_resource_uri)
-        # output = (
-        #     self.storage_client
-        #     .get_bucket(matches.group(1))
-        #     .blob(f"{matches.group(2)}.000000000")
-        #     .download_as_string()
-        # )
-
-    def _configure_batch(self):
+
+        # submit the batch
+        operation = self._batch_controller_client.create_batch(request)
+
+        # wait for the batch to complete
+        response: Batch = operation.result(polling=self._polling_retry)
+
+        return response
+
+    def _create_batch(self) -> Batch:
         # create the Dataproc Serverless job config
         # need to pin dataproc version to 1.1 as it now defaults to 2.0
         # https://cloud.google.com/dataproc-serverless/docs/concepts/properties
         # https://cloud.google.com/dataproc-serverless/docs/reference/rest/v1/projects.locations.batches#runtimeconfig
-        batch = dataproc_v1.Batch(
+        batch = Batch(
             {
-                "runtime_config": dataproc_v1.RuntimeConfig(
+                "runtime_config": RuntimeConfig(
                     version="1.1",
                     properties={
                         "spark.executor.instances": "2",
                     },
-                )
+                ),
+                "pyspark_batch": {
+                    "main_python_file_uri": self._gcs_path,
+                    "jar_file_uris": [self._jar_file_uri],
+                },
             }
         )
-        # Apply defaults
-        batch.pyspark_batch.main_python_file_uri = self.gcs_location
-        jar_file_uri = self.parsed_model["config"].get(
-            "jar_file_uri",
-            DEFAULT_JAR_FILE_URI,
-        )
-        batch.pyspark_batch.jar_file_uris = [jar_file_uri]
 
         # Apply configuration from dataproc_batch key, possibly overriding defaults.
-        if self.credential.dataproc_batch:
-            batch = update_batch_from_config(self.credential.dataproc_batch, batch)
+        if self._dataproc_batch:
+            batch = _update_batch_from_config(self._dataproc_batch, batch)
+
         return batch
+
+
+def _update_batch_from_config(
+    config_dict: Union[Dict, DataprocBatchConfig], target: Batch
+) -> Batch:
+    try:
+        # updates in place
+        ParseDict(config_dict, target._pb)
+    except Exception as e:
+        docurl = (
+            "https://cloud.google.com/dataproc-serverless/docs/reference/rpc/google.cloud.dataproc.v1"
+            "#google.cloud.dataproc.v1.Batch"
+        )
+        raise ValueError(
+            f"Unable to parse dataproc_batch as valid batch specification. See {docurl}. {str(e)}"
+        ) from e
+    return target
diff --git a/dbt/adapters/bigquery/retry.py b/dbt/adapters/bigquery/retry.py
new file mode 100644
index 000000000..391c00e46
--- /dev/null
+++ b/dbt/adapters/bigquery/retry.py
@@ -0,0 +1,128 @@
+from typing import Callable, Optional
+
+from google.api_core.exceptions import Forbidden
+from google.api_core.future.polling import DEFAULT_POLLING
+from google.api_core.retry import Retry
+from google.cloud.bigquery.retry import DEFAULT_RETRY
+from google.cloud.exceptions import BadGateway, BadRequest, ServerError
+from requests.exceptions import ConnectionError
+
+from dbt.adapters.contracts.connection import Connection, ConnectionState
+from dbt.adapters.events.logging import AdapterLogger
+from dbt.adapters.exceptions.connection import FailedToConnectError
+
+from dbt.adapters.bigquery.clients import create_bigquery_client
+from dbt.adapters.bigquery.credentials import BigQueryCredentials
+
+
+_logger = AdapterLogger("BigQuery")
+
+
+_SECOND = 1.0
+_MINUTE = 60 * _SECOND
+_HOUR = 60 * _MINUTE
+_DAY = 24 * _HOUR
+_DEFAULT_INITIAL_DELAY = _SECOND
+_DEFAULT_MAXIMUM_DELAY = 3 * _SECOND
+_DEFAULT_POLLING_MAXIMUM_DELAY = 10 * _SECOND
+
+
+class RetryFactory:
+
+    def __init__(self, credentials: BigQueryCredentials) -> None:
+        self._retries = credentials.job_retries or 0
+        self._job_creation_timeout = credentials.job_creation_timeout_seconds
+        self._job_execution_timeout = credentials.job_execution_timeout_seconds
+        self._job_deadline = credentials.job_retry_deadline_seconds
+
+    def create_job_creation_timeout(self, fallback: float = _MINUTE) -> float:
+        return (
+            self._job_creation_timeout or fallback
+        )  # keep _MINUTE here so it's not overridden by passing fallback=None
+
+    def create_job_execution_timeout(self, fallback: float = _DAY) -> float:
+        return (
+            self._job_execution_timeout or fallback
+        )  # keep _DAY here so it's not overridden by passing fallback=None
+
+    def create_retry(self, fallback: Optional[float] = None) -> Retry:
+        return DEFAULT_RETRY.with_timeout(self._job_execution_timeout or fallback or _DAY)
+
+    def create_polling(self, model_timeout: Optional[float] = None) -> Retry:
+        return DEFAULT_POLLING.with_timeout(model_timeout or self._job_execution_timeout or _DAY)
+
+    def create_reopen_with_deadline(self, connection: Connection) -> Retry:
+        """
+        This strategy mimics what was accomplished with _retry_and_handle
+        """
+        return Retry(
+            predicate=_DeferredException(self._retries),
+            initial=_DEFAULT_INITIAL_DELAY,
+            maximum=_DEFAULT_MAXIMUM_DELAY,
+            deadline=self._job_deadline,
+            on_error=_create_reopen_on_error(connection),
+        )
+
+
+class _DeferredException:
+    """
+    Count ALL errors, not just retryable errors, up to a threshold.
+    Raise the next error, regardless of whether it is retryable.
+    """
+
+    def __init__(self, retries: int) -> None:
+        self._retries: int = retries
+        self._error_count = 0
+
+    def __call__(self, error: Exception) -> bool:
+        # exit immediately if the user does not want retries
+        if self._retries == 0:
+            return False
+
+        # count all errors
+        self._error_count += 1
+
+        # if the error is retryable, and we haven't breached the threshold, log and continue
+        if _is_retryable(error) and self._error_count <= self._retries:
+            _logger.debug(
+                f"Retry attempt {self._error_count} of {self._retries} after error: {repr(error)}"
+            )
+            return True
+
+        # otherwise raise
+        return False
+
+
+def _create_reopen_on_error(connection: Connection) -> Callable[[Exception], None]:
+
+    def on_error(error: Exception):
+        if isinstance(error, (ConnectionResetError, ConnectionError)):
+            _logger.warning("Reopening connection after {!r}".format(error))
+            connection.handle.close()
+
+            try:
+                connection.handle = create_bigquery_client(connection.credentials)
+                connection.state = ConnectionState.OPEN
+
+            except Exception as e:
+                _logger.debug(
+                    f"""Got an error when attempting to create a bigquery " "client: '{e}'"""
+                )
+                connection.handle = None
+                connection.state = ConnectionState.FAIL
+                raise FailedToConnectError(str(e))
+
+    return on_error
+
+
+def _is_retryable(error: Exception) -> bool:
+    """Return true for errors that are unlikely to occur again if retried."""
+    if isinstance(
+        error, (BadGateway, BadRequest, ConnectionError, ConnectionResetError, ServerError)
+    ):
+        return True
+    elif isinstance(error, Forbidden) and any(
+        e["reason"] == "rateLimitExceeded" for e in error.errors
+    ):
+        return True
+    return False
diff --git a/dbt/adapters/bigquery/utility.py b/dbt/adapters/bigquery/utility.py
index 557986b38..5914280a3 100644
--- a/dbt/adapters/bigquery/utility.py
+++ b/dbt/adapters/bigquery/utility.py
@@ -1,7 +1,5 @@
-import base64
-import binascii
 import json
-from typing import Any, Optional, Union
+from typing import Any, Optional
 
 import dbt_common.exceptions
 
@@ -45,39 +43,3 @@ def sql_escape(string):
     if not isinstance(string, str):
         raise dbt_common.exceptions.CompilationError(f"cannot escape a non-string: {string}")
     return json.dumps(string)[1:-1]
-
-
-def is_base64(s: Union[str, bytes]) -> bool:
-    """
-    Checks if the given string or bytes object is valid Base64 encoded.
-
-    Args:
-        s: The string or bytes object to check.
-
-    Returns:
-        True if the input is valid Base64, False otherwise.
-    """
-
-    if isinstance(s, str):
-        # For strings, ensure they consist only of valid Base64 characters
-        if not s.isascii():
-            return False
-        # Convert to bytes for decoding
-        s = s.encode("ascii")
-
-    try:
-        # Use the 'validate' parameter to enforce strict Base64 decoding rules
-        base64.b64decode(s, validate=True)
-        return True
-    except TypeError:
-        return False
-    except binascii.Error:  # Catch specific errors from the base64 module
-        return False
-
-
-def base64_to_string(b):
-    return base64.b64decode(b).decode("utf-8")
-
-
-def string_to_base64(s):
-    return base64.b64encode(s.encode("utf-8"))
diff --git a/tests/conftest.py b/tests/conftest.py
index 6dc9e6443..33f7f9d17 100644
--- a/tests/conftest.py
+++ b/tests/conftest.py
@@ -1,9 +1,9 @@
 import pytest
 import os
 import json
-from dbt.adapters.bigquery.utility import is_base64, base64_to_string
+from dbt.adapters.bigquery.credentials import _is_base64, _base64_to_string
 
-# Import the fuctional fixtures as a plugin
+# Import the functional fixtures as a plugin
 # Note: fixtures with session scope need to be local
 
 pytest_plugins = ["dbt.tests.fixtures.project"]
@@ -39,8 +39,8 @@ def oauth_target():
 
 def service_account_target():
     credentials_json_str = os.getenv("BIGQUERY_TEST_SERVICE_ACCOUNT_JSON").replace("'", "")
-    if is_base64(credentials_json_str):
-        credentials_json_str = base64_to_string(credentials_json_str)
+    if _is_base64(credentials_json_str):
+        credentials_json_str = _base64_to_string(credentials_json_str)
     credentials = json.loads(credentials_json_str)
     project_id = credentials.get("project_id")
     return {
diff --git a/tests/functional/adapter/test_json_keyfile.py b/tests/functional/adapter/test_json_keyfile.py
index 91e41a3f1..a5caaebdf 100644
--- a/tests/functional/adapter/test_json_keyfile.py
+++ b/tests/functional/adapter/test_json_keyfile.py
@@ -1,6 +1,11 @@
+import base64
 import json
 import pytest
-from dbt.adapters.bigquery.utility import string_to_base64, is_base64
+from dbt.adapters.bigquery.credentials import _is_base64
+
+
+def string_to_base64(s):
+    return base64.b64encode(s.encode("utf-8"))
 
 
 @pytest.fixture
@@ -53,7 +58,7 @@ def test_valid_base64_strings(example_json_keyfile_b64):
     ]
 
     for s in valid_strings:
-        assert is_base64(s) is True
+        assert _is_base64(s) is True
 
 
 def test_valid_base64_bytes(example_json_keyfile_b64):
@@ -65,7 +70,7 @@ def test_valid_base64_bytes(example_json_keyfile_b64):
         example_json_keyfile_b64,
     ]
     for s in valid_bytes:
-        assert is_base64(s) is True
+        assert _is_base64(s) is True
 
 
 def test_invalid_base64(example_json_keyfile):
@@ -79,4 +84,4 @@ def test_invalid_base64(example_json_keyfile):
         example_json_keyfile,
     ]
     for s in invalid_inputs:
-        assert is_base64(s) is False
+        assert _is_base64(s) is False
diff --git a/tests/unit/test_bigquery_adapter.py b/tests/unit/test_bigquery_adapter.py
index ca3bfc24c..e57db9a62 100644
--- a/tests/unit/test_bigquery_adapter.py
+++ b/tests/unit/test_bigquery_adapter.py
@@ -203,7 +203,7 @@ def get_adapter(self, target) -> BigQueryAdapter:
 
 class TestBigQueryAdapterAcquire(BaseTestBigQueryAdapter):
     @patch(
-        "dbt.adapters.bigquery.credentials.get_bigquery_defaults",
+        "dbt.adapters.bigquery.credentials._create_bigquery_defaults",
         return_value=("credentials", "project_id"),
     )
     @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
@@ -244,10 +244,12 @@ def test_acquire_connection_oauth_validations(self, mock_open_connection):
         mock_open_connection.assert_called_once()
 
     @patch(
-        "dbt.adapters.bigquery.credentials.get_bigquery_defaults",
+        "dbt.adapters.bigquery.credentials._create_bigquery_defaults",
         return_value=("credentials", "project_id"),
     )
-    @patch("dbt.adapters.bigquery.BigQueryConnectionManager.open", return_value=_bq_conn())
+    @patch(
+        "dbt.adapters.bigquery.connections.BigQueryConnectionManager.open", return_value=_bq_conn()
+    )
     def test_acquire_connection_dataproc_serverless(
         self, mock_open_connection, mock_get_bigquery_defaults
     ):
@@ -386,21 +388,20 @@ def test_cancel_open_connections_single(self):
         adapter.connections.thread_connections.update({key: master, 1: model})
         self.assertEqual(len(list(adapter.cancel_open_connections())), 1)
 
-    @patch("dbt.adapters.bigquery.impl.google.api_core.client_options.ClientOptions")
-    @patch("dbt.adapters.bigquery.impl.google.auth.default")
-    @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
-    def test_location_user_agent(self, mock_bq, mock_auth_default, MockClientOptions):
+    @patch("dbt.adapters.bigquery.clients.ClientOptions")
+    @patch("dbt.adapters.bigquery.credentials.default")
+    @patch("dbt.adapters.bigquery.clients.BigQueryClient")
+    def test_location_user_agent(self, MockClient, mock_auth_default, MockClientOptions):
         creds = MagicMock()
         mock_auth_default.return_value = (creds, MagicMock())
         adapter = self.get_adapter("loc")
 
         connection = adapter.acquire_connection("dummy")
-        mock_client = mock_bq.Client
         mock_client_options = MockClientOptions.return_value
 
-        mock_client.assert_not_called()
+        MockClient.assert_not_called()
         connection.handle
-        mock_client.assert_called_once_with(
+        MockClient.assert_called_once_with(
             "dbt-unit-000000",
             creds,
             location="Luna Station",
diff --git a/tests/unit/test_bigquery_connection_manager.py b/tests/unit/test_bigquery_connection_manager.py
index 1c14100f6..d4c95792e 100644
--- a/tests/unit/test_bigquery_connection_manager.py
+++ b/tests/unit/test_bigquery_connection_manager.py
@@ -1,81 +1,59 @@
 import json
 import unittest
-from contextlib import contextmanager
 from requests.exceptions import ConnectionError
 from unittest.mock import patch, MagicMock, Mock, ANY
 
 import dbt.adapters
+import google.cloud.bigquery
 
 from dbt.adapters.bigquery import BigQueryCredentials
 from dbt.adapters.bigquery import BigQueryRelation
 from dbt.adapters.bigquery.connections import BigQueryConnectionManager
+from dbt.adapters.bigquery.retry import RetryFactory
 
 
 class TestBigQueryConnectionManager(unittest.TestCase):
     def setUp(self):
-        credentials = Mock(BigQueryCredentials)
-        profile = Mock(query_comment=None, credentials=credentials)
-        self.connections = BigQueryConnectionManager(profile=profile, mp_context=Mock())
+        self.credentials = Mock(BigQueryCredentials)
+        self.credentials.method = "oauth"
+        self.credentials.job_retries = 1
+        self.credentials.job_retry_deadline_seconds = 1
+        self.credentials.scopes = tuple()
 
-        self.mock_client = Mock(dbt.adapters.bigquery.impl.google.cloud.bigquery.Client)
-        self.mock_connection = MagicMock()
+        self.mock_client = Mock(google.cloud.bigquery.Client)
 
+        self.mock_connection = MagicMock()
         self.mock_connection.handle = self.mock_client
+        self.mock_connection.credentials = self.credentials
 
+        self.connections = BigQueryConnectionManager(
+            profile=Mock(credentials=self.credentials, query_comment=None),
+            mp_context=Mock(),
+        )
         self.connections.get_thread_connection = lambda: self.mock_connection
-        self.connections.get_job_retry_deadline_seconds = lambda x: None
-        self.connections.get_job_retries = lambda x: 1
-
-    @patch("dbt.adapters.bigquery.connections._is_retryable", return_value=True)
-    def test_retry_and_handle(self, is_retryable):
-        self.connections.DEFAULT_MAXIMUM_DELAY = 2.0
-
-        @contextmanager
-        def dummy_handler(msg):
-            yield
-
-        self.connections.exception_handler = dummy_handler
-
-        class DummyException(Exception):
-            """Count how many times this exception is raised"""
-
-            count = 0
 
-            def __init__(self):
-                DummyException.count += 1
+    @patch(
+        "dbt.adapters.bigquery.retry.create_bigquery_client",
+        return_value=Mock(google.cloud.bigquery.Client),
+    )
+    def test_retry_connection_reset(self, mock_client_factory):
+        new_mock_client = mock_client_factory.return_value
 
-        def raiseDummyException():
-            raise DummyException()
+        @self.connections._retry.create_reopen_with_deadline(self.mock_connection)
+        def generate_connection_reset_error():
+            raise ConnectionResetError
 
-        with self.assertRaises(DummyException):
-            self.connections._retry_and_handle(
-                "some sql", Mock(credentials=Mock(retries=8)), raiseDummyException
-            )
-            self.assertEqual(DummyException.count, 9)
+        assert self.mock_connection.handle is self.mock_client
 
-    @patch("dbt.adapters.bigquery.connections._is_retryable", return_value=True)
-    def test_retry_connection_reset(self, is_retryable):
-        self.connections.open = MagicMock()
-        self.connections.close = MagicMock()
-        self.connections.DEFAULT_MAXIMUM_DELAY = 2.0
-
-        @contextmanager
-        def dummy_handler(msg):
-            yield
-
-        self.connections.exception_handler = dummy_handler
-
-        def raiseConnectionResetError():
-            raise ConnectionResetError("Connection broke")
-
-        mock_conn = Mock(credentials=Mock(retries=1))
         with self.assertRaises(ConnectionResetError):
-            self.connections._retry_and_handle("some sql", mock_conn, raiseConnectionResetError)
-        self.connections.close.assert_called_once_with(mock_conn)
-        self.connections.open.assert_called_once_with(mock_conn)
+            # this will always raise the error, we just want to test that the connection was reopening in between
+            generate_connection_reset_error()
+
+        assert self.mock_connection.handle is new_mock_client
+        assert new_mock_client is not self.mock_client
 
     def test_is_retryable(self):
-        _is_retryable = dbt.adapters.bigquery.connections._is_retryable
+        _is_retryable = dbt.adapters.bigquery.retry._is_retryable
         exceptions = dbt.adapters.bigquery.impl.google.cloud.exceptions
         internal_server_error = exceptions.InternalServerError("code broke")
         bad_request_error = exceptions.BadRequest("code broke")
@@ -104,29 +82,30 @@ def test_drop_dataset(self):
         self.mock_client.delete_table.assert_not_called()
         self.mock_client.delete_dataset.assert_called_once()
 
-    @patch("dbt.adapters.bigquery.impl.google.cloud.bigquery")
-    def test_query_and_results(self, mock_bq):
+    @patch("dbt.adapters.bigquery.connections.QueryJobConfig")
+    def test_query_and_results(self, MockQueryJobConfig):
         self.connections._query_and_results(
-            self.mock_client,
+            self.mock_connection,
             "sql",
-            {"job_param_1": "blah"},
+            {"dry_run": True},
             job_id=1,
-            job_creation_timeout=15,
-            job_execution_timeout=100,
         )
 
-        mock_bq.QueryJobConfig.assert_called_once()
+        MockQueryJobConfig.assert_called_once()
         self.mock_client.query.assert_called_once_with(
-            query="sql", job_config=mock_bq.QueryJobConfig(), job_id=1, timeout=15
+            query="sql",
+            job_config=MockQueryJobConfig(),
+            job_id=1,
+            timeout=self.credentials.job_creation_timeout_seconds,
         )
 
     def test_copy_bq_table_appends(self):
         self._copy_table(write_disposition=dbt.adapters.bigquery.impl.WRITE_APPEND)
-        args, kwargs = self.mock_client.copy_table.call_args
         self.mock_client.copy_table.assert_called_once_with(
             [self._table_ref("project", "dataset", "table1")],
             self._table_ref("project", "dataset", "table2"),
             job_config=ANY,
+            retry=ANY,
         )
         args, kwargs = self.mock_client.copy_table.call_args
         self.assertEqual(
@@ -140,6 +119,7 @@ def test_copy_bq_table_truncates(self):
             [self._table_ref("project", "dataset", "table1")],
             self._table_ref("project", "dataset", "table2"),
             job_config=ANY,
+            retry=ANY,
         )
         args, kwargs = self.mock_client.copy_table.call_args
         self.assertEqual(
@@ -161,7 +141,7 @@ def test_list_dataset_correctly_calls_lists_datasets(self):
         self.mock_client.list_datasets = mock_list_dataset
         result = self.connections.list_dataset("project")
         self.mock_client.list_datasets.assert_called_once_with(
-            project="project", max_results=10000
+            project="project", max_results=10000, retry=ANY
         )
         assert result == ["d1"]
 
diff --git a/tests/unit/test_configure_dataproc_batch.py b/tests/unit/test_configure_dataproc_batch.py
index f56aee129..6e5757589 100644
--- a/tests/unit/test_configure_dataproc_batch.py
+++ b/tests/unit/test_configure_dataproc_batch.py
@@ -1,6 +1,6 @@
 from unittest.mock import patch
 
-from dbt.adapters.bigquery.dataproc.batch import update_batch_from_config
+from dbt.adapters.bigquery.python_submissions import _update_batch_from_config
 from google.cloud import dataproc_v1
 
 from .test_bigquery_adapter import BaseTestBigQueryAdapter
@@ -12,7 +12,7 @@
 # parsed credentials
 class TestConfigureDataprocBatch(BaseTestBigQueryAdapter):
     @patch(
-        "dbt.adapters.bigquery.credentials.get_bigquery_defaults",
+        "dbt.adapters.bigquery.credentials._create_bigquery_defaults",
         return_value=("credentials", "project_id"),
     )
     def test_update_dataproc_serverless_batch(self, mock_get_bigquery_defaults):
@@ -39,7 +39,7 @@ def test_update_dataproc_serverless_batch(self, mock_get_bigquery_defaults):
 
         batch = dataproc_v1.Batch()
 
-        batch = update_batch_from_config(raw_batch_config, batch)
+        batch = _update_batch_from_config(raw_batch_config, batch)
 
         def to_str_values(d):
             """google's protobuf types expose maps as dict[str, str]"""
@@ -64,7 +64,7 @@ def to_str_values(d):
         )
 
     @patch(
-        "dbt.adapters.bigquery.credentials.get_bigquery_defaults",
+        "dbt.adapters.bigquery.credentials._create_bigquery_defaults",
         return_value=("credentials", "project_id"),
     )
     def test_default_dataproc_serverless_batch(self, mock_get_bigquery_defaults):

From 62695c9322f71339fd0cc839761643244d4d7ad1 Mon Sep 17 00:00:00 2001
From: FishtownBuildBot <77737458+FishtownBuildBot@users.noreply.github.com>
Date: Mon, 2 Dec 2024 13:58:59 -0500
Subject: [PATCH 852/860] Cleanup main after cutting new 1.9.latest branch
 (#1419)

* Clean up changelog on main

* Bumping version to 1.10.0a1

* Code quality cleanup
---
 .bumpversion.cfg                              |  2 +-
 .changes/1.9.0-b1.md                          | 44 ------------------
 .../1.9.0/Dependencies-20231211-001048.yaml   |  6 ---
 .../1.9.0/Dependencies-20231220-002130.yaml   |  6 ---
 .../1.9.0/Dependencies-20231222-002351.yaml   |  6 ---
 .../1.9.0/Dependencies-20240105-004800.yaml   |  6 ---
 .../1.9.0/Dependencies-20240429-005158.yaml   |  6 ---
 .../1.9.0/Dependencies-20240429-005159.yaml   |  6 ---
 .../1.9.0/Dependencies-20240520-230208.yaml   |  6 ---
 .../1.9.0/Dependencies-20240718-005755.yaml   |  6 ---
 .../1.9.0/Dependencies-20240718-005756.yaml   |  6 ---
 .../1.9.0/Dependencies-20240718-005757.yaml   |  6 ---
 .../1.9.0/Dependencies-20240719-003740.yaml   |  6 ---
 .changes/1.9.0/Features-20240426-105319.yaml  |  7 ---
 .changes/1.9.0/Features-20240430-185650.yaml  |  6 ---
 .changes/1.9.0/Features-20240501-151902.yaml  |  6 ---
 .changes/1.9.0/Features-20240516-125735.yaml  |  6 ---
 .changes/1.9.0/Features-20240730-135911.yaml  |  6 ---
 .changes/1.9.0/Features-20240925-232238.yaml  |  6 ---
 .changes/1.9.0/Fixes-20240120-180818.yaml     |  6 ---
 .changes/1.9.0/Fixes-20240201-145323.yaml     |  6 ---
 .changes/1.9.0/Fixes-20240226-233024.yaml     |  6 ---
 .changes/1.9.0/Fixes-20240426-105224.yaml     |  7 ---
 .changes/1.9.0/Fixes-20241001-193207.yaml     |  7 ---
 .../1.9.0/Under the Hood-20240331-101418.yaml |  6 ---
 .../1.9.0/Under the Hood-20240718-193206.yaml |  6 ---
 .../Breaking Changes-20241016-185117.yaml     |  6 ---
 .../Dependencies-20240724-040744.yaml         |  6 ---
 .../unreleased/Features-20240505-011838.yaml  |  6 ---
 .../unreleased/Features-20240911-234859.yaml  |  6 ---
 .../unreleased/Fixes-20241028-172719.yaml     |  6 ---
 .../Under the Hood-20240910-212052.yaml       |  6 ---
 .../Under the Hood-20241104-173815.yaml       |  7 ---
 .../Under the Hood-20241107-143856.yaml       |  6 ---
 CHANGELOG.md                                  | 46 -------------------
 dbt/adapters/bigquery/__version__.py          |  2 +-
 36 files changed, 2 insertions(+), 288 deletions(-)
 delete mode 100644 .changes/1.9.0-b1.md
 delete mode 100644 .changes/1.9.0/Dependencies-20231211-001048.yaml
 delete mode 100644 .changes/1.9.0/Dependencies-20231220-002130.yaml
 delete mode 100644 .changes/1.9.0/Dependencies-20231222-002351.yaml
 delete mode 100644 .changes/1.9.0/Dependencies-20240105-004800.yaml
 delete mode 100644 .changes/1.9.0/Dependencies-20240429-005158.yaml
 delete mode 100644 .changes/1.9.0/Dependencies-20240429-005159.yaml
 delete mode 100644 .changes/1.9.0/Dependencies-20240520-230208.yaml
 delete mode 100644 .changes/1.9.0/Dependencies-20240718-005755.yaml
 delete mode 100644 .changes/1.9.0/Dependencies-20240718-005756.yaml
 delete mode 100644 .changes/1.9.0/Dependencies-20240718-005757.yaml
 delete mode 100644 .changes/1.9.0/Dependencies-20240719-003740.yaml
 delete mode 100644 .changes/1.9.0/Features-20240426-105319.yaml
 delete mode 100644 .changes/1.9.0/Features-20240430-185650.yaml
 delete mode 100644 .changes/1.9.0/Features-20240501-151902.yaml
 delete mode 100644 .changes/1.9.0/Features-20240516-125735.yaml
 delete mode 100644 .changes/1.9.0/Features-20240730-135911.yaml
 delete mode 100644 .changes/1.9.0/Features-20240925-232238.yaml
 delete mode 100644 .changes/1.9.0/Fixes-20240120-180818.yaml
 delete mode 100644 .changes/1.9.0/Fixes-20240201-145323.yaml
 delete mode 100644 .changes/1.9.0/Fixes-20240226-233024.yaml
 delete mode 100644 .changes/1.9.0/Fixes-20240426-105224.yaml
 delete mode 100644 .changes/1.9.0/Fixes-20241001-193207.yaml
 delete mode 100644 .changes/1.9.0/Under the Hood-20240331-101418.yaml
 delete mode 100644 .changes/1.9.0/Under the Hood-20240718-193206.yaml
 delete mode 100644 .changes/unreleased/Breaking Changes-20241016-185117.yaml
 delete mode 100644 .changes/unreleased/Dependencies-20240724-040744.yaml
 delete mode 100644 .changes/unreleased/Features-20240505-011838.yaml
 delete mode 100644 .changes/unreleased/Features-20240911-234859.yaml
 delete mode 100644 .changes/unreleased/Fixes-20241028-172719.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20240910-212052.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20241104-173815.yaml
 delete mode 100644 .changes/unreleased/Under the Hood-20241107-143856.yaml

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
index bd9430cbe..b56a8ee4f 100644
--- a/.bumpversion.cfg
+++ b/.bumpversion.cfg
@@ -1,5 +1,5 @@
 [bumpversion]
-current_version = 1.9.0b1
+current_version = 1.10.0a1
 parse = (?P<major>[\d]+) # major version number
 	\.(?P<minor>[\d]+) # minor version number
 	\.(?P<patch>[\d]+) # patch version number
diff --git a/.changes/1.9.0-b1.md b/.changes/1.9.0-b1.md
deleted file mode 100644
index 7d0dd2c8f..000000000
--- a/.changes/1.9.0-b1.md
+++ /dev/null
@@ -1,44 +0,0 @@
-## dbt-bigquery 1.9.0-b1 - October 02, 2024
-
-### Features
-
-- Add configuration options `enable_list_inference` and `intermediate_format` for python models ([#1047](https://github.com/dbt-labs/dbt-bigquery/issues/1047), [#1114](https://github.com/dbt-labs/dbt-bigquery/issues/1114))
-- Add tests for cross-database `cast` macro ([#1214](https://github.com/dbt-labs/dbt-bigquery/issues/1214))
-- Cross-database `date` macro ([#1221](https://github.com/dbt-labs/dbt-bigquery/issues/1221))
-- Add support for base 64 encoded json keyfile credentials ([#923](https://github.com/dbt-labs/dbt-bigquery/issues/923))
-- Add support for cancelling queries on keyboard interrupt ([#917](https://github.com/dbt-labs/dbt-bigquery/issues/917))
-- Add Microbatch Strategy to dbt-spark ([#1354](https://github.com/dbt-labs/dbt-bigquery/issues/1354))
-
-### Fixes
-
-- Drop intermediate objects created in BigQuery for incremental models ([#1036](https://github.com/dbt-labs/dbt-bigquery/issues/1036))
-- Fix null column index issue during `dbt docs generate` for external tables ([#1079](https://github.com/dbt-labs/dbt-bigquery/issues/1079))
-- make seed delimiter configurable via `field_delimeter` in model config ([#1119](https://github.com/dbt-labs/dbt-bigquery/issues/1119))
-- Default `enableListInference` to `True` for python models to support nested lists ([#1047](https://github.com/dbt-labs/dbt-bigquery/issues/1047), [#1114](https://github.com/dbt-labs/dbt-bigquery/issues/1114))
-- Catch additional database error exception, NotFound, as a DbtDatabaseError instead of defaulting to a DbtRuntimeError ([#1360](https://github.com/dbt-labs/dbt-bigquery/issues/1360))
-
-### Under the Hood
-
-- Lazy load `agate` ([#1162](https://github.com/dbt-labs/dbt-bigquery/issues/1162))
-- Simplify linting environment and dev dependencies ([#1291](https://github.com/dbt-labs/dbt-bigquery/issues/1291))
-
-### Dependencies
-
-- Update pre-commit requirement from ~=3.5 to ~=3.7 ([#1052](https://github.com/dbt-labs/dbt-bigquery/pull/1052))
-- Update freezegun requirement from ~=1.3 to ~=1.4 ([#1062](https://github.com/dbt-labs/dbt-bigquery/pull/1062))
-- Bump mypy from 1.7.1 to 1.8.0 ([#1064](https://github.com/dbt-labs/dbt-bigquery/pull/1064))
-- Update flake8 requirement from ~=6.1 to ~=7.0 ([#1069](https://github.com/dbt-labs/dbt-bigquery/pull/1069))
-- Bump actions/download-artifact from 3 to 4 ([#1209](https://github.com/dbt-labs/dbt-bigquery/pull/1209))
-- Bump actions/upload-artifact from 3 to 4 ([#1210](https://github.com/dbt-labs/dbt-bigquery/pull/1210))
-- Bump ubuntu from 22.04 to 24.04 in /docker ([#1247](https://github.com/dbt-labs/dbt-bigquery/pull/1247))
-- Update pre-commit-hooks requirement from ~=4.5 to ~=4.6 ([#1281](https://github.com/dbt-labs/dbt-bigquery/pull/1281))
-- Update pytest-xdist requirement from ~=3.5 to ~=3.6 ([#1282](https://github.com/dbt-labs/dbt-bigquery/pull/1282))
-- Update flaky requirement from ~=3.7 to ~=3.8 ([#1283](https://github.com/dbt-labs/dbt-bigquery/pull/1283))
-- Update twine requirement from ~=4.0 to ~=5.1 ([#1293](https://github.com/dbt-labs/dbt-bigquery/pull/1293))
-
-### Contributors
-- [@d-cole](https://github.com/d-cole) ([#917](https://github.com/dbt-labs/dbt-bigquery/issues/917))
-- [@dwreeves](https://github.com/dwreeves) ([#1162](https://github.com/dbt-labs/dbt-bigquery/issues/1162))
-- [@robeleb1](https://github.com/robeleb1) ([#923](https://github.com/dbt-labs/dbt-bigquery/issues/923))
-- [@salimmoulouel](https://github.com/salimmoulouel) ([#1119](https://github.com/dbt-labs/dbt-bigquery/issues/1119))
-- [@vinit2107](https://github.com/vinit2107) ([#1036](https://github.com/dbt-labs/dbt-bigquery/issues/1036))
diff --git a/.changes/1.9.0/Dependencies-20231211-001048.yaml b/.changes/1.9.0/Dependencies-20231211-001048.yaml
deleted file mode 100644
index 6f2bfada4..000000000
--- a/.changes/1.9.0/Dependencies-20231211-001048.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pre-commit requirement from ~=3.5 to ~=3.7"
-time: 2023-12-11T00:10:48.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1052
diff --git a/.changes/1.9.0/Dependencies-20231220-002130.yaml b/.changes/1.9.0/Dependencies-20231220-002130.yaml
deleted file mode 100644
index d62e50bf2..000000000
--- a/.changes/1.9.0/Dependencies-20231220-002130.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update freezegun requirement from ~=1.3 to ~=1.4"
-time: 2023-12-20T00:21:30.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1062
diff --git a/.changes/1.9.0/Dependencies-20231222-002351.yaml b/.changes/1.9.0/Dependencies-20231222-002351.yaml
deleted file mode 100644
index 76591de93..000000000
--- a/.changes/1.9.0/Dependencies-20231222-002351.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump mypy from 1.7.1 to 1.8.0"
-time: 2023-12-22T00:23:51.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1064
diff --git a/.changes/1.9.0/Dependencies-20240105-004800.yaml b/.changes/1.9.0/Dependencies-20240105-004800.yaml
deleted file mode 100644
index b0d33ceed..000000000
--- a/.changes/1.9.0/Dependencies-20240105-004800.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update flake8 requirement from ~=6.1 to ~=7.0"
-time: 2024-01-05T00:48:00.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1069
diff --git a/.changes/1.9.0/Dependencies-20240429-005158.yaml b/.changes/1.9.0/Dependencies-20240429-005158.yaml
deleted file mode 100644
index 5d380952c..000000000
--- a/.changes/1.9.0/Dependencies-20240429-005158.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump actions/download-artifact from 3 to 4"
-time: 2024-04-29T00:51:58.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1209
diff --git a/.changes/1.9.0/Dependencies-20240429-005159.yaml b/.changes/1.9.0/Dependencies-20240429-005159.yaml
deleted file mode 100644
index adf2cae65..000000000
--- a/.changes/1.9.0/Dependencies-20240429-005159.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump actions/upload-artifact from 3 to 4"
-time: 2024-04-29T00:51:59.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1210
diff --git a/.changes/1.9.0/Dependencies-20240520-230208.yaml b/.changes/1.9.0/Dependencies-20240520-230208.yaml
deleted file mode 100644
index f89057233..000000000
--- a/.changes/1.9.0/Dependencies-20240520-230208.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Bump ubuntu from 22.04 to 24.04 in /docker"
-time: 2024-05-20T23:02:08.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1247
diff --git a/.changes/1.9.0/Dependencies-20240718-005755.yaml b/.changes/1.9.0/Dependencies-20240718-005755.yaml
deleted file mode 100644
index 3d2cca66c..000000000
--- a/.changes/1.9.0/Dependencies-20240718-005755.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pre-commit-hooks requirement from ~=4.5 to ~=4.6"
-time: 2024-07-18T00:57:55.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1281
diff --git a/.changes/1.9.0/Dependencies-20240718-005756.yaml b/.changes/1.9.0/Dependencies-20240718-005756.yaml
deleted file mode 100644
index ac6b791a8..000000000
--- a/.changes/1.9.0/Dependencies-20240718-005756.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update pytest-xdist requirement from ~=3.5 to ~=3.6"
-time: 2024-07-18T00:57:56.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1282
diff --git a/.changes/1.9.0/Dependencies-20240718-005757.yaml b/.changes/1.9.0/Dependencies-20240718-005757.yaml
deleted file mode 100644
index 29e12d68e..000000000
--- a/.changes/1.9.0/Dependencies-20240718-005757.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update flaky requirement from ~=3.7 to ~=3.8"
-time: 2024-07-18T00:57:57.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1283
diff --git a/.changes/1.9.0/Dependencies-20240719-003740.yaml b/.changes/1.9.0/Dependencies-20240719-003740.yaml
deleted file mode 100644
index feb483a60..000000000
--- a/.changes/1.9.0/Dependencies-20240719-003740.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update twine requirement from ~=4.0 to ~=5.1"
-time: 2024-07-19T00:37:40.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1293
diff --git a/.changes/1.9.0/Features-20240426-105319.yaml b/.changes/1.9.0/Features-20240426-105319.yaml
deleted file mode 100644
index 0af2f9aa8..000000000
--- a/.changes/1.9.0/Features-20240426-105319.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Features
-body: Add configuration options `enable_list_inference` and `intermediate_format` for python
-  models
-time: 2024-04-26T10:53:19.874239-04:00
-custom:
-  Author: mikealfare
-  Issue: 1047 1114
diff --git a/.changes/1.9.0/Features-20240430-185650.yaml b/.changes/1.9.0/Features-20240430-185650.yaml
deleted file mode 100644
index 0c0eef567..000000000
--- a/.changes/1.9.0/Features-20240430-185650.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Add tests for cross-database `cast` macro
-time: 2024-04-30T18:56:50.238027-06:00
-custom:
-  Author: dbeatty10
-  Issue: "1214"
diff --git a/.changes/1.9.0/Features-20240501-151902.yaml b/.changes/1.9.0/Features-20240501-151902.yaml
deleted file mode 100644
index 1522e9775..000000000
--- a/.changes/1.9.0/Features-20240501-151902.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Cross-database `date` macro
-time: 2024-05-01T15:19:02.929676-06:00
-custom:
-  Author: dbeatty10
-  Issue: 1221
diff --git a/.changes/1.9.0/Features-20240516-125735.yaml b/.changes/1.9.0/Features-20240516-125735.yaml
deleted file mode 100644
index d84b098b2..000000000
--- a/.changes/1.9.0/Features-20240516-125735.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Add support for base 64 encoded json keyfile credentials
-time: 2024-05-16T12:57:35.383416-07:00
-custom:
-    Author: robeleb1
-    Issue: "923"
diff --git a/.changes/1.9.0/Features-20240730-135911.yaml b/.changes/1.9.0/Features-20240730-135911.yaml
deleted file mode 100644
index 52868c2ee..000000000
--- a/.changes/1.9.0/Features-20240730-135911.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Add support for cancelling queries on keyboard interrupt
-time: 2024-07-30T13:59:11.585452-07:00
-custom:
-  Author: d-cole MichelleArk colin-rogers-dbt
-  Issue: "917"
diff --git a/.changes/1.9.0/Features-20240925-232238.yaml b/.changes/1.9.0/Features-20240925-232238.yaml
deleted file mode 100644
index 903884196..000000000
--- a/.changes/1.9.0/Features-20240925-232238.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Add Microbatch Strategy to dbt-spark
-time: 2024-09-25T23:22:38.216277+01:00
-custom:
-  Author: michelleark
-  Issue: "1354"
diff --git a/.changes/1.9.0/Fixes-20240120-180818.yaml b/.changes/1.9.0/Fixes-20240120-180818.yaml
deleted file mode 100644
index 0d0740361..000000000
--- a/.changes/1.9.0/Fixes-20240120-180818.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Drop intermediate objects created in BigQuery for incremental models
-time: 2024-01-20T18:08:18.817915-06:00
-custom:
-  Author: vinit2107
-  Issue: "1036"
diff --git a/.changes/1.9.0/Fixes-20240201-145323.yaml b/.changes/1.9.0/Fixes-20240201-145323.yaml
deleted file mode 100644
index ea198e54a..000000000
--- a/.changes/1.9.0/Fixes-20240201-145323.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: Fix null column index issue during `dbt docs generate` for external tables
-time: 2024-02-01T14:53:23.434624-05:00
-custom:
-  Author: mikealfare
-  Issue: "1079"
diff --git a/.changes/1.9.0/Fixes-20240226-233024.yaml b/.changes/1.9.0/Fixes-20240226-233024.yaml
deleted file mode 100644
index efb1b077c..000000000
--- a/.changes/1.9.0/Fixes-20240226-233024.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: make seed delimiter configurable via `field_delimeter` in model config
-time: 2024-02-26T23:30:24.141213+01:00
-custom:
-  Author: salimmoulouel
-  Issue: "1119"
diff --git a/.changes/1.9.0/Fixes-20240426-105224.yaml b/.changes/1.9.0/Fixes-20240426-105224.yaml
deleted file mode 100644
index 624006ba5..000000000
--- a/.changes/1.9.0/Fixes-20240426-105224.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: Default `enableListInference` to `True` for python models to support nested
-  lists
-time: 2024-04-26T10:52:24.827314-04:00
-custom:
-  Author: mikealfare
-  Issue: 1047 1114
diff --git a/.changes/1.9.0/Fixes-20241001-193207.yaml b/.changes/1.9.0/Fixes-20241001-193207.yaml
deleted file mode 100644
index 584445a5b..000000000
--- a/.changes/1.9.0/Fixes-20241001-193207.yaml
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Fixes
-body: Catch additional database error exception, NotFound, as a DbtDatabaseError instead
-  of defaulting to a DbtRuntimeError
-time: 2024-10-01T19:32:07.304353-04:00
-custom:
-  Author: mikealfare
-  Issue: "1360"
diff --git a/.changes/1.9.0/Under the Hood-20240331-101418.yaml b/.changes/1.9.0/Under the Hood-20240331-101418.yaml
deleted file mode 100644
index baea00248..000000000
--- a/.changes/1.9.0/Under the Hood-20240331-101418.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Lazy load `agate`
-time: 2024-03-31T10:14:18.260074-04:00
-custom:
-  Author: dwreeves
-  Issue: "1162"
diff --git a/.changes/1.9.0/Under the Hood-20240718-193206.yaml b/.changes/1.9.0/Under the Hood-20240718-193206.yaml
deleted file mode 100644
index 32b3084f5..000000000
--- a/.changes/1.9.0/Under the Hood-20240718-193206.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Simplify linting environment and dev dependencies
-time: 2024-07-18T19:32:06.044016-04:00
-custom:
-  Author: mikealfare
-  Issue: "1291"
diff --git a/.changes/unreleased/Breaking Changes-20241016-185117.yaml b/.changes/unreleased/Breaking Changes-20241016-185117.yaml
deleted file mode 100644
index 55bb37461..000000000
--- a/.changes/unreleased/Breaking Changes-20241016-185117.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Breaking Changes
-body: Drop support for Python 3.8
-time: 2024-10-16T18:51:17.581547-04:00
-custom:
-  Author: mikealfare
-  Issue: "1373"
diff --git a/.changes/unreleased/Dependencies-20240724-040744.yaml b/.changes/unreleased/Dependencies-20240724-040744.yaml
deleted file mode 100644
index fd713788e..000000000
--- a/.changes/unreleased/Dependencies-20240724-040744.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: "Dependencies"
-body: "Update wheel requirement from ~=0.42 to ~=0.43"
-time: 2024-07-24T04:07:44.00000Z
-custom:
-  Author: dependabot[bot]
-  PR: 1304
diff --git a/.changes/unreleased/Features-20240505-011838.yaml b/.changes/unreleased/Features-20240505-011838.yaml
deleted file mode 100644
index 66411853f..000000000
--- a/.changes/unreleased/Features-20240505-011838.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: add is_retryable test case when raise ServiceUnavailable
-time: 2024-05-05T01:18:38.737882+09:00
-custom:
-  Author: jx2lee
-  Issue: "682"
diff --git a/.changes/unreleased/Features-20240911-234859.yaml b/.changes/unreleased/Features-20240911-234859.yaml
deleted file mode 100644
index 5351c3315..000000000
--- a/.changes/unreleased/Features-20240911-234859.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Features
-body: Adds the ability to set optional `quota_project` in profile
-time: 2024-09-11T23:48:59.767649+01:00
-custom:
-    Author: jcarpenter12
-    Issue: 1343 1344
diff --git a/.changes/unreleased/Fixes-20241028-172719.yaml b/.changes/unreleased/Fixes-20241028-172719.yaml
deleted file mode 100644
index 87ee2c25d..000000000
--- a/.changes/unreleased/Fixes-20241028-172719.yaml
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Fixes
-body: use "direct" write for non-partitioned python model materializations
-time: 2024-10-28T17:27:19.306348-07:00
-custom:
-  Author: colin-rogers-dbt
-  Issue: "1318"
diff --git a/.changes/unreleased/Under the Hood-20240910-212052.yaml b/.changes/unreleased/Under the Hood-20240910-212052.yaml
deleted file mode 100644
index 3e4885dcd..000000000
--- a/.changes/unreleased/Under the Hood-20240910-212052.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Isolating distribution testing
-time: 2024-09-10T21:20:52.574204-04:00
-custom:
-    Author: leahwicz
-    Issue: "1290"
diff --git a/.changes/unreleased/Under the Hood-20241104-173815.yaml b/.changes/unreleased/Under the Hood-20241104-173815.yaml
deleted file mode 100644
index e3e81dec1..000000000
--- a/.changes/unreleased/Under the Hood-20241104-173815.yaml	
+++ /dev/null
@@ -1,7 +0,0 @@
-kind: Under the Hood
-body: Separate credentials functionality into its own module for reuse in retry and
-  python submissions
-time: 2024-11-04T17:38:15.940962-05:00
-custom:
-  Author: mikealfare
-  Issue: "1391"
diff --git a/.changes/unreleased/Under the Hood-20241107-143856.yaml b/.changes/unreleased/Under the Hood-20241107-143856.yaml
deleted file mode 100644
index db8557bf0..000000000
--- a/.changes/unreleased/Under the Hood-20241107-143856.yaml	
+++ /dev/null
@@ -1,6 +0,0 @@
-kind: Under the Hood
-body: Create a retry factory to simplify retry strategies across dbt-bigquery
-time: 2024-11-07T14:38:56.210445-05:00
-custom:
-  Author: mikealfare osalama
-  Issue: "1395"
diff --git a/CHANGELOG.md b/CHANGELOG.md
index b9bda350a..ade60b8f6 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -5,52 +5,6 @@
 - "Breaking changes" listed under a version may require action from end users or external maintainers when upgrading to that version.
 - Do not edit this file directly. This file is auto-generated using [changie](https://github.com/miniscruff/changie). For details on how to document a change, see [the contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md#adding-changelog-entry)
 
-## dbt-bigquery 1.9.0-b1 - October 02, 2024
-
-### Features
-
-- Add configuration options `enable_list_inference` and `intermediate_format` for python models ([#1047](https://github.com/dbt-labs/dbt-bigquery/issues/1047), [#1114](https://github.com/dbt-labs/dbt-bigquery/issues/1114))
-- Add tests for cross-database `cast` macro ([#1214](https://github.com/dbt-labs/dbt-bigquery/issues/1214))
-- Cross-database `date` macro ([#1221](https://github.com/dbt-labs/dbt-bigquery/issues/1221))
-- Add support for base 64 encoded json keyfile credentials ([#923](https://github.com/dbt-labs/dbt-bigquery/issues/923))
-- Add support for cancelling queries on keyboard interrupt ([#917](https://github.com/dbt-labs/dbt-bigquery/issues/917))
-- Add Microbatch Strategy to dbt-spark ([#1354](https://github.com/dbt-labs/dbt-bigquery/issues/1354))
-
-### Fixes
-
-- Drop intermediate objects created in BigQuery for incremental models ([#1036](https://github.com/dbt-labs/dbt-bigquery/issues/1036))
-- Fix null column index issue during `dbt docs generate` for external tables ([#1079](https://github.com/dbt-labs/dbt-bigquery/issues/1079))
-- make seed delimiter configurable via `field_delimeter` in model config ([#1119](https://github.com/dbt-labs/dbt-bigquery/issues/1119))
-- Default `enableListInference` to `True` for python models to support nested lists ([#1047](https://github.com/dbt-labs/dbt-bigquery/issues/1047), [#1114](https://github.com/dbt-labs/dbt-bigquery/issues/1114))
-- Catch additional database error exception, NotFound, as a DbtDatabaseError instead of defaulting to a DbtRuntimeError ([#1360](https://github.com/dbt-labs/dbt-bigquery/issues/1360))
-
-### Under the Hood
-
-- Lazy load `agate` ([#1162](https://github.com/dbt-labs/dbt-bigquery/issues/1162))
-- Simplify linting environment and dev dependencies ([#1291](https://github.com/dbt-labs/dbt-bigquery/issues/1291))
-
-### Dependencies
-
-- Update pre-commit requirement from ~=3.5 to ~=3.7 ([#1052](https://github.com/dbt-labs/dbt-bigquery/pull/1052))
-- Update freezegun requirement from ~=1.3 to ~=1.4 ([#1062](https://github.com/dbt-labs/dbt-bigquery/pull/1062))
-- Bump mypy from 1.7.1 to 1.8.0 ([#1064](https://github.com/dbt-labs/dbt-bigquery/pull/1064))
-- Update flake8 requirement from ~=6.1 to ~=7.0 ([#1069](https://github.com/dbt-labs/dbt-bigquery/pull/1069))
-- Bump actions/download-artifact from 3 to 4 ([#1209](https://github.com/dbt-labs/dbt-bigquery/pull/1209))
-- Bump actions/upload-artifact from 3 to 4 ([#1210](https://github.com/dbt-labs/dbt-bigquery/pull/1210))
-- Bump ubuntu from 22.04 to 24.04 in /docker ([#1247](https://github.com/dbt-labs/dbt-bigquery/pull/1247))
-- Update pre-commit-hooks requirement from ~=4.5 to ~=4.6 ([#1281](https://github.com/dbt-labs/dbt-bigquery/pull/1281))
-- Update pytest-xdist requirement from ~=3.5 to ~=3.6 ([#1282](https://github.com/dbt-labs/dbt-bigquery/pull/1282))
-- Update flaky requirement from ~=3.7 to ~=3.8 ([#1283](https://github.com/dbt-labs/dbt-bigquery/pull/1283))
-- Update twine requirement from ~=4.0 to ~=5.1 ([#1293](https://github.com/dbt-labs/dbt-bigquery/pull/1293))
-
-### Contributors
-- [@d-cole](https://github.com/d-cole) ([#917](https://github.com/dbt-labs/dbt-bigquery/issues/917))
-- [@dwreeves](https://github.com/dwreeves) ([#1162](https://github.com/dbt-labs/dbt-bigquery/issues/1162))
-- [@robeleb1](https://github.com/robeleb1) ([#923](https://github.com/dbt-labs/dbt-bigquery/issues/923))
-- [@salimmoulouel](https://github.com/salimmoulouel) ([#1119](https://github.com/dbt-labs/dbt-bigquery/issues/1119))
-- [@vinit2107](https://github.com/vinit2107) ([#1036](https://github.com/dbt-labs/dbt-bigquery/issues/1036))
-
-
 ## Previous Releases
 For information on prior major and minor releases, see their changelogs:
 - [1.6](https://github.com/dbt-labs/dbt-bigquery/blob/1.6.latest/CHANGELOG.md)
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt/adapters/bigquery/__version__.py
index a4077fff2..1af777a62 100644
--- a/dbt/adapters/bigquery/__version__.py
+++ b/dbt/adapters/bigquery/__version__.py
@@ -1 +1 @@
-version = "1.9.0b1"
+version = "1.10.0a1"

From 1798601e7ec04ea9211a53fc7b4bdbb9f235a7de Mon Sep 17 00:00:00 2001
From: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
Date: Wed, 4 Dec 2024 09:05:26 -0800
Subject: [PATCH 853/860] update libpq-dev dependency to 13.18-0+deb11u1
 (#1420)

* use dynamic schema in test_grant_access_to.py

* use dynamic schema in test_grant_access_to.py

* revert setup

* update libpq-dev dependency to 13.18-0+deb11u1
---
 docker/Dockerfile | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docker/Dockerfile b/docker/Dockerfile
index bda507dc5..8f371d6b4 100644
--- a/docker/Dockerfile
+++ b/docker/Dockerfile
@@ -9,7 +9,7 @@ RUN apt-get update \
     build-essential=12.9 \
     ca-certificates=20210119 \
     git=1:2.30.2-1+deb11u2 \
-    libpq-dev=13.14-0+deb11u1 \
+    libpq-dev=13.18-0+deb11u1 \
     make=4.3-4.1 \
     openssh-client=1:8.4p1-5+deb11u3 \
     software-properties-common=0.96.20.2-2.1 \

From 26c19e911f6d493397578c05dd218496de03a334 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Thu, 5 Dec 2024 14:47:35 -0500
Subject: [PATCH 854/860] Remove custom retry in get_table call (#1423)

* remove custom retry in get_table call

* changelog
---
 .changes/unreleased/Fixes-20241205-133606.yaml | 6 ++++++
 dbt/adapters/bigquery/connections.py           | 5 +----
 2 files changed, 7 insertions(+), 4 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20241205-133606.yaml

diff --git a/.changes/unreleased/Fixes-20241205-133606.yaml b/.changes/unreleased/Fixes-20241205-133606.yaml
new file mode 100644
index 000000000..b88a0981c
--- /dev/null
+++ b/.changes/unreleased/Fixes-20241205-133606.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fix issue where rate limit errors on table service calls are not retried
+time: 2024-12-05T13:36:06.436005-05:00
+custom:
+  Author: mikealfare
+  Issue: "1423"
diff --git a/dbt/adapters/bigquery/connections.py b/dbt/adapters/bigquery/connections.py
index 61fa87d40..bb062f330 100644
--- a/dbt/adapters/bigquery/connections.py
+++ b/dbt/adapters/bigquery/connections.py
@@ -519,10 +519,7 @@ def get_bq_table(self, database, schema, identifier) -> Table:
         # backwards compatibility: fill in with defaults if not specified
         database = database or conn.credentials.database
         schema = schema or conn.credentials.schema
-        return client.get_table(
-            table=self.table_ref(database, schema, identifier),
-            retry=self._retry.create_reopen_with_deadline(conn),
-        )
+        return client.get_table(self.table_ref(database, schema, identifier))
 
     def drop_dataset(self, database, schema) -> None:
         conn = self.get_thread_connection()

From 4d255b2f854d21d5d8871bdaa8d7ab47e7e863a3 Mon Sep 17 00:00:00 2001
From: Michelle Ark <MichelleArk@users.noreply.github.com>
Date: Thu, 5 Dec 2024 16:26:52 -0500
Subject: [PATCH 855/860] Fix: Cast to timestamp prior to event time comparison
 (#1422)

---
 .../unreleased/Fixes-20241204-105846.yaml     |  7 +++++
 dbt/adapters/bigquery/relation.py             | 25 +++++++++++++++-
 .../incremental_strategy_fixtures.py          | 20 ++++++++++++-
 .../test_incremental_microbatch.py            | 30 +++++++++++++++++--
 4 files changed, 78 insertions(+), 4 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20241204-105846.yaml

diff --git a/.changes/unreleased/Fixes-20241204-105846.yaml b/.changes/unreleased/Fixes-20241204-105846.yaml
new file mode 100644
index 000000000..2693e4513
--- /dev/null
+++ b/.changes/unreleased/Fixes-20241204-105846.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: Cast `event_time` to a timestamp prior to comparing against microbatch start/end
+  time
+time: 2024-12-04T10:58:46.573608-05:00
+custom:
+  Author: michelleark
+  Issue: "1422"
diff --git a/dbt/adapters/bigquery/relation.py b/dbt/adapters/bigquery/relation.py
index 4edc8d7ac..037761918 100644
--- a/dbt/adapters/bigquery/relation.py
+++ b/dbt/adapters/bigquery/relation.py
@@ -4,7 +4,12 @@
 
 from dbt_common.exceptions import CompilationError
 from dbt_common.utils.dict import filter_null_values
-from dbt.adapters.base.relation import BaseRelation, ComponentName, InformationSchema
+from dbt.adapters.base.relation import (
+    BaseRelation,
+    ComponentName,
+    InformationSchema,
+    EventTimeFilter,
+)
 from dbt.adapters.contracts.relation import RelationConfig, RelationType
 from dbt.adapters.relation_configs import RelationConfigChangeAction
 
@@ -116,6 +121,24 @@ def materialized_view_config_changeset(
     def information_schema(self, identifier: Optional[str] = None) -> "BigQueryInformationSchema":
         return BigQueryInformationSchema.from_relation(self, identifier)
 
+    def _render_event_time_filtered(self, event_time_filter: EventTimeFilter) -> str:
+        """
+        Returns "" if start and end are both None
+        """
+        filter = ""
+        if event_time_filter.start and event_time_filter.end:
+            filter = f"cast({event_time_filter.field_name} as timestamp) >= '{event_time_filter.start}' and cast({event_time_filter.field_name} as timestamp) < '{event_time_filter.end}'"
+        elif event_time_filter.start:
+            filter = (
+                f"cast({event_time_filter.field_name} as timestamp) >= '{event_time_filter.start}'"
+            )
+        elif event_time_filter.end:
+            filter = (
+                f"cast({event_time_filter.field_name} as timestamp) < '{event_time_filter.end}'"
+            )
+
+        return filter
+
 
 @dataclass(frozen=True, eq=False, repr=False)
 class BigQueryInformationSchema(InformationSchema):
diff --git a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
index 02efbb6c2..365aba8c8 100644
--- a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
+++ b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
@@ -570,7 +570,7 @@
     begin=modules.datetime.datetime(2020, 1, 1, 0, 0, 0)
     )
 }}
-select * from {{ ref('input_model') }}
+select id, cast(event_time as timestamp) as event_time from {{ ref('input_model') }}
 """
 
 microbatch_input_sql = """
@@ -582,6 +582,24 @@
 select 3 as id, TIMESTAMP '2020-01-03 00:00:00-0' as event_time
 """
 
+microbatch_input_event_time_date_sql = """
+{{ config(materialized='table', event_time='event_time') }}
+select 1 as id, DATE '2020-01-01' as event_time
+union all
+select 2 as id, DATE '2020-01-02' as event_time
+union all
+select 3 as id, DATE '2020-01-03' as event_time
+"""
+
+microbatch_input_event_time_datetime_sql = """
+{{ config(materialized='table', event_time='event_time') }}
+select 1 as id, DATETIME '2020-01-01' as event_time
+union all
+select 2 as id, DATETIME '2020-01-02' as event_time
+union all
+select 3 as id, DATETIME '2020-01-03' as event_time
+"""
+
 microbatch_model_no_partition_by_sql = """
 {{ config(
     materialized='incremental',
diff --git a/tests/functional/adapter/incremental/test_incremental_microbatch.py b/tests/functional/adapter/incremental/test_incremental_microbatch.py
index d1bbbcea3..d0f8b62b7 100644
--- a/tests/functional/adapter/incremental/test_incremental_microbatch.py
+++ b/tests/functional/adapter/incremental/test_incremental_microbatch.py
@@ -13,6 +13,8 @@
     microbatch_input_sql,
     microbatch_model_no_partition_by_sql,
     microbatch_model_invalid_partition_by_sql,
+    microbatch_input_event_time_date_sql,
+    microbatch_input_event_time_datetime_sql,
 )
 
 
@@ -22,6 +24,32 @@ def microbatch_model_sql(self) -> str:
         return microbatch_model_no_unique_id_sql
 
 
+class TestBigQueryMicrobatchInputWithDate(TestBigQueryMicrobatch):
+    @pytest.fixture(scope="class")
+    def input_model_sql(self) -> str:
+        return microbatch_input_event_time_date_sql
+
+    @pytest.fixture(scope="class")
+    def insert_two_rows_sql(self, project) -> str:
+        test_schema_relation = project.adapter.Relation.create(
+            database=project.database, schema=project.test_schema
+        )
+        return f"insert into {test_schema_relation}.input_model (id, event_time) values (4, DATE '2020-01-04'), (5, DATE '2020-01-05')"
+
+
+class TestBigQueryMicrobatchInputWithDatetime(TestBigQueryMicrobatch):
+    @pytest.fixture(scope="class")
+    def input_model_sql(self) -> str:
+        return microbatch_input_event_time_datetime_sql
+
+    @pytest.fixture(scope="class")
+    def insert_two_rows_sql(self, project) -> str:
+        test_schema_relation = project.adapter.Relation.create(
+            database=project.database, schema=project.test_schema
+        )
+        return f"insert into {test_schema_relation}.input_model (id, event_time) values (4, DATETIME '2020-01-04'), (5, DATETIME '2020-01-05')"
+
+
 class TestBigQueryMicrobatchMissingPartitionBy:
     @pytest.fixture(scope="class")
     def models(self) -> str:
@@ -30,7 +58,6 @@ def models(self) -> str:
             "input_model.sql": microbatch_input_sql,
         }
 
-    @mock.patch.dict(os.environ, {"DBT_EXPERIMENTAL_MICROBATCH": "True"})
     def test_execution_failure_no_partition_by(self, project):
         with patch_microbatch_end_time("2020-01-03 13:57:00"):
             _, stdout = run_dbt_and_capture(["run"], expect_pass=False)
@@ -45,7 +72,6 @@ def models(self) -> str:
             "input_model.sql": microbatch_input_sql,
         }
 
-    @mock.patch.dict(os.environ, {"DBT_EXPERIMENTAL_MICROBATCH": "True"})
     def test_execution_failure_no_partition_by(self, project):
         with patch_microbatch_end_time("2020-01-03 13:57:00"):
             _, stdout = run_dbt_and_capture(["run"], expect_pass=False)

From 827fc78af56803ede19a990daa432a94c27108fc Mon Sep 17 00:00:00 2001
From: Borja Vazquez-Barreiros <75082703+borjavb@users.noreply.github.com>
Date: Tue, 10 Dec 2024 21:51:03 +0000
Subject: [PATCH 856/860] [Feature] Allow `copy_partitions` when using
 `microbatch` (#1421)

---
 .../unreleased/Features-20241202-223835.yaml   |  6 ++++++
 .../macros/materializations/incremental.sql    |  4 ++--
 .../incremental_strategy_fixtures.py           | 18 ++++++++++++++++++
 .../incremental/test_incremental_microbatch.py |  7 +++++++
 4 files changed, 33 insertions(+), 2 deletions(-)
 create mode 100644 .changes/unreleased/Features-20241202-223835.yaml

diff --git a/.changes/unreleased/Features-20241202-223835.yaml b/.changes/unreleased/Features-20241202-223835.yaml
new file mode 100644
index 000000000..ab59abd99
--- /dev/null
+++ b/.changes/unreleased/Features-20241202-223835.yaml
@@ -0,0 +1,6 @@
+kind: Features
+body: Allow copy_partitions in microbatch
+time: 2024-12-02T22:38:35.479052Z
+custom:
+    Author: borjavb
+    Issue: "1414"
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt/include/bigquery/macros/materializations/incremental.sql
index 935280d63..25a83b0c6 100644
--- a/dbt/include/bigquery/macros/materializations/incremental.sql
+++ b/dbt/include/bigquery/macros/materializations/incremental.sql
@@ -95,9 +95,9 @@
 
   {{ run_hooks(pre_hooks) }}
 
-  {% if partition_by.copy_partitions is true and strategy != 'insert_overwrite' %} {#-- We can't copy partitions with merge strategy --#}
+  {% if partition_by.copy_partitions is true and strategy not in ['insert_overwrite', 'microbatch'] %} {#-- We can't copy partitions with merge strategy --#}
         {% set wrong_strategy_msg -%}
-        The 'copy_partitions' option requires the 'incremental_strategy' option to be set to 'insert_overwrite'.
+        The 'copy_partitions' option requires the 'incremental_strategy' option to be set to 'insert_overwrite' or 'microbatch'.
         {%- endset %}
         {% do exceptions.raise_compiler_error(wrong_strategy_msg) %}
 
diff --git a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
index 365aba8c8..21d5f15b6 100644
--- a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
+++ b/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
@@ -629,3 +629,21 @@
 }}
 select * from {{ ref('input_model') }}
 """
+
+microbatch_model_no_unique_id_copy_partitions_sql = """
+{{ config(
+    materialized='incremental',
+    incremental_strategy='microbatch',
+    partition_by={
+      'field': 'event_time',
+      'data_type': 'timestamp',
+      'granularity': 'day',
+      'copy_partitions': true
+    },
+    event_time='event_time',
+    batch_size='day',
+    begin=modules.datetime.datetime(2020, 1, 1, 0, 0, 0)
+    )
+}}
+select * from {{ ref('input_model') }}
+"""
diff --git a/tests/functional/adapter/incremental/test_incremental_microbatch.py b/tests/functional/adapter/incremental/test_incremental_microbatch.py
index d0f8b62b7..912f96eec 100644
--- a/tests/functional/adapter/incremental/test_incremental_microbatch.py
+++ b/tests/functional/adapter/incremental/test_incremental_microbatch.py
@@ -13,6 +13,7 @@
     microbatch_input_sql,
     microbatch_model_no_partition_by_sql,
     microbatch_model_invalid_partition_by_sql,
+    microbatch_model_no_unique_id_copy_partitions_sql,
     microbatch_input_event_time_date_sql,
     microbatch_input_event_time_datetime_sql,
 )
@@ -79,3 +80,9 @@ def test_execution_failure_no_partition_by(self, project):
             "The 'microbatch' strategy requires a `partition_by` config with the same granularity as its configured `batch_size`"
             in stdout
         )
+
+
+class TestBigQueryMicrobatchWithCopyPartitions(BaseMicrobatch):
+    @pytest.fixture(scope="class")
+    def microbatch_model_sql(self) -> str:
+        return microbatch_model_no_unique_id_copy_partitions_sql

From e1b6e74974f1cddf4100f82f2b92a3baa22a37cf Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Tue, 10 Dec 2024 19:01:49 -0500
Subject: [PATCH 857/860] ADAP-1116: Move to `hatch` and `pyproject.toml`
 (#1407)

* move setup.py to pyproject.toml
* move dev tool config to pyproject.toml
* update integration.yml to use hatch commands
* update main.yml to use hatch commands
* update scripts for pyproject.toml and hatch.toml references
* update release workflow to use hatch

---------

Co-authored-by: Colin Rogers <111200756+colin-rogers-dbt@users.noreply.github.com>
---
 .bumpversion.cfg                              |  35 --
 .../Under the Hood-20241117-194746.yaml       |   6 +
 {scripts => .github/scripts}/env-setup.sh     |   0
 .github/scripts/update_dependencies.sh        |   6 +-
 .../scripts/update_dev_dependency_branches.sh |   4 +-
 .github/workflows/integration.yml             |  52 +-
 .github/workflows/main.yml                    |  60 +--
 .github/workflows/nightly-release.yml         |  19 +-
 .github/workflows/release.yml                 | 112 ++---
 .github/workflows/release_prep_hatch.yml      | 455 ++++++++++++++++++
 .github/workflows/version-bump.yml            |  28 --
 MANIFEST.in                                   |   1 -
 Makefile                                      |  21 -
 dev-requirements.txt                          |  20 -
 hatch.toml                                    |  61 +++
 mypy.ini                                      |   2 -
 pyproject.toml                                |  57 +++
 pytest.ini                                    |  10 -
 scripts/build-dist.sh                         |  20 -
 setup.py                                      |  78 ---
 tox.ini                                       |  54 ---
 21 files changed, 657 insertions(+), 444 deletions(-)
 delete mode 100644 .bumpversion.cfg
 create mode 100644 .changes/unreleased/Under the Hood-20241117-194746.yaml
 rename {scripts => .github/scripts}/env-setup.sh (100%)
 create mode 100644 .github/workflows/release_prep_hatch.yml
 delete mode 100644 .github/workflows/version-bump.yml
 delete mode 100644 MANIFEST.in
 delete mode 100644 Makefile
 delete mode 100644 dev-requirements.txt
 create mode 100644 hatch.toml
 delete mode 100644 mypy.ini
 create mode 100644 pyproject.toml
 delete mode 100644 pytest.ini
 delete mode 100755 scripts/build-dist.sh
 delete mode 100644 setup.py
 delete mode 100644 tox.ini

diff --git a/.bumpversion.cfg b/.bumpversion.cfg
deleted file mode 100644
index b56a8ee4f..000000000
--- a/.bumpversion.cfg
+++ /dev/null
@@ -1,35 +0,0 @@
-[bumpversion]
-current_version = 1.10.0a1
-parse = (?P<major>[\d]+) # major version number
-	\.(?P<minor>[\d]+) # minor version number
-	\.(?P<patch>[\d]+) # patch version number
-	(?P<prerelease>  # optional pre-release - ex: a1, b2, rc25
-	(?P<prekind>a|b|rc) # pre-release type
-	(?P<num>[\d]+) # pre-release version number
-	)?
-	( # optional nightly release indicator
-	\.(?P<nightly>dev[0-9]+) # ex: .dev02142023
-	)? # expected matches: `1.15.0`, `1.5.0a11`, `1.5.0a1.dev123`, `1.5.0.dev123457`, expected failures: `1`, `1.5`, `1.5.2-a1`, `text1.5.0`
-serialize =
-	{major}.{minor}.{patch}{prekind}{num}.{nightly}
-	{major}.{minor}.{patch}.{nightly}
-	{major}.{minor}.{patch}{prekind}{num}
-	{major}.{minor}.{patch}
-commit = False
-tag = False
-
-[bumpversion:part:prekind]
-first_value = a
-optional_value = final
-values =
-	a
-	b
-	rc
-	final
-
-[bumpversion:part:num]
-first_value = 1
-
-[bumpversion:part:nightly]
-
-[bumpversion:file:dbt/adapters/bigquery/__version__.py]
diff --git a/.changes/unreleased/Under the Hood-20241117-194746.yaml b/.changes/unreleased/Under the Hood-20241117-194746.yaml
new file mode 100644
index 000000000..e8658ee20
--- /dev/null
+++ b/.changes/unreleased/Under the Hood-20241117-194746.yaml	
@@ -0,0 +1,6 @@
+kind: Under the Hood
+body: Move from setup.py to pyproject.toml and to hatch as a dev tool
+time: 2024-11-17T19:47:46.341-05:00
+custom:
+  Author: mikealfare
+  Issue: "1407"
diff --git a/scripts/env-setup.sh b/.github/scripts/env-setup.sh
similarity index 100%
rename from scripts/env-setup.sh
rename to .github/scripts/env-setup.sh
diff --git a/.github/scripts/update_dependencies.sh b/.github/scripts/update_dependencies.sh
index c3df48e52..fabdadff2 100644
--- a/.github/scripts/update_dependencies.sh
+++ b/.github/scripts/update_dependencies.sh
@@ -2,9 +2,9 @@
 set -e
 
 git_branch=$1
-target_req_file="dev-requirements.txt"
-core_req_sed_pattern="s|dbt-core.git.*#egg=dbt-core|dbt-core.git@${git_branch}#egg=dbt-core|g"
-tests_req_sed_pattern="s|dbt-core.git.*#egg=dbt-tests|dbt-core.git@${git_branch}#egg=dbt-tests|g"
+target_req_file="hatch.toml"
+core_req_sed_pattern="s|dbt-core.git.*#subdirectory=core|dbt-core.git@${git_branch}#subdirectory=core|g"
+tests_req_sed_pattern="s|dbt-adapters.git.*#subdirectory=dbt-tests-adapter|dbt-adapters.git@${git_branch}#subdirectory=dbt-tests-adapter|g"
 if [[ "$OSTYPE" == darwin* ]]; then
  # mac ships with a different version of sed that requires a delimiter arg
  sed -i "" "$core_req_sed_pattern" $target_req_file
diff --git a/.github/scripts/update_dev_dependency_branches.sh b/.github/scripts/update_dev_dependency_branches.sh
index 022df6a8a..9385cf885 100755
--- a/.github/scripts/update_dev_dependency_branches.sh
+++ b/.github/scripts/update_dev_dependency_branches.sh
@@ -5,8 +5,8 @@ set -e
 dbt_adapters_branch=$1
 dbt_core_branch=$2
 dbt_common_branch=$3
-target_req_file="dev-requirements.txt"
-core_req_sed_pattern="s|dbt-core.git.*#egg=dbt-core|dbt-core.git@${dbt_core_branch}#egg=dbt-core|g"
+target_req_file="hatch.toml"
+core_req_sed_pattern="s|dbt-core.git.*#subdirectory=core|dbt-core.git@${dbt_core_branch}#subdirectory=core|g"
 adapters_req_sed_pattern="s|dbt-adapters.git|dbt-adapters.git@${dbt_adapters_branch}|g"
 common_req_sed_pattern="s|dbt-common.git|dbt-common.git@${dbt_common_branch}|g"
 if [[ "$OSTYPE" == darwin* ]]; then
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index a9179f9ce..32d937ef8 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -127,7 +127,8 @@ jobs:
             bigquery:
               - 'dbt/**'
               - 'tests/**'
-              - 'dev-requirements.txt'
+              - 'hatch.toml'
+              - 'pyproject.toml'
               - '.github/**'
               - '*.py'
 
@@ -164,8 +165,6 @@ jobs:
       matrix: ${{ fromJSON(needs.test-metadata.outputs.matrix) }}
 
     env:
-      TOXENV: integration-${{ matrix.adapter }}
-      PYTEST_ADDOPTS: "-v --color=yes -n4 --csv integration_results.csv"
       DBT_INVOCATION_ENV: github-actions
       DD_CIVISIBILITY_AGENTLESS_ENABLED: true
       DD_API_KEY: ${{ secrets.DATADOG_API_KEY }}
@@ -201,12 +200,7 @@ jobs:
         with:
           python-version: ${{ matrix.python-version }}
 
-      - name: Install python dependencies
-        run: |
-          python -m pip install --user --upgrade pip
-          python -m pip install tox
-          python -m pip --version
-          tox --version
+      - uses: pypa/hatch@install
 
       - name: Update Adapters and Core branches (update dev_requirements.txt)
         if: github.event_name == 'workflow_dispatch'
@@ -215,10 +209,9 @@ jobs:
             ${{ inputs.dbt_adapters_branch }} \
             ${{ inputs.dbt_core_branch }} \
             ${{ inputs.dbt_common_branch }}
-          cat dev-requirements.txt
+          cat hatch.toml
 
-      - name: Run tox (bigquery)
-        if: matrix.adapter == 'bigquery'
+      - run: hatch run integration-tests tests/functional -k "not TestPython"
         env:
           BIGQUERY_TEST_SERVICE_ACCOUNT_JSON: ${{ secrets.BIGQUERY_TEST_SERVICE_ACCOUNT_JSON }}
           BIGQUERY_TEST_ALT_DATABASE: ${{ secrets.BIGQUERY_TEST_ALT_DATABASE }}
@@ -229,36 +222,13 @@ jobs:
           DATAPROC_REGION: us-central1
           DATAPROC_CLUSTER_NAME: dbt-test-1
           GCS_BUCKET: dbt-ci
-        run: tox -- --ddtrace
-
-      - name: Get current date
-        if: always()
-        id: date
-        run: |
-          echo "date=$(date +'%Y-%m-%dT%H_%M_%S')" >> $GITHUB_OUTPUT #no colons allowed for artifacts
-
-      - uses: actions/upload-artifact@v4
-        if: always()
-        with:
-          name: logs_${{ matrix.python-version }}_${{ matrix.os }}_${{ matrix.adapter }}-${{ steps.date.outputs.date }}
-          path: ./logs
-          overwrite: true
-
-      - uses: actions/upload-artifact@v4
-        if: always()
-        with:
-          name: integration_results_${{ matrix.python-version }}_${{ matrix.os }}_${{ matrix.adapter }}-${{ steps.date.outputs.date }}.csv
-          path: integration_results.csv
-          overwrite: true
 
   # python integration tests are slow so we only run them seperately and for a single OS / python version
   test-python:
     name: "test-python"
-    needs: test-metadata
+    needs: test
     runs-on: ubuntu-latest
     if: >-
-      needs.test-metadata.outputs.matrix &&
-      fromJSON( needs.test-metadata.outputs.matrix ).include[0] &&
       (
         github.event_name != 'pull_request_target' ||
         github.event.pull_request.head.repo.full_name == github.repository ||
@@ -286,14 +256,9 @@ jobs:
         with:
           python-version: "3.9"
 
-      - name: Install python dependencies
-        run: |
-          python -m pip install --user --upgrade pip
-          python -m pip install tox
-          python -m pip --version
-          tox --version
+      - uses: pypa/hatch@install
 
-      - name: Run tox (python models)
+      - run: hatch run integration-tests tests/functional -n1 -k "TestPython" --ddtrace
         env:
           BIGQUERY_TEST_SERVICE_ACCOUNT_JSON: ${{ secrets.BIGQUERY_TEST_SERVICE_ACCOUNT_JSON }}
           BIGQUERY_TEST_ALT_DATABASE: ${{ secrets.BIGQUERY_TEST_ALT_DATABASE }}
@@ -304,7 +269,6 @@ jobs:
           DATAPROC_REGION: us-central1
           DATAPROC_CLUSTER_NAME: dbt-test-1
           GCS_BUCKET: dbt-ci
-        run: tox -e python-tests -- --ddtrace
 
   require-label-comment:
     runs-on: ubuntu-latest
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 7b82f3e0f..9ad70ce5c 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -52,15 +52,8 @@ jobs:
         with:
           python-version: '3.9'
 
-      - name: Install python dependencies
-        run: |
-          python -m pip install --user --upgrade pip
-          python -m pip install -r dev-requirements.txt
-          python -m pip --version
-          pre-commit --version
-          dbt --version
-      - name: Run pre-comit hooks
-        run: pre-commit run --all-files --show-diff-on-failure
+      - name: Run pre-commit hooks
+        uses: pre-commit/action@v3.0.1
 
   unit:
     name: unit test / python ${{ matrix.python-version }}
@@ -72,10 +65,6 @@ jobs:
       matrix:
         python-version: ['3.9', '3.10', '3.11', '3.12']
 
-    env:
-      TOXENV: "unit"
-      PYTEST_ADDOPTS: "-v --color=yes --csv unit_results.csv"
-
     steps:
       - name: Check out the repository
         uses: actions/checkout@v4
@@ -88,27 +77,9 @@ jobs:
           python-version: ${{ matrix.python-version }}
 
       - name: Install python dependencies
-        run: |
-          python -m pip install --user --upgrade pip
-          python -m pip install tox
-          python -m pip --version
-          tox --version
-
-      - name: Run tox
-        run: tox
+        uses: pypa/hatch@install
 
-      - name: Get current date
-        if: always()
-        id: date
-        run: |
-          echo "date=$(date +'%Y-%m-%dT%H_%M_%S')" >> $GITHUB_OUTPUT #no colons allowed for artifacts
-
-      - uses: actions/upload-artifact@v4
-        if: always()
-        with:
-          name: unit_results_${{ matrix.python-version }}-${{ steps.date.outputs.date }}.csv
-          path: unit_results.csv
-          overwrite: true
+      - run: hatch run unit-tests
 
   build:
     name: build packages
@@ -129,25 +100,16 @@ jobs:
         with:
           python-version: '3.9'
 
-      - name: Install python dependencies
-        run: |
-          python -m pip install --user --upgrade pip
-          python -m pip install --upgrade setuptools wheel twine check-wheel-contents
-          python -m pip --version
+      - uses: pypa/hatch@install
 
       - name: Build distributions
-        run: ./scripts/build-dist.sh
+        run: hatch build
 
       - name: Show distributions
         run: ls -lh dist/
 
       - name: Check distribution descriptions
-        run: |
-          twine check dist/*
-
-      - name: Check wheel contents
-        run: |
-          check-wheel-contents dist/*.whl --ignore W007,W008
+        run: hatch run build:check-all
 
       - name: Check if this is an alpha version
         id: check-is-alpha
@@ -174,7 +136,7 @@ jobs:
     strategy:
       fail-fast: false
       matrix:
-        os: [ubuntu-latest, macos-14, windows-latest]
+        os: [ubuntu-22.04, macos-14, windows-2022]
         python-version: ['3.9', '3.10', '3.11', '3.12']
         dist-type: ["whl", "gz"]
 
@@ -184,12 +146,6 @@ jobs:
         with:
           python-version: ${{ matrix.python-version }}
 
-      - name: Install python dependencies
-        run: |
-          python -m pip install --user --upgrade pip
-          python -m pip install --upgrade wheel
-          python -m pip --version
-
       - uses: actions/download-artifact@v4
         with:
           name: dist
diff --git a/.github/workflows/nightly-release.yml b/.github/workflows/nightly-release.yml
index 1dc396154..c986a4d0f 100644
--- a/.github/workflows/nightly-release.yml
+++ b/.github/workflows/nightly-release.yml
@@ -50,11 +50,15 @@ jobs:
           commit_sha=$(git rev-parse HEAD)
           echo "release_commit=$commit_sha" >> $GITHUB_OUTPUT
 
-      - name: "Get Current Version Number"
-        id: version-number-sources
-        run: |
-          current_version=`awk -F"current_version = " '{print $2}' .bumpversion.cfg | tr '\n' ' '`
-          echo "current_version=$current_version" >> $GITHUB_OUTPUT
+      - name: Set up Python
+        uses: actions/setup-python@v5
+        with:
+          python-version: '3.9'
+
+      - uses: pypa/hatch@install
+
+      - id: version-number-sources
+        run: echo "current_version=$(hatch version)" >> $GITHUB_OUTPUT
 
       - name: "Audit Version And Parse Into Parts"
         id: semver
@@ -108,10 +112,5 @@ jobs:
       sha: ${{ needs.aggregate-release-data.outputs.commit_sha }}
       target_branch: ${{ needs.aggregate-release-data.outputs.release_branch }}
       version_number: ${{ needs.aggregate-release-data.outputs.version_number }}
-      build_script_path: "scripts/build-dist.sh"
-      env_setup_script_path: "scripts/env-setup.sh"
-      s3_bucket_name: "core-team-artifacts"
-      package_test_command: "dbt -h"
-      test_run: true
       nightly_release: true
     secrets: inherit
diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml
index ad7cf76b4..15840e5ed 100644
--- a/.github/workflows/release.yml
+++ b/.github/workflows/release.yml
@@ -39,7 +39,7 @@ on:
       env_setup_script_path:
         description: "Environment setup script path"
         type: string
-        default: "scripts/env-setup.sh"
+        default: "./.github/scripts/env-setup.sh"
         required: false
       s3_bucket_name:
         description: "AWS S3 bucket name"
@@ -88,7 +88,7 @@ on:
       env_setup_script_path:
         description: "Environment setup script path"
         type: string
-        default: "scripts/env-setup.sh"
+        default: "./.github/scripts/env-setup.sh"
         required: false
       s3_bucket_name:
         description: "AWS S3 bucket name"
@@ -119,86 +119,70 @@ defaults:
     shell: bash
 
 jobs:
-  log-inputs:
-    name: Log Inputs
-    runs-on: ubuntu-latest
-    steps:
-      - name: "[DEBUG] Print Variables"
-        run: |
-          echo The last commit sha in the release: ${{ inputs.sha }}
-          echo The branch to release from:         ${{ inputs.target_branch }}
-          echo The release version number:         ${{ inputs.version_number }}
-          echo Build script path:                  ${{ inputs.build_script_path }}
-          echo Environment setup script path:      ${{ inputs.env_setup_script_path }}
-          echo AWS S3 bucket name:                 ${{ inputs.s3_bucket_name }}
-          echo Package test command:               ${{ inputs.package_test_command }}
-          echo Test run:                           ${{ inputs.test_run }}
-          echo Nightly release:                    ${{ inputs.nightly_release }}
-          echo Only Docker:                        ${{ inputs.only_docker }}
-
-  bump-version-generate-changelog:
-    name: Bump package version, Generate changelog
-    uses: dbt-labs/dbt-release/.github/workflows/release-prep.yml@main
+  release-prep:
+    name: "Release prep: generate changelog, bump version"
+    uses: ./.github/workflows/release_prep_hatch.yml
     with:
-      sha: ${{ inputs.sha }}
-      version_number: ${{ inputs.version_number }}
-      target_branch: ${{ inputs.target_branch }}
-      env_setup_script_path: ${{ inputs.env_setup_script_path }}
-      test_run: ${{ inputs.test_run }}
-      nightly_release: ${{ inputs.nightly_release }}
+      branch: ${{ inputs.branch }}
+      version: ${{ inputs.version }}
+      deploy-to: ${{ inputs.deploy-to }}
     secrets: inherit
 
-  log-outputs-bump-version-generate-changelog:
-    name: "[Log output] Bump package version, Generate changelog"
-    if: ${{ !failure() && !cancelled() && !inputs.only_docker }}
-    needs: [bump-version-generate-changelog]
+  build-release:
+    name: "Build release"
+    needs: release-prep
     runs-on: ubuntu-latest
+    outputs:
+      archive-name: ${{ steps.archive.outputs.name }}
     steps:
-      - name: Print variables
+      - uses: actions/checkout@v4
+        with:
+          ref: ${{ needs.release-prep.outputs.release-branch }}
+          persist-credentials: false
+      - uses: actions/setup-python@v5
+        with:
+          python-version: ${{ inputs.python_version }}
+      - uses: pypa/hatch@install
+      - id: archive
         run: |
-          echo Final SHA     : ${{ needs.bump-version-generate-changelog.outputs.final_sha }}
-          echo Changelog path: ${{ needs.bump-version-generate-changelog.outputs.changelog_path }}
-
-  build-test-package:
-    name: Build, Test, Package
-    if: ${{ !failure() && !cancelled() && !inputs.only_docker }}
-    needs: [bump-version-generate-changelog]
-    uses: dbt-labs/dbt-release/.github/workflows/build.yml@main
-    with:
-      sha: ${{ needs.bump-version-generate-changelog.outputs.final_sha }}
-      version_number: ${{ inputs.version_number }}
-      changelog_path: ${{ needs.bump-version-generate-changelog.outputs.changelog_path }}
-      build_script_path: ${{ inputs.build_script_path }}
-      s3_bucket_name: ${{ inputs.s3_bucket_name }}
-      package_test_command: ${{ inputs.package_test_command }}
-      test_run: ${{ inputs.test_run }}
-      nightly_release: ${{ inputs.nightly_release }}
-    secrets:
-      AWS_ACCESS_KEY_ID: ${{ secrets.AWS_ACCESS_KEY_ID }}
-      AWS_SECRET_ACCESS_KEY: ${{ secrets.AWS_SECRET_ACCESS_KEY }}
+          archive_name=${{ github.event.repository.name }}-${{ inputs.version }}-${{ inputs.deploy-to }}
+          echo "name=$archive_name" >> $GITHUB_OUTPUT
+      - run: hatch build && hatch run build:check-all
+      - uses: actions/upload-artifact@v4
+        with:
+          name: ${{ steps.archive.outputs.name }}
+          path: dist/
+          retention-days: 3
 
   github-release:
     name: GitHub Release
     if: ${{ !failure() && !cancelled() && !inputs.only_docker }}
-    needs: [bump-version-generate-changelog, build-test-package]
-    uses: dbt-labs/dbt-release/.github/workflows/github-release.yml@main
+    needs: [build-release, release-prep]
+    uses: dbt-labs/dbt-adapters/.github/workflows/github-release.yml@main
     with:
       sha: ${{ needs.bump-version-generate-changelog.outputs.final_sha }}
       version_number: ${{ inputs.version_number }}
       changelog_path: ${{ needs.bump-version-generate-changelog.outputs.changelog_path }}
       test_run: ${{ inputs.test_run }}
+      archive_name: ${{ needs.build-release.outputs.archive-name }}
 
   pypi-release:
     name: PyPI Release
     if: ${{ !failure() && !cancelled() && !inputs.only_docker }}
-    needs: [bump-version-generate-changelog, build-test-package]
-    uses: dbt-labs/dbt-release/.github/workflows/pypi-release.yml@main
-    with:
-      version_number: ${{ inputs.version_number }}
-      test_run: ${{ inputs.test_run }}
-    secrets:
-      PYPI_API_TOKEN: ${{ secrets.PYPI_API_TOKEN }}
-      TEST_PYPI_API_TOKEN: ${{ secrets.TEST_PYPI_API_TOKEN }}
+    needs: build-release
+    runs-on: ubuntu-latest
+    environment:
+      name: ${{ inputs.deploy-to }}
+      url: ${{ vars.PYPI_PROJECT_URL }}
+    permissions:
+      # this permission is required for trusted publishing
+      # see https://github.com/marketplace/actions/pypi-publish
+      id-token: write
+    steps:
+      - uses: dbt-labs/dbt-adapters/.github/actions/publish-pypi@main
+        with:
+          repository-url: ${{ vars.PYPI_REPOSITORY_URL }}
+          archive-name: ${{ needs.build-release.outputs.archive-name }}
 
   docker-release:
     name: "Docker Release"
@@ -206,7 +190,7 @@ jobs:
     # what we need to release but draft releases don't actually tag the commit so it
     # finds nothing to release
     if: ${{ !failure() && !cancelled() && (!inputs.test_run || inputs.only_docker) }}
-    needs: [bump-version-generate-changelog, build-test-package, github-release]
+    needs: github-release
     permissions:
       packages: write
     uses: dbt-labs/dbt-release/.github/workflows/release-docker.yml@main
diff --git a/.github/workflows/release_prep_hatch.yml b/.github/workflows/release_prep_hatch.yml
new file mode 100644
index 000000000..ee092600d
--- /dev/null
+++ b/.github/workflows/release_prep_hatch.yml
@@ -0,0 +1,455 @@
+# **what?**
+# Perform the version bump, generate the changelog and run tests.
+#
+# Inputs:
+#  branch:             The branch that we will release from
+#  version:            The release version number (i.e. 1.0.0b1, 1.2.3rc2, 1.0.0)
+#  deploy-to:          If we are deploying to prod or test, if test then release from branch
+#  is-nightly-release: Identifier that this is nightly release
+#
+# Outputs:
+#   release-sha:       The sha that will actually be released.  This can differ from the
+#                      input sha if adding a version bump and/or changelog
+#   changelog-path:    Path to the changelog file (ex .changes/1.2.3-rc1.md)
+#
+# Branching strategy:
+#  - During execution workflow execution the temp branch will be generated.
+#  - For normal runs the temp branch will be removed once changes were merged to target branch;
+#  - For test runs we will keep temp branch and will use it for release;
+#  Naming strategy:
+#  - For normal runs:      prep-release/${{ inputs.deploy-to}}/${{ inputs.version }}_$GITHUB_RUN_ID
+#  - For nightly releases: prep-release/nightly-release/${{ inputs.version }}_$GITHUB_RUN_ID
+#
+# **why?**
+# Reusable and consistent GitHub release process.
+#
+# **when?**
+# Call when ready to kick off a build and release
+#
+# Validation Checks
+#
+#  1. Bump the version if it has not been bumped
+#  2. Generate the changelog (via changie) if there is no markdown file for this version
+name: "Release prep"
+run-name: "Release prep: Generate changelog and bump to ${{ inputs.version }} for release to ${{ inputs.deploy-to }}"
+on:
+  workflow_call:
+    inputs:
+      branch:
+        description: "The branch to release from"
+        type: string
+        default: "main"
+      version:
+        description: "The version to release"
+        required: true
+        type: string
+      deploy-to:
+        description: "Deploy to test or prod"
+        type: string
+        default: "prod"
+      is-nightly-release:
+        description: "Identify if this is a nightly release"
+        type: boolean
+        default: false
+    outputs:
+      release-branch:
+        description: "The branch to be released from"
+        value: ${{ jobs.release.outputs.branch }}
+      release-sha:
+        description: "The SHA to be released"
+        value: ${{ jobs.release.outputs.sha }}
+      changelog-path:
+        description: "The path to the changelog from the repo root for this version, e.g. .changes/1.8.0-b1.md"
+        value: ${{ jobs.release-inputs.outputs.changelog-path }}
+    secrets:
+      FISHTOWN_BOT_PAT:
+        description: "Token to commit/merge changes into branches"
+        required: true
+      IT_TEAM_MEMBERSHIP:
+        description: "Token that can view org level teams"
+        required: true
+
+permissions:
+  contents: write
+
+defaults:
+  run:
+    shell: bash
+
+env:
+  PYTHON_DEFAULT_VERSION: 3.9
+  NOTIFICATION_PREFIX: "[Release Prep]"
+
+jobs:
+  release-inputs:
+    runs-on: ubuntu-latest
+    outputs:
+      changelog-path: ${{ steps.changelog.outputs.path }}
+      changelog-exists: ${{ steps.changelog.outputs.exists }}
+      base-version: ${{ steps.semver.outputs.base-version }}
+      pre-release: ${{ steps.semver.outputs.pre-release }}
+      is-pre-release: ${{ steps.semver.outputs.is-pre-release }}
+      version-is-current: ${{ steps.version.outputs.is-current }}
+
+    steps:
+      - name: "[DEBUG] Log inputs"
+        run: |
+          # WORKFLOW INPUTS
+          echo Branch:               ${{ inputs.branch }}
+          echo Release version:      ${{ inputs.version }}
+          echo Deploy to:            ${{ inputs.deploy-to }}
+          echo Nightly release:      ${{ inputs.is-nightly-release }}
+          # ENVIRONMENT VARIABLES
+          echo Python version:       ${{ env.PYTHON_DEFAULT_VERSION }}
+          echo Notification prefix:  ${{ env.NOTIFICATION_PREFIX }}
+
+      - name: "Checkout ${{ github.event.repository.name }}@${{ inputs.branch }}"
+        uses: actions/checkout@v4
+        with:
+          ref: ${{ inputs.branch }}
+
+      - uses: actions/setup-python@v5
+        with:
+          python-version: ${{ env.PYTHON_DEFAULT_VERSION }}
+
+      - uses: pypa/hatch@install
+
+      - name: "Parse input version"
+        id: semver
+        uses: dbt-labs/actions/parse-semver@v1.1.1
+        with:
+          version: ${{ inputs.version }}
+
+      - name: "Audit version"
+        id: version
+        run: |
+          is_current=false
+          current_version=$(hatch version)
+          if test "$current_version" = "${{ inputs.version }}"
+          then
+            is_current=true
+          fi
+          echo "is-current=$is_current" >> $GITHUB_OUTPUT
+
+      - name: "[INFO] Skip version bump"
+        if: steps.version.outputs.is-current == 'true'
+        run: |
+          title="Skip version bump"
+          message="The version matches the input version ${{ inputs.version }}, skipping version bump"
+          echo "::notice title=${{ env.NOTIFICATION_PREFIX }}: $title::$message"
+
+      - name: "Audit changelog"
+        id: changelog
+        run: |
+          path=".changes/"
+          if [[ ${{ steps.semver.outputs.is-pre-release }} -eq 1 ]]
+          then
+            path+="${{ steps.semver.outputs.base-version }}-${{ steps.semver.outputs.pre-release }}.md"
+          else
+            path+="${{ steps.semver.outputs.base-version }}.md"
+          fi
+          echo "path=$path" >> $GITHUB_OUTPUT
+
+          does_exist=false
+          if test -f $path
+          then
+            does_exist=true
+          fi
+          echo "exists=$does_exist">> $GITHUB_OUTPUT
+
+      - name: "[INFO] Skip changelog generation"
+        if: steps.changelog.outputs.exists == 'true'
+        run: |
+          title="Skip changelog generation"
+          message="A changelog already exists at ${{ steps.changelog.outputs.path }}, skipping generating changelog"
+          echo "::notice title=${{ env.NOTIFICATION_PREFIX }}: $title::$message"
+
+  release-branch:
+    runs-on: ubuntu-latest
+    needs: release-inputs
+    if: |
+        needs.release-inputs.outputs.changelog-exists == 'false' ||
+        needs.release-inputs.outputs.version-is-current == 'false'
+    outputs:
+      name: ${{ steps.release-branch.outputs.name }}
+
+    steps:
+      - name: "Checkout ${{ github.event.repository.name }}@${{ inputs.branch }}"
+        uses: actions/checkout@v4
+        with:
+          ref: ${{ inputs.branch }}
+
+      - name: "Set release branch"
+        id: release-branch
+        run: |
+          name="prep-release/"
+          if [[ ${{ inputs.is-nightly-release }} == true  ]]
+          then
+            name+="nightly-release/"
+          else
+            name+="${{ inputs.deploy-to }}/"
+          fi
+          name+="${{ inputs.version }}_$GITHUB_RUN_ID"
+          echo "name=$name" >> $GITHUB_OUTPUT
+
+      - name: "Create release branch ${{ steps.release-branch.outputs.name }}"
+        run: |
+          git checkout -b ${{ steps.release-branch.outputs.name }}
+          git push -u origin ${{ steps.release-branch.outputs.name }}
+
+      - name: "[INFO] Create release branch"
+        run: |
+          title="Create release branch"
+          message="Create release branch: ${{ steps.release-branch.outputs.name }}"
+          echo "::notice title=${{ env.NOTIFICATION_PREFIX }}: $title::$message"
+
+  core-team:
+    if: needs.release-inputs.outputs.changelog-exists == 'false'
+    needs: release-inputs
+    uses: dbt-labs/actions/.github/workflows/determine-team-membership.yml@main
+    with:
+      github_team: "core-group"
+    secrets: inherit
+
+  generate-changelog:
+    runs-on: ubuntu-latest
+    if: needs.release-inputs.outputs.changelog-exists == 'false'
+    # only runs if we need to make changes, determined by not skipping release-branch
+    needs:
+      - release-inputs
+      - release-branch
+      - core-team
+
+    steps:
+      - name: "Checkout ${{ github.event.repository.name }}@${{ needs.release-branch.outputs.name }}"
+        uses: actions/checkout@v4
+        with:
+          ref: ${{ needs.release-branch.outputs.name }}
+
+      - uses: actions/setup-python@v5
+        with:
+          python-version: ${{ env.PYTHON_DEFAULT_VERSION }}
+
+      - uses: pypa/hatch@install
+
+      - name: "Install `changie`"
+        run: |
+          brew tap miniscruff/changie https://github.com/miniscruff/changie
+          brew install changie
+
+      - name: "Generate changelog at ${{ needs.release-inputs.outputs.changelog-path }}"
+        run: |
+          if [[ ${{ needs.release-inputs.outputs.is-pre-release }} -eq 1 ]]
+          then
+            changie batch ${{ needs.release-inputs.outputs.base-version }} \
+              --move-dir '${{ needs.release-inputs.outputs.base-version }}' \
+              --prerelease ${{ needs.release-inputs.outputs.pre-release }}
+          elif [[ -d ".changes/${{ needs.release-inputs.outputs.base-version }}" ]]
+          then
+            changie batch ${{ needs.release-inputs.outputs.base-version }} \
+              --include '${{ needs.release-inputs.outputs.base-version }}' \
+              --remove-prereleases
+          else # releasing a final patch with no pre-releases
+            changie batch ${{ needs.release-inputs.outputs.base-version }}
+          fi
+          changie merge
+        env:
+          CHANGIE_CORE_TEAM: ${{ needs.core-team.outputs.team_membership }}
+
+      - name: "Remove trailing whitespace and missing new lines"
+        # this step will fail on whitespace errors but also correct them
+        continue-on-error: true
+        run: hatch run code-quality
+
+      - name: "Commit & push changes"
+        run: |
+          git config user.name "$USER"
+          git config user.email "$EMAIL"
+          git pull
+          git add .
+          git commit -m "$COMMIT_MESSAGE"
+          git push
+        env:
+          USER: "GitHub Build Bot"
+          EMAIL: "buildbot@fishtownanalytics.com"
+          COMMIT_MESSAGE: "Generate changelog at ${{ needs.release-inputs.outputs.changelog-path }}"
+
+      - name: "[INFO] Generated changelog at ${{ needs.release-inputs.outputs.changelog-path }}"
+        run: |
+          title="Changelog generation"
+          if [[ -f ${{ needs.release-inputs.outputs.changelog-path }} ]]
+          then
+            message="Generated changelog file successfully"
+            echo "::notice title=${{ env.NOTIFICATION_PREFIX }}: $title::$message"
+          else
+            message="Failed to generate changelog file"
+            echo "::error title=${{ env.NOTIFICATION_PREFIX }}: $title::$message"
+            exit 1
+          fi
+
+  bump-version:
+    runs-on: ubuntu-latest
+    if: needs.release-inputs.outputs.version-is-current == 'false'
+    # only runs if we need to make changes, determined by not skipping release-branch
+    needs:
+      - release-inputs
+      - release-branch
+      - generate-changelog
+
+    steps:
+      - name: "Checkout ${{ github.event.repository.name }}@${{ needs.release-branch.outputs.name }}"
+        uses: actions/checkout@v4
+        with:
+          ref: ${{ needs.release-branch.outputs.name }}
+
+      - uses: actions/setup-python@v5
+        with:
+          python-version: ${{ env.PYTHON_DEFAULT_VERSION }}
+
+      - uses: pypa/hatch@install
+
+      - name: "Bump version to ${{ inputs.version }}"
+        run: hatch version ${{ inputs.version }}
+
+      - name: "Commit & push changes"
+        run: |
+          git config user.name "$USER"
+          git config user.email "$EMAIL"
+          git pull
+          git add .
+          git commit -m "$COMMIT_MESSAGE"
+          git push
+        env:
+          USER: "GitHub Build Bot"
+          EMAIL: "buildbot@fishtownanalytics.com"
+          COMMIT_MESSAGE: "Bump version to ${{ inputs.version }}"
+
+      - name: "[INFO] Bumped version to ${{ inputs.version }}"
+        run: |
+          title="Version bump"
+          message="Bumped version to ${{ inputs.version }}"
+          echo "::notice title=${{ env.NOTIFICATION_PREFIX }}: $title::$message"
+
+  unit-tests:
+    runs-on: ubuntu-latest
+    # only run unit tests if we created a release branch and already bumped the version and generated the changelog
+    if: |
+      !failure() && !cancelled() &&
+      needs.release-branch.outputs.name != ''
+    needs:
+      - release-branch
+      - generate-changelog
+      - bump-version
+
+    steps:
+      - name: "Checkout ${{ github.event.repository.name }}@${{ needs.release-branch.outputs.name }}"
+        uses: actions/checkout@v4
+        with:
+          ref: ${{ needs.release-branch.outputs.name }}
+
+      - uses: actions/setup-python@v5
+        with:
+          python-version: ${{ matrix.python-version }}
+
+      - uses: pypa/hatch@install
+
+      - name: "Run unit tests"
+        run: hatch run unit-tests
+
+  integration-tests:
+    runs-on: ubuntu-latest
+    # only run integration tests if we created a release branch and already bumped the version and generated the changelog
+    if: |
+      !failure() && !cancelled() &&
+      needs.release-branch.outputs.name != ''
+    needs:
+      - release-branch
+      - generate-changelog
+      - bump-version
+
+    steps:
+      - name: "Checkout ${{ github.event.repository.name }}@${{ needs.release-branch.outputs.name }}"
+        uses: actions/checkout@v4
+        with:
+          ref: ${{ needs.release-branch.outputs.name }}
+
+      - uses: actions/setup-python@v5
+        with:
+          python-version: ${{ env.PYTHON_DEFAULT_VERSION }}
+
+      - uses: pypa/hatch@install
+
+      - name: "Run integration tests"
+        run: hatch run integration-tests
+
+  merge-release-branch:
+    runs-on: ubuntu-latest
+    needs:
+      - unit-tests
+      - integration-tests
+      - release-branch
+      - release-inputs
+    if: |
+        !failure() && !cancelled() &&
+        needs.release-branch.result == 'success' &&
+        inputs.deploy-to == 'prod'
+
+    steps:
+      - name: "Checkout ${{ github.event.repository.name }}"
+        uses: actions/checkout@v4
+
+      - name: "Merge changes into ${{ inputs.branch }}"
+        uses: everlytic/branch-merge@1.1.5
+        with:
+          source_ref: ${{ needs.release-branch.outputs.name }}
+          target_branch: ${{ inputs.branch }}
+          github_token: ${{ secrets.FISHTOWN_BOT_PAT }}
+          commit_message_template: "[Automated] Merged {source_ref} into target {target_branch} during release process"
+
+      - name: "[INFO] Merge changes into ${{ inputs.branch }}"
+        run: |
+          title="Merge changes"
+          message="Merge ${{ needs.release-branch.outputs.name }} into ${{ inputs.branch }}"
+          echo "::notice title=${{ env.NOTIFICATION_PREFIX }}: $title::$message"
+
+  release:
+    runs-on: ubuntu-latest
+    needs:
+      - release-branch
+      - merge-release-branch
+    if: ${{ !failure() && !cancelled() }}
+
+    # Get the SHA that will be released.
+    # If the changelog already exists and the version was already current on the input branch, then release from there.
+    # Otherwise, we generated a changelog and/or did the version bump in this workflow and there is a
+    # new sha to use from the merge we just did. Grab that here instead.
+    outputs:
+      branch: ${{ steps.branch.outputs.name }}
+      sha: ${{ steps.sha.outputs.sha }}
+
+    steps:
+      - name: "Set release branch"
+        id: branch
+        # If a release branch was created and not merged, use the release branch
+        # Otherwise, use the input branch because either nothing was done, or the changes were merged back in
+        run: |
+          if [[ ${{ needs.release-branch.result == 'success' }} && ${{ needs.merge-release-branch.result == 'skipped' }} ]]; then
+            branch="${{ needs.release-branch.outputs.name }}"
+          else
+            branch="${{ inputs.branch }}"
+          fi
+          echo "name=$branch" >> $GITHUB_OUTPUT
+
+      - name: "Checkout ${{ github.event.repository.name }}@${{ steps.branch.outputs.name }}"
+        uses: actions/checkout@v4
+        with:
+          ref: ${{ steps.branch.outputs.name }}
+
+      - name: "Set release SHA"
+        id: sha
+        run: echo "sha=$(git rev-parse HEAD)" >> $GITHUB_OUTPUT
+
+      # if this is a real release and a release branch was created, delete it
+      - name: "Delete release branch: ${{ needs.release-branch.outputs.name }}"
+        if: ${{ inputs.deploy-to == 'prod' && inputs.is-nightly-release == 'false' && needs.release-branch.outputs.name != '' }}
+        run: git push origin -d ${{ needs.release-branch.outputs.name }}
diff --git a/.github/workflows/version-bump.yml b/.github/workflows/version-bump.yml
deleted file mode 100644
index bde34d683..000000000
--- a/.github/workflows/version-bump.yml
+++ /dev/null
@@ -1,28 +0,0 @@
-# **what?**
-# This workflow will take the new version number to bump to. With that
-# it will run versionbump to update the version number everywhere in the
-# code base and then run changie to create the corresponding changelog.
-# A PR will be created with the changes that can be reviewed before committing.
-
-# **why?**
-# This is to aid in releasing dbt and making sure we have updated
-# the version in all places and generated the changelog.
-
-# **when?**
-# This is triggered manually
-
-name: Version Bump
-
-on:
-  workflow_dispatch:
-    inputs:
-      version_number:
-       description: 'The version number to bump to (ex. 1.2.0, 1.3.0b1)'
-       required: true
-
-jobs:
-  version_bump_and_changie:
-    uses: dbt-labs/actions/.github/workflows/version-bump.yml@main
-    with:
-      version_number: ${{ inputs.version_number }}
-    secrets: inherit  # ok since what we are calling is internally maintained
diff --git a/MANIFEST.in b/MANIFEST.in
deleted file mode 100644
index cfbc714ed..000000000
--- a/MANIFEST.in
+++ /dev/null
@@ -1 +0,0 @@
-recursive-include dbt/include *.sql *.yml *.md
diff --git a/Makefile b/Makefile
deleted file mode 100644
index bdacb538b..000000000
--- a/Makefile
+++ /dev/null
@@ -1,21 +0,0 @@
-.DEFAULT_GOAL:=help
-
-.PHONY: dev
-dev: ## Installs adapter in develop mode along with development dependencies
-	@\
-	pip install -e . -r dev-requirements.txt && pre-commit install
-
-.PHONY: dev-uninstall
-dev-uninstall: ## Uninstalls all packages while maintaining the virtual environment
-               ## Useful when updating versions, or if you accidentally installed into the system interpreter
-	pip freeze | grep -v "^-e" | cut -d "@" -f1 | xargs pip uninstall -y
-	pip uninstall -y dbt-bigquery
-
-.PHONY: docker-dev
-docker-dev:
-	docker build -f docker/dev.Dockerfile -t dbt-bigquery-dev .
-	docker run --rm -it --name dbt-bigquery-dev -v $(shell pwd):/opt/code dbt-bigquery-dev
-
-.PHONY: docker-prod
-docker-prod:
-	docker build -f docker/Dockerfile -t dbt-bigquery .
diff --git a/dev-requirements.txt b/dev-requirements.txt
deleted file mode 100644
index 2c0134110..000000000
--- a/dev-requirements.txt
+++ /dev/null
@@ -1,20 +0,0 @@
-# install latest changes in dbt-core
-git+https://github.com/dbt-labs/dbt-adapters.git
-git+https://github.com/dbt-labs/dbt-adapters.git#subdirectory=dbt-tests-adapter
-git+https://github.com/dbt-labs/dbt-common.git
-git+https://github.com/dbt-labs/dbt-core.git#egg=dbt-core&subdirectory=core
-
-# dev
-ddtrace==2.3.0
-pre-commit~=3.7.0
-pytest~=7.4
-pytest-csv~=3.0
-pytest-dotenv~=0.5.2
-pytest-logbook~=1.2
-pytest-xdist~=3.6
-tox~=4.11
-
-# build
-bumpversion~=0.6.0
-twine~=5.1
-wheel~=0.43
diff --git a/hatch.toml b/hatch.toml
new file mode 100644
index 000000000..34ba6d2a3
--- /dev/null
+++ b/hatch.toml
@@ -0,0 +1,61 @@
+[version]
+path = "dbt/adapters/bigquery/__version__.py"
+
+[build.targets.sdist]
+packages = ["dbt"]
+
+[build.targets.wheel]
+packages = ["dbt"]
+
+[envs.default]
+dependencies = [
+    "dbt-adapters @ git+https://github.com/dbt-labs/dbt-adapters.git",
+    "dbt-common @ git+https://github.com/dbt-labs/dbt-common.git",
+    "dbt-tests-adapter @ git+https://github.com/dbt-labs/dbt-adapters.git#subdirectory=dbt-tests-adapter",
+    "dbt-core @ git+https://github.com/dbt-labs/dbt-core.git#subdirectory=core",
+    "ddtrace==2.3.0",
+    "ipdb~=0.13.13",
+    "pre-commit==3.7.0",
+    "freezegun",
+    "pytest>=7.0,<8.0",
+    "pytest-csv~=3.0",
+    "pytest-dotenv",
+    "pytest-logbook~=1.2",
+    "pytest-mock",
+    "pytest-xdist",
+]
+
+[envs.default.scripts]
+setup = "pre-commit install"
+code-quality = "pre-commit run --all-files"
+unit-tests = "python -m pytest {args:tests/unit}"
+integration-tests = "python -m pytest --profile service_account {args:tests/functional}"
+docker-dev = [
+    "docker build -f docker/dev.Dockerfile -t dbt-bigquery-dev .",
+	"docker run --rm -it --name dbt-bigquery-dev -v $(shell pwd):/opt/code dbt-bigquery-dev",
+]
+
+[envs.build]
+detached = true
+dependencies = [
+    "wheel",
+    "twine",
+    "check-wheel-contents",
+]
+
+[envs.build.scripts]
+check-all = [
+    "- check-wheel",
+    "- check-sdist",
+]
+check-wheel = [
+    "twine check dist/*",
+    "find ./dist/dbt_bigquery-*.whl -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/",
+    "pip freeze | grep dbt-bigquery",
+]
+check-sdist = [
+    "check-wheel-contents dist/*.whl --ignore W007,W008",
+    "find ./dist/dbt_bigquery-*.gz -maxdepth 1 -type f | xargs python -m pip install --force-reinstall --find-links=dist/",
+    "pip freeze | grep dbt-bigquery",
+]
+docker-prod = "docker build -f docker/Dockerfile -t dbt-bigquery ."
diff --git a/mypy.ini b/mypy.ini
deleted file mode 100644
index 247a47fec..000000000
--- a/mypy.ini
+++ /dev/null
@@ -1,2 +0,0 @@
-[mypy]
-mypy_path = third-party-stubs/
diff --git a/pyproject.toml b/pyproject.toml
new file mode 100644
index 000000000..b2d55b25f
--- /dev/null
+++ b/pyproject.toml
@@ -0,0 +1,57 @@
+[build-system]
+requires = ["hatchling"]
+build-backend = "hatchling.build"
+
+[project]
+dynamic = ["version"]
+name = "dbt-bigquery"
+description = "The BigQuery adapter plugin for dbt"
+readme = "README.md"
+keywords = ["dbt", "adapter", "adapters", "database", "elt", "dbt-core", "dbt Core", "dbt Cloud", "dbt Labs", "bigquery", "google"]
+requires-python = ">=3.9.0"
+authors = [{ name = "dbt Labs", email = "info@dbtlabs.com" }]
+maintainers = [{ name = "dbt Labs", email = "info@dbtlabs.com" }]
+classifiers = [
+    "Development Status :: 5 - Production/Stable",
+    "License :: OSI Approved :: Apache Software License",
+    "Operating System :: MacOS :: MacOS X",
+    "Operating System :: Microsoft :: Windows",
+    "Operating System :: POSIX :: Linux",
+    "Programming Language :: Python :: 3.9",
+    "Programming Language :: Python :: 3.10",
+    "Programming Language :: Python :: 3.11",
+    "Programming Language :: Python :: 3.12",
+]
+dependencies = [
+    "dbt-common>=1.10,<2.0",
+    "dbt-adapters>=1.7,<2.0",
+    # 3.20 introduced pyarrow>=3.0 under the `pandas` extra
+    "google-cloud-bigquery[pandas]>=3.0,<4.0",
+    "google-cloud-storage~=2.4",
+    "google-cloud-dataproc~=5.0",
+    # ----
+    # Expect compatibility with all new versions of these packages, so lower bounds only.
+    "google-api-core>=2.11.0",
+    # add dbt-core to ensure backwards compatibility of installation, this is not a functional dependency
+    "dbt-core>=1.8.0",
+]
+
+[project.urls]
+Homepage = "https://github.com/dbt-labs/dbt-bigquery"
+Documentation = "https://docs.getdbt.com"
+Repository = "https://github.com/dbt-labs/dbt-bigquery.git"
+Issues = "https://github.com/dbt-labs/dbt-bigquery/issues"
+Changelog = "https://github.com/dbt-labs/dbt-bigquery/blob/main/CHANGELOG.md"
+
+[tool.mypy]
+mypy_path = "third-party-stubs/"
+
+[tool.pytest.ini_options]
+testpaths = ["tests/functional", "tests/unit"]
+env_files = ["test.env"]
+addopts = "-v -n auto"
+color = true
+filterwarnings = [
+    "ignore:.*'soft_unicode' has been renamed to 'soft_str'*:DeprecationWarning",
+    "ignore:unclosed file .*:ResourceWarning",
+]
diff --git a/pytest.ini b/pytest.ini
deleted file mode 100644
index b04a6ccf3..000000000
--- a/pytest.ini
+++ /dev/null
@@ -1,10 +0,0 @@
-[pytest]
-filterwarnings =
-    ignore:.*'soft_unicode' has been renamed to 'soft_str'*:DeprecationWarning
-    ignore:unclosed file .*:ResourceWarning
-env_files =
-    test.env
-testpaths =
-    tests/unit
-    tests/integration
-    tests/functional
diff --git a/scripts/build-dist.sh b/scripts/build-dist.sh
deleted file mode 100755
index 3c3808399..000000000
--- a/scripts/build-dist.sh
+++ /dev/null
@@ -1,20 +0,0 @@
-#!/bin/bash
-
-set -eo pipefail
-
-DBT_PATH="$( cd "$(dirname "$0")/.." ; pwd -P )"
-
-PYTHON_BIN=${PYTHON_BIN:-python}
-
-echo "$PYTHON_BIN"
-
-set -x
-
-rm -rf "$DBT_PATH"/dist
-rm -rf "$DBT_PATH"/build
-mkdir -p "$DBT_PATH"/dist
-
-cd "$DBT_PATH"
-$PYTHON_BIN setup.py sdist bdist_wheel
-
-set +x
diff --git a/setup.py b/setup.py
deleted file mode 100644
index 79f6025ea..000000000
--- a/setup.py
+++ /dev/null
@@ -1,78 +0,0 @@
-#!/usr/bin/env python
-import sys
-
-# require a supported version of Python
-if sys.version_info < (3, 9):
-    print("Error: dbt does not support this version of Python.")
-    print("Please upgrade to Python 3.9 or higher.")
-    sys.exit(1)
-
-try:
-    from setuptools import find_namespace_packages
-except ImportError:
-    # the user has a downlevel version of setuptools.
-    print("Error: dbt requires setuptools v40.1.0 or higher.")
-    print('Please upgrade setuptools with "pip install --upgrade setuptools" and try again')
-    sys.exit(1)
-
-from pathlib import Path
-from setuptools import setup
-
-
-# pull the long description from the README
-README = Path(__file__).parent / "README.md"
-
-# used for this adapter's version and in determining the compatible dbt-core version
-VERSION = Path(__file__).parent / "dbt/adapters/bigquery/__version__.py"
-
-
-def _dbt_bigquery_version() -> str:
-    """
-    Pull the package version from the main package version file
-    """
-    attributes = {}
-    exec(VERSION.read_text(), attributes)
-    return attributes["version"]
-
-
-package_name = "dbt-bigquery"
-description = """The BigQuery adapter plugin for dbt"""
-
-setup(
-    name="dbt-bigquery",
-    version=_dbt_bigquery_version(),
-    description="The Bigquery adapter plugin for dbt",
-    long_description=README.read_text(),
-    long_description_content_type="text/markdown",
-    author="dbt Labs",
-    author_email="info@dbtlabs.com",
-    url="https://github.com/dbt-labs/dbt-bigquery",
-    packages=find_namespace_packages(include=["dbt", "dbt.*"]),
-    include_package_data=True,
-    install_requires=[
-        "dbt-common>=1.10,<2.0",
-        "dbt-adapters>=1.7,<2.0",
-        # 3.20 introduced pyarrow>=3.0 under the `pandas` extra
-        "google-cloud-bigquery[pandas]>=3.0,<4.0",
-        "google-cloud-storage~=2.4",
-        "google-cloud-dataproc~=5.0",
-        # ----
-        # Expect compatibility with all new versions of these packages, so lower bounds only.
-        "google-api-core>=2.11.0",
-        # add dbt-core to ensure backwards compatibility of installation, this is not a functional dependency
-        "dbt-core>=1.8.0",
-    ],
-    zip_safe=False,
-    classifiers=[
-        "Development Status :: 5 - Production/Stable",
-        "License :: OSI Approved :: Apache Software License",
-        "Operating System :: Microsoft :: Windows",
-        "Operating System :: MacOS :: MacOS X",
-        "Operating System :: POSIX :: Linux",
-        "Programming Language :: Python :: 3.9",
-        "Programming Language :: Python :: 3.10",
-        "Programming Language :: Python :: 3.11",
-        "Programming Language :: Python :: 3.12",
-    ],
-    python_requires=">=3.9",
-)
diff --git a/tox.ini b/tox.ini
deleted file mode 100644
index 240d85e34..000000000
--- a/tox.ini
+++ /dev/null
@@ -1,54 +0,0 @@
-[tox]
-skipsdist = True
-envlist = py39,py310,py311,py312
-
-[testenv:{unit,py39,py310,py311,py312,py}]
-description = unit testing
-skip_install = true
-passenv =
-    DBT_*
-    PYTEST_ADDOPTS
-commands = {envpython} -m pytest {posargs} tests/unit
-deps =
-  -rdev-requirements.txt
-  -e.
-
-[testenv:{integration,py39,py310,py311,py312,py}-{bigquery}]
-description = adapter plugin integration testing
-skip_install = true
-passenv =
-    DBT_*
-    BIGQUERY_TEST_*
-    PYTEST_ADDOPTS
-    DATAPROC_*
-    GCS_BUCKET
-    DD_CIVISIBILITY_AGENTLESS_ENABLED
-    DD_API_KEY
-    DD_SITE
-    DD_ENV
-    DD_SERVICE
-commands =
-  bigquery: {envpython} -m pytest -n auto {posargs} -vv tests/functional -k "not TestPython" --profile service_account
-deps =
-  -rdev-requirements.txt
-  .
-
-[testenv:{python-tests,py39,py310,py311,py312,py}]
-description = python integration testing
-skip_install = true
-passenv =
-    DBT_*
-    BIGQUERY_TEST_*
-    PYTEST_ADDOPTS
-    DATAPROC_*
-    GCS_BUCKET
-    DD_CIVISIBILITY_AGENTLESS_ENABLED
-    DD_API_KEY
-    DD_SITE
-    DD_ENV
-    DD_SERVICE
-commands =
-  {envpython} -m pytest {posargs} -vv tests/functional -k "TestPython" --profile service_account
-deps =
-  -rdev-requirements.txt
-  -e.

From 2e1a5fdcc7b9cd978660e6e9f030e03075f6fdc5 Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Fri, 13 Dec 2024 18:18:02 -0500
Subject: [PATCH 858/860] [Bug] Use bigquery default retryable exceptions
 (#1431)

* replace our custom list of retryable exceptions with BigQuery's defaults
* remove BadRequest as a retryable error
---
 .../unreleased/Fixes-20241211-144752.yaml     |  6 ++++++
 dbt/adapters/bigquery/retry.py                | 19 ++-----------------
 .../unit/test_bigquery_connection_manager.py  |  6 ++++--
 3 files changed, 12 insertions(+), 19 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20241211-144752.yaml

diff --git a/.changes/unreleased/Fixes-20241211-144752.yaml b/.changes/unreleased/Fixes-20241211-144752.yaml
new file mode 100644
index 000000000..e666d5c31
--- /dev/null
+++ b/.changes/unreleased/Fixes-20241211-144752.yaml
@@ -0,0 +1,6 @@
+kind: Fixes
+body: Fix retry scenarios so that dbt always retries when BigQuery recommends a retry
+time: 2024-12-11T14:47:52.36905-05:00
+custom:
+  Author: mikealfare
+  Issue: "263"
diff --git a/dbt/adapters/bigquery/retry.py b/dbt/adapters/bigquery/retry.py
index 391c00e46..2cbdaa245 100644
--- a/dbt/adapters/bigquery/retry.py
+++ b/dbt/adapters/bigquery/retry.py
@@ -1,10 +1,8 @@
 from typing import Callable, Optional
 
-from google.api_core.exceptions import Forbidden
 from google.api_core.future.polling import DEFAULT_POLLING
 from google.api_core.retry import Retry
-from google.cloud.bigquery.retry import DEFAULT_RETRY
-from google.cloud.exceptions import BadGateway, BadRequest, ServerError
+from google.cloud.bigquery.retry import DEFAULT_RETRY, _job_should_retry
 from requests.exceptions import ConnectionError
 
 from dbt.adapters.contracts.connection import Connection, ConnectionState
@@ -83,7 +81,7 @@ def __call__(self, error: Exception) -> bool:
         self._error_count += 1
 
         # if the error is retryable, and we haven't breached the threshold, log and continue
-        if _is_retryable(error) and self._error_count <= self._retries:
+        if _job_should_retry(error) and self._error_count <= self._retries:
             _logger.debug(
                 f"Retry attempt {self._error_count} of {self._retries} after error: {repr(error)}"
             )
@@ -113,16 +111,3 @@ def on_error(error: Exception):
                 raise FailedToConnectError(str(e))
 
     return on_error
-
-
-def _is_retryable(error: Exception) -> bool:
-    """Return true for errors that are unlikely to occur again if retried."""
-    if isinstance(
-        error, (BadGateway, BadRequest, ConnectionError, ConnectionResetError, ServerError)
-    ):
-        return True
-    elif isinstance(error, Forbidden) and any(
-        e["reason"] == "rateLimitExceeded" for e in error.errors
-    ):
-        return True
-    return False
diff --git a/tests/unit/test_bigquery_connection_manager.py b/tests/unit/test_bigquery_connection_manager.py
index d4c95792e..e7afd692f 100644
--- a/tests/unit/test_bigquery_connection_manager.py
+++ b/tests/unit/test_bigquery_connection_manager.py
@@ -53,7 +53,7 @@ def generate_connection_reset_error():
         assert new_mock_client is not self.mock_client
 
     def test_is_retryable(self):
-        _is_retryable = dbt.adapters.bigquery.retry._is_retryable
+        _is_retryable = google.cloud.bigquery.retry._job_should_retry
         exceptions = dbt.adapters.bigquery.impl.google.cloud.exceptions
         internal_server_error = exceptions.InternalServerError("code broke")
         bad_request_error = exceptions.BadRequest("code broke")
@@ -65,7 +65,9 @@ def test_is_retryable(self):
         service_unavailable_error = exceptions.ServiceUnavailable("service is unavailable")
 
         self.assertTrue(_is_retryable(internal_server_error))
-        self.assertTrue(_is_retryable(bad_request_error))
+        self.assertFalse(
+            _is_retryable(bad_request_error)
+        )  # this was removed after initially being included
         self.assertTrue(_is_retryable(connection_error))
         self.assertFalse(_is_retryable(client_error))
         self.assertTrue(_is_retryable(rate_limit_error))

From a219818c5a38339568bfb4e561405cfe8f6732eb Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Tue, 17 Dec 2024 17:19:56 -0500
Subject: [PATCH 859/860] Retry on 503 (#1408)

* add default retry on all client factories, which includes 502 and 503 errors
* update retries to use defaults and ensure that a timeout or deadline is set
---
 .../unreleased/Fixes-20241120-163101.yaml     |  7 ++++
 dbt/adapters/bigquery/clients.py              | 12 +++----
 dbt/adapters/bigquery/retry.py                | 33 ++++++++++---------
 hatch.toml                                    |  1 +
 4 files changed, 31 insertions(+), 22 deletions(-)
 create mode 100644 .changes/unreleased/Fixes-20241120-163101.yaml

diff --git a/.changes/unreleased/Fixes-20241120-163101.yaml b/.changes/unreleased/Fixes-20241120-163101.yaml
new file mode 100644
index 000000000..ba1f4e937
--- /dev/null
+++ b/.changes/unreleased/Fixes-20241120-163101.yaml
@@ -0,0 +1,7 @@
+kind: Fixes
+body: Fix issue where dbt-bigquery was not retrying in certain retryable scenarios,
+  e.g. 503's
+time: 2024-11-20T16:31:01.60689-05:00
+custom:
+  Author: mikealfare
+  Issue: "682"
diff --git a/dbt/adapters/bigquery/clients.py b/dbt/adapters/bigquery/clients.py
index 18c59fc12..722266240 100644
--- a/dbt/adapters/bigquery/clients.py
+++ b/dbt/adapters/bigquery/clients.py
@@ -1,10 +1,10 @@
 from google.api_core.client_info import ClientInfo
 from google.api_core.client_options import ClientOptions
-from google.api_core.retry import Retry
 from google.auth.exceptions import DefaultCredentialsError
-from google.cloud.bigquery import Client as BigQueryClient
+from google.cloud.bigquery import Client as BigQueryClient, DEFAULT_RETRY as BQ_DEFAULT_RETRY
 from google.cloud.dataproc_v1 import BatchControllerClient, JobControllerClient
 from google.cloud.storage import Client as StorageClient
+from google.cloud.storage.retry import DEFAULT_RETRY as GCS_DEFAULT_RETRY
 
 from dbt.adapters.events.logging import AdapterLogger
 
@@ -28,7 +28,7 @@ def create_bigquery_client(credentials: BigQueryCredentials) -> BigQueryClient:
         return _create_bigquery_client(credentials)
 
 
-@Retry()  # google decorator. retries on transient errors with exponential backoff
+@GCS_DEFAULT_RETRY
 def create_gcs_client(credentials: BigQueryCredentials) -> StorageClient:
     return StorageClient(
         project=credentials.execution_project,
@@ -36,7 +36,7 @@ def create_gcs_client(credentials: BigQueryCredentials) -> StorageClient:
     )
 
 
-@Retry()  # google decorator. retries on transient errors with exponential backoff
+# dataproc does not appear to have a default retry like BQ and GCS
 def create_dataproc_job_controller_client(credentials: BigQueryCredentials) -> JobControllerClient:
     return JobControllerClient(
         credentials=create_google_credentials(credentials),
@@ -44,7 +44,7 @@ def create_dataproc_job_controller_client(credentials: BigQueryCredentials) -> J
     )
 
 
-@Retry()  # google decorator. retries on transient errors with exponential backoff
+# dataproc does not appear to have a default retry like BQ and GCS
 def create_dataproc_batch_controller_client(
     credentials: BigQueryCredentials,
 ) -> BatchControllerClient:
@@ -54,7 +54,7 @@ def create_dataproc_batch_controller_client(
     )
 
 
-@Retry()  # google decorator. retries on transient errors with exponential backoff
+@BQ_DEFAULT_RETRY
 def _create_bigquery_client(credentials: BigQueryCredentials) -> BigQueryClient:
     return BigQueryClient(
         credentials.execution_project,
diff --git a/dbt/adapters/bigquery/retry.py b/dbt/adapters/bigquery/retry.py
index 2cbdaa245..cc197a7d3 100644
--- a/dbt/adapters/bigquery/retry.py
+++ b/dbt/adapters/bigquery/retry.py
@@ -2,7 +2,7 @@
 
 from google.api_core.future.polling import DEFAULT_POLLING
 from google.api_core.retry import Retry
-from google.cloud.bigquery.retry import DEFAULT_RETRY, _job_should_retry
+from google.cloud.bigquery.retry import DEFAULT_JOB_RETRY, _job_should_retry
 from requests.exceptions import ConnectionError
 
 from dbt.adapters.contracts.connection import Connection, ConnectionState
@@ -15,14 +15,8 @@
 
 _logger = AdapterLogger("BigQuery")
 
-
-_SECOND = 1.0
-_MINUTE = 60 * _SECOND
-_HOUR = 60 * _MINUTE
-_DAY = 24 * _HOUR
-_DEFAULT_INITIAL_DELAY = _SECOND
-_DEFAULT_MAXIMUM_DELAY = 3 * _SECOND
-_DEFAULT_POLLING_MAXIMUM_DELAY = 10 * _SECOND
+_MINUTE = 60.0
+_DAY = 24 * 60 * 60.0
 
 
 class RetryFactory:
@@ -44,7 +38,7 @@ def create_job_execution_timeout(self, fallback: float = _DAY) -> float:
         )  # keep _DAY here so it's not overridden by passing fallback=None
 
     def create_retry(self, fallback: Optional[float] = None) -> Retry:
-        return DEFAULT_RETRY.with_timeout(self._job_execution_timeout or fallback or _DAY)
+        return DEFAULT_JOB_RETRY.with_timeout(self._job_execution_timeout or fallback or _DAY)
 
     def create_polling(self, model_timeout: Optional[float] = None) -> Retry:
         return DEFAULT_POLLING.with_timeout(model_timeout or self._job_execution_timeout or _DAY)
@@ -53,14 +47,21 @@ def create_reopen_with_deadline(self, connection: Connection) -> Retry:
         """
         This strategy mimics what was accomplished with _retry_and_handle
         """
-        return Retry(
-            predicate=_DeferredException(self._retries),
-            initial=_DEFAULT_INITIAL_DELAY,
-            maximum=_DEFAULT_MAXIMUM_DELAY,
-            deadline=self._job_deadline,
-            on_error=_create_reopen_on_error(connection),
+
+        retry = DEFAULT_JOB_RETRY.with_delay(maximum=3.0).with_predicate(
+            _DeferredException(self._retries)
         )
 
+        # there is no `with_on_error` method, but we want to retain the defaults on `DEFAULT_JOB_RETRY
+        retry._on_error = _create_reopen_on_error(connection)
+
+        # don't override the default deadline to None if the user did not provide one,
+        # the process will never end
+        if deadline := self._job_deadline:
+            return retry.with_deadline(deadline)
+
+        return retry
+
 
 class _DeferredException:
     """
diff --git a/hatch.toml b/hatch.toml
index 34ba6d2a3..4e0baf9f4 100644
--- a/hatch.toml
+++ b/hatch.toml
@@ -8,6 +8,7 @@ packages = ["dbt"]
 packages = ["dbt"]
 
 [envs.default]
+python = "3.9"
 dependencies = [
     "dbt-adapters @ git+https://github.com/dbt-labs/dbt-adapters.git",
     "dbt-common @ git+https://github.com/dbt-labs/dbt-common.git",

From ed59a9a45749ce4e143038c91325947b8881edcf Mon Sep 17 00:00:00 2001
From: Mike Alfare <13974384+mikealfare@users.noreply.github.com>
Date: Fri, 20 Dec 2024 11:13:43 -0500
Subject: [PATCH 860/860] ADAP-1120: Move dbt-bigquery into a subpackage to
 prepare for migrating to the monorepo (#1434)

* move all the files into a namespace package
* update workflows with the new working directory
* add src to the version file path
---
 .github/workflows/changelog-existence.yml           |   5 ++---
 .github/workflows/integration.yml                   |   5 ++++-
 .github/workflows/main.yml                          |   5 +++++
 .github/workflows/nightly-release.yml               |   1 +
 .github/workflows/release.yml                       |   2 ++
 .github/workflows/release_prep_hatch.yml            |   7 +++++++
 {.changes => dbt-bigquery/.changes}/0.0.0.md        |   0
 {.changes => dbt-bigquery/.changes}/README.md       |   0
 {.changes => dbt-bigquery/.changes}/header.tpl.md   |   0
 .../.changes}/unreleased/.gitkeep                   |   0
 .../unreleased/Features-20241202-223835.yaml        |   0
 .../.changes}/unreleased/Fixes-20241120-163101.yaml |   0
 .../.changes}/unreleased/Fixes-20241204-105846.yaml |   0
 .../.changes}/unreleased/Fixes-20241205-133606.yaml |   0
 .../.changes}/unreleased/Fixes-20241211-144752.yaml |   0
 .../unreleased/Under the Hood-20241117-194746.yaml  |   0
 .changie.yaml => dbt-bigquery/.changie.yaml         |   0
 CHANGELOG.md => dbt-bigquery/CHANGELOG.md           |   0
 CONTRIBUTING.md => dbt-bigquery/CONTRIBUTING.md     |   0
 LICENSE.md => dbt-bigquery/LICENSE.md               |   0
 README.md => dbt-bigquery/README.md                 |   0
 {docker => dbt-bigquery/docker}/Dockerfile          |   0
 {docker => dbt-bigquery/docker}/README.md           |   0
 {docker => dbt-bigquery/docker}/dev.Dockerfile      |   0
 hatch.toml => dbt-bigquery/hatch.toml               |   8 +++++---
 pyproject.toml => dbt-bigquery/pyproject.toml       |   0
 .../scripts/update_dev_dependency_branches.sh       |   0
 {dbt => dbt-bigquery/src/dbt}/__init__.py           |   0
 .../src/dbt}/adapters/bigquery/__init__.py          |   0
 .../src/dbt}/adapters/bigquery/__version__.py       |   0
 .../src/dbt}/adapters/bigquery/clients.py           |   0
 .../src/dbt}/adapters/bigquery/column.py            |   0
 .../src/dbt}/adapters/bigquery/connections.py       |   0
 .../src/dbt}/adapters/bigquery/credentials.py       |   0
 .../src/dbt}/adapters/bigquery/dataset.py           |   0
 .../src/dbt}/adapters/bigquery/impl.py              |   0
 .../dbt}/adapters/bigquery/python_submissions.py    |   0
 .../src/dbt}/adapters/bigquery/relation.py          |   0
 .../adapters/bigquery/relation_configs/__init__.py  |   0
 .../adapters/bigquery/relation_configs/_base.py     |   0
 .../adapters/bigquery/relation_configs/_cluster.py  |   0
 .../bigquery/relation_configs/_materialized_view.py |   0
 .../adapters/bigquery/relation_configs/_options.py  |   0
 .../bigquery/relation_configs/_partition.py         |   0
 .../adapters/bigquery/relation_configs/_policies.py |   0
 .../src/dbt}/adapters/bigquery/retry.py             |   0
 .../src/dbt}/adapters/bigquery/utility.py           |   0
 .../src/dbt}/include/bigquery/__init__.py           |   0
 .../src/dbt}/include/bigquery/dbt_project.yml       |   0
 .../src/dbt}/include/bigquery/macros/adapters.sql   |   0
 .../bigquery/macros/adapters/apply_grants.sql       |   0
 .../include/bigquery/macros/adapters/columns.sql    |   0
 .../include/bigquery/macros/catalog/by_relation.sql |   0
 .../include/bigquery/macros/catalog/by_schema.sql   |   0
 .../include/bigquery/macros/catalog/catalog.sql     |   0
 .../src/dbt}/include/bigquery/macros/etc.sql        |   0
 .../bigquery/macros/materializations/clone.sql      |   0
 .../bigquery/macros/materializations/copy.sql       |   0
 .../macros/materializations/incremental.sql         |   0
 .../incremental_strategy/common.sql                 |   0
 .../incremental_strategy/insert_overwrite.sql       |   0
 .../materializations/incremental_strategy/merge.sql |   0
 .../incremental_strategy/microbatch.sql             |   0
 .../incremental_strategy/time_ingestion_tables.sql  |   0
 .../bigquery/macros/materializations/seed.sql       |   0
 .../bigquery/macros/materializations/snapshot.sql   |   0
 .../bigquery/macros/materializations/table.sql      |   0
 .../bigquery/macros/materializations/view.sql       |   0
 .../include/bigquery/macros/python_model/python.sql |   0
 .../include/bigquery/macros/relations/cluster.sql   |   0
 .../dbt}/include/bigquery/macros/relations/drop.sql |   0
 .../macros/relations/materialized_view/alter.sql    |   0
 .../macros/relations/materialized_view/create.sql   |   0
 .../macros/relations/materialized_view/drop.sql     |   0
 .../macros/relations/materialized_view/refresh.sql  |   0
 .../macros/relations/materialized_view/replace.sql  |   0
 .../include/bigquery/macros/relations/options.sql   |   0
 .../include/bigquery/macros/relations/partition.sql |   0
 .../include/bigquery/macros/relations/rename.sql    |   0
 .../bigquery/macros/relations/table/drop.sql        |   0
 .../bigquery/macros/relations/table/options.sql     |   0
 .../bigquery/macros/relations/table/rename.sql      |   0
 .../include/bigquery/macros/relations/view/drop.sql |   0
 .../bigquery/macros/relations/view/options.sql      |   0
 .../bigquery/macros/relations/view/rename.sql       |   0
 .../bigquery/macros/relations/view/replace.sql      |   0
 .../include/bigquery/macros/utils/array_append.sql  |   0
 .../include/bigquery/macros/utils/array_concat.sql  |   0
 .../bigquery/macros/utils/array_construct.sql       |   0
 .../dbt}/include/bigquery/macros/utils/bool_or.sql  |   0
 .../src/dbt}/include/bigquery/macros/utils/date.sql |   0
 .../include/bigquery/macros/utils/date_trunc.sql    |   0
 .../dbt}/include/bigquery/macros/utils/dateadd.sql  |   0
 .../dbt}/include/bigquery/macros/utils/datediff.sql |   0
 .../bigquery/macros/utils/escape_single_quotes.sql  |   0
 .../dbt}/include/bigquery/macros/utils/except.sql   |   0
 .../bigquery/macros/utils/get_columns_spec_ddl.sql  |   0
 .../src/dbt}/include/bigquery/macros/utils/hash.sql |   0
 .../include/bigquery/macros/utils/intersect.sql     |   0
 .../dbt}/include/bigquery/macros/utils/listagg.sql  |   0
 .../dbt}/include/bigquery/macros/utils/position.sql |   0
 .../dbt}/include/bigquery/macros/utils/right.sql    |   0
 .../include/bigquery/macros/utils/safe_cast.sql     |   0
 .../include/bigquery/macros/utils/split_part.sql    |   0
 .../bigquery/macros/utils/string_literal.sql        |   0
 .../include/bigquery/macros/utils/timestamps.sql    |   0
 .../src/dbt}/include/bigquery/profile_template.yml  |   0
 test.env.example => dbt-bigquery/test.env.example   |   0
 {tests => dbt-bigquery/tests}/__init__.py           |   0
 .../tests}/boundary/test_bigquery_sdk.py            |   0
 {tests => dbt-bigquery/tests}/conftest.py           |   0
 .../functional/adapter/catalog_tests/files.py       |   0
 .../adapter/catalog_tests/test_relation_types.py    |   0
 .../functional/adapter/column_types/fixtures.py     |   0
 .../adapter/column_types/test_alter_column_types.py |   0
 .../adapter/column_types/test_column_types.py       |   0
 .../functional/adapter/constraints/fixtures.py      |   0
 .../adapter/constraints/test_constraints.py         |   0
 .../functional/adapter/dbt_clone/test_dbt_clone.py  |   0
 .../functional/adapter/dbt_show/test_dbt_show.py    |   0
 .../functional/adapter/describe_relation/_files.py  |   0
 .../describe_relation/test_describe_relation.py     |   0
 .../tests}/functional/adapter/empty/test_empty.py   |   0
 .../tests}/functional/adapter/expected_stats.py     |   0
 .../functional/adapter/hooks/data/seed_model.sql    |   0
 .../functional/adapter/hooks/data/seed_run.sql      |   0
 .../functional/adapter/hooks/test_model_hooks.py    |   0
 .../functional/adapter/hooks/test_run_hooks.py      |   0
 .../incremental/incremental_strategy_fixtures.py    |   0
 .../tests}/functional/adapter/incremental/seeds.py  |   0
 .../test_incremental_merge_exclude_columns.py       |   0
 .../incremental/test_incremental_microbatch.py      |   0
 .../test_incremental_on_schema_change.py            |   0
 .../incremental/test_incremental_predicates.py      |   0
 .../incremental/test_incremental_strategies.py      |   0
 .../incremental/test_incremental_unique_id.py       |   0
 .../adapter/materialized_view_tests/_files.py       |   0
 .../adapter/materialized_view_tests/_mixin.py       |   0
 .../test_materialized_view.py                       |   0
 .../test_materialized_view_changes.py               |   0
 .../test_materialized_view_cluster_changes.py       |   0
 .../test_materialized_view_partition_changes.py     |   0
 .../adapter/query_comment_test/test_job_label.py    |   0
 .../query_comment_test/test_query_comment.py        |   0
 .../adapter/simple_bigquery_view/fixtures.py        |   0
 .../adapter/simple_bigquery_view/seeds.py           |   0
 .../test_simple_bigquery_view.py                    |   0
 .../functional/adapter/simple_copy/fixtures.py      |   0
 .../adapter/simple_copy/test_simple_copy.py         |   0
 .../adapter/sources_freshness_tests/files.py        |   0
 .../test_get_relation_last_modified.py              |   0
 .../test_store_test_failures.py                     |   0
 .../tests}/functional/adapter/test_aliases.py       |   0
 .../tests}/functional/adapter/test_basic.py         |   0
 .../adapter/test_changing_relation_type.py          |   0
 .../functional/adapter/test_copy_materialization.py |   0
 .../tests}/functional/adapter/test_dbt_debug.py     |   0
 .../functional/adapter/test_grant_access_to.py      |   0
 .../tests}/functional/adapter/test_grants.py        |   0
 .../tests}/functional/adapter/test_json_keyfile.py  |   0
 .../tests}/functional/adapter/test_persist_docs.py  |   0
 .../tests}/functional/adapter/test_python_model.py  |   0
 .../tests}/functional/adapter/test_simple_seed.py   |   0
 .../functional/adapter/test_simple_snaphot.py       |   0
 .../functional/adapter/test_string_literal_macro.py |   0
 .../adapter/unit_testing/test_unit_testing.py       |   0
 .../adapter/upload_file_tests/data/csv/source.csv   |   0
 .../upload_file_tests/data/ndjson/source.ndjson     |   0
 .../upload_file_tests/data/parquet/source.parquet   | Bin
 .../adapter/upload_file_tests/test_upload_file.py   |   0
 .../adapter/utils/fixture_array_append.py           |   0
 .../adapter/utils/fixture_array_concat.py           |   0
 .../adapter/utils/fixture_array_construct.py        |   0
 .../adapter/utils/fixture_get_intervals_between.py  |   0
 .../functional/adapter/utils/test_data_types.py     |   0
 .../functional/adapter/utils/test_timestamps.py     |   0
 .../tests}/functional/adapter/utils/test_utils.py   |   0
 .../functional/python_model_tests/__init__.py       |   0
 .../tests}/functional/python_model_tests/files.py   |   0
 .../python_model_tests/test_list_inference.py       |   0
 .../tests}/functional/test_cancel.py                |   0
 .../tests}/functional/test_changing_partitions.py   |   0
 .../tests}/functional/test_delete_column_policy.py  |   0
 .../tests}/functional/test_drop_temp_relation.py    |   0
 .../test_get_columns_incomplete_database.py         |   0
 .../tests}/functional/test_hours_to_expiration.py   |   0
 .../functional/test_incremental_materialization.py  |   0
 .../tests}/functional/test_job_timeout.py           |   0
 .../tests}/functional/test_location_change.py       |   0
 .../functional/test_override_database/fixtures.py   |   0
 .../test_override_database.py                       |   0
 .../tests}/functional/test_quota_project.py         |   0
 .../tests}/functional/test_update_column_policy.py  |   0
 .../functional/test_update_field_description.py     |   0
 {tests => dbt-bigquery/tests}/unit/__init__.py      |   0
 {tests => dbt-bigquery/tests}/unit/mock_adapter.py  |   0
 .../tests}/unit/test_bigquery_adapter.py            |   0
 .../tests}/unit/test_bigquery_connection_manager.py |   0
 {tests => dbt-bigquery/tests}/unit/test_column.py   |   0
 .../tests}/unit/test_configure_dataproc_batch.py    |   0
 {tests => dbt-bigquery/tests}/unit/test_dataset.py  |   0
 .../tests}/unit/test_renamed_relations.py           |   0
 {tests => dbt-bigquery/tests}/unit/utils.py         |   0
 .../third-party-stubs}/agate/__init__.pyi           |   0
 .../third-party-stubs}/agate/data_types.pyi         |   0
 205 files changed, 26 insertions(+), 7 deletions(-)
 rename {.changes => dbt-bigquery/.changes}/0.0.0.md (100%)
 rename {.changes => dbt-bigquery/.changes}/README.md (100%)
 rename {.changes => dbt-bigquery/.changes}/header.tpl.md (100%)
 rename {.changes => dbt-bigquery/.changes}/unreleased/.gitkeep (100%)
 rename {.changes => dbt-bigquery/.changes}/unreleased/Features-20241202-223835.yaml (100%)
 rename {.changes => dbt-bigquery/.changes}/unreleased/Fixes-20241120-163101.yaml (100%)
 rename {.changes => dbt-bigquery/.changes}/unreleased/Fixes-20241204-105846.yaml (100%)
 rename {.changes => dbt-bigquery/.changes}/unreleased/Fixes-20241205-133606.yaml (100%)
 rename {.changes => dbt-bigquery/.changes}/unreleased/Fixes-20241211-144752.yaml (100%)
 rename {.changes => dbt-bigquery/.changes}/unreleased/Under the Hood-20241117-194746.yaml (100%)
 rename .changie.yaml => dbt-bigquery/.changie.yaml (100%)
 rename CHANGELOG.md => dbt-bigquery/CHANGELOG.md (100%)
 rename CONTRIBUTING.md => dbt-bigquery/CONTRIBUTING.md (100%)
 rename LICENSE.md => dbt-bigquery/LICENSE.md (100%)
 rename README.md => dbt-bigquery/README.md (100%)
 rename {docker => dbt-bigquery/docker}/Dockerfile (100%)
 rename {docker => dbt-bigquery/docker}/README.md (100%)
 rename {docker => dbt-bigquery/docker}/dev.Dockerfile (100%)
 rename hatch.toml => dbt-bigquery/hatch.toml (93%)
 rename pyproject.toml => dbt-bigquery/pyproject.toml (100%)
 rename {.github => dbt-bigquery}/scripts/update_dev_dependency_branches.sh (100%)
 rename {dbt => dbt-bigquery/src/dbt}/__init__.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/__init__.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/__version__.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/clients.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/column.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/connections.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/credentials.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/dataset.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/impl.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/python_submissions.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/relation.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/relation_configs/__init__.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/relation_configs/_base.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/relation_configs/_cluster.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/relation_configs/_materialized_view.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/relation_configs/_options.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/relation_configs/_partition.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/relation_configs/_policies.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/retry.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/adapters/bigquery/utility.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/__init__.py (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/dbt_project.yml (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/adapters.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/adapters/apply_grants.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/adapters/columns.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/catalog/by_relation.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/catalog/by_schema.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/catalog/catalog.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/etc.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/materializations/clone.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/materializations/copy.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/materializations/incremental.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/materializations/incremental_strategy/common.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/materializations/incremental_strategy/merge.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/materializations/incremental_strategy/microbatch.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/materializations/seed.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/materializations/snapshot.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/materializations/table.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/materializations/view.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/python_model/python.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/cluster.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/drop.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/materialized_view/alter.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/materialized_view/create.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/materialized_view/drop.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/materialized_view/refresh.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/materialized_view/replace.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/options.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/partition.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/rename.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/table/drop.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/table/options.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/table/rename.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/view/drop.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/view/options.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/view/rename.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/relations/view/replace.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/array_append.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/array_concat.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/array_construct.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/bool_or.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/date.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/date_trunc.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/dateadd.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/datediff.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/escape_single_quotes.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/except.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/get_columns_spec_ddl.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/hash.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/intersect.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/listagg.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/position.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/right.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/safe_cast.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/split_part.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/string_literal.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/macros/utils/timestamps.sql (100%)
 rename {dbt => dbt-bigquery/src/dbt}/include/bigquery/profile_template.yml (100%)
 rename test.env.example => dbt-bigquery/test.env.example (100%)
 rename {tests => dbt-bigquery/tests}/__init__.py (100%)
 rename {tests => dbt-bigquery/tests}/boundary/test_bigquery_sdk.py (100%)
 rename {tests => dbt-bigquery/tests}/conftest.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/catalog_tests/files.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/catalog_tests/test_relation_types.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/column_types/fixtures.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/column_types/test_alter_column_types.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/column_types/test_column_types.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/constraints/fixtures.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/constraints/test_constraints.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/dbt_clone/test_dbt_clone.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/dbt_show/test_dbt_show.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/describe_relation/_files.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/describe_relation/test_describe_relation.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/empty/test_empty.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/expected_stats.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/hooks/data/seed_model.sql (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/hooks/data/seed_run.sql (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/hooks/test_model_hooks.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/hooks/test_run_hooks.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/incremental/incremental_strategy_fixtures.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/incremental/seeds.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/incremental/test_incremental_merge_exclude_columns.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/incremental/test_incremental_microbatch.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/incremental/test_incremental_on_schema_change.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/incremental/test_incremental_predicates.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/incremental/test_incremental_strategies.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/incremental/test_incremental_unique_id.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/materialized_view_tests/_files.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/materialized_view_tests/_mixin.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/materialized_view_tests/test_materialized_view.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/materialized_view_tests/test_materialized_view_changes.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/materialized_view_tests/test_materialized_view_cluster_changes.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/materialized_view_tests/test_materialized_view_partition_changes.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/query_comment_test/test_job_label.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/query_comment_test/test_query_comment.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/simple_bigquery_view/fixtures.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/simple_bigquery_view/seeds.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/simple_bigquery_view/test_simple_bigquery_view.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/simple_copy/fixtures.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/simple_copy/test_simple_copy.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/sources_freshness_tests/files.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/sources_freshness_tests/test_get_relation_last_modified.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/store_test_failures_tests/test_store_test_failures.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_aliases.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_basic.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_changing_relation_type.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_copy_materialization.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_dbt_debug.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_grant_access_to.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_grants.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_json_keyfile.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_persist_docs.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_python_model.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_simple_seed.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_simple_snaphot.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/test_string_literal_macro.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/unit_testing/test_unit_testing.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/upload_file_tests/data/csv/source.csv (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/upload_file_tests/data/ndjson/source.ndjson (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/upload_file_tests/data/parquet/source.parquet (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/upload_file_tests/test_upload_file.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/utils/fixture_array_append.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/utils/fixture_array_concat.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/utils/fixture_array_construct.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/utils/fixture_get_intervals_between.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/utils/test_data_types.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/utils/test_timestamps.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/adapter/utils/test_utils.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/python_model_tests/__init__.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/python_model_tests/files.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/python_model_tests/test_list_inference.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_cancel.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_changing_partitions.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_delete_column_policy.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_drop_temp_relation.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_get_columns_incomplete_database.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_hours_to_expiration.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_incremental_materialization.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_job_timeout.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_location_change.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_override_database/fixtures.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_override_database/test_override_database.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_quota_project.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_update_column_policy.py (100%)
 rename {tests => dbt-bigquery/tests}/functional/test_update_field_description.py (100%)
 rename {tests => dbt-bigquery/tests}/unit/__init__.py (100%)
 rename {tests => dbt-bigquery/tests}/unit/mock_adapter.py (100%)
 rename {tests => dbt-bigquery/tests}/unit/test_bigquery_adapter.py (100%)
 rename {tests => dbt-bigquery/tests}/unit/test_bigquery_connection_manager.py (100%)
 rename {tests => dbt-bigquery/tests}/unit/test_column.py (100%)
 rename {tests => dbt-bigquery/tests}/unit/test_configure_dataproc_batch.py (100%)
 rename {tests => dbt-bigquery/tests}/unit/test_dataset.py (100%)
 rename {tests => dbt-bigquery/tests}/unit/test_renamed_relations.py (100%)
 rename {tests => dbt-bigquery/tests}/unit/utils.py (100%)
 rename {third-party-stubs => dbt-bigquery/third-party-stubs}/agate/__init__.pyi (100%)
 rename {third-party-stubs => dbt-bigquery/third-party-stubs}/agate/data_types.pyi (100%)

diff --git a/.github/workflows/changelog-existence.yml b/.github/workflows/changelog-existence.yml
index 02bce4582..19fa8e0fc 100644
--- a/.github/workflows/changelog-existence.yml
+++ b/.github/workflows/changelog-existence.yml
@@ -34,8 +34,7 @@ permissions:
 
 jobs:
   changelog:
-    uses: dbt-labs/actions/.github/workflows/changelog-existence.yml@main
+    uses: dbt-labs/dbt-adapters/.github/workflows/_changelog-entry-check.yml@main
     with:
-      changelog_comment: 'Thank you for your pull request! We could not find a changelog entry for this change. For details on how to document a change, see the [dbt-bigquery contributing guide](https://github.com/dbt-labs/dbt-bigquery/blob/main/CONTRIBUTING.md).'
-      skip_label: 'Skip Changelog'
+      pull-request: ${{ github.event.pull_request.number }}
     secrets: inherit # this is only acceptable because we own the action we're calling
diff --git a/.github/workflows/integration.yml b/.github/workflows/integration.yml
index 32d937ef8..1474ee0e0 100644
--- a/.github/workflows/integration.yml
+++ b/.github/workflows/integration.yml
@@ -205,11 +205,12 @@ jobs:
       - name: Update Adapters and Core branches (update dev_requirements.txt)
         if: github.event_name == 'workflow_dispatch'
         run: |
-          ./.github/scripts/update_dev_dependency_branches.sh \
+          scripts/update_dev_dependency_branches.sh \
             ${{ inputs.dbt_adapters_branch }} \
             ${{ inputs.dbt_core_branch }} \
             ${{ inputs.dbt_common_branch }}
           cat hatch.toml
+        working-directory: ./dbt-bigquery
 
       - run: hatch run integration-tests tests/functional -k "not TestPython"
         env:
@@ -222,6 +223,7 @@ jobs:
           DATAPROC_REGION: us-central1
           DATAPROC_CLUSTER_NAME: dbt-test-1
           GCS_BUCKET: dbt-ci
+        working-directory: ./dbt-bigquery
 
   # python integration tests are slow so we only run them seperately and for a single OS / python version
   test-python:
@@ -269,6 +271,7 @@ jobs:
           DATAPROC_REGION: us-central1
           DATAPROC_CLUSTER_NAME: dbt-test-1
           GCS_BUCKET: dbt-ci
+        working-directory: ./dbt-bigquery
 
   require-label-comment:
     runs-on: ubuntu-latest
diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml
index 9ad70ce5c..7f0523771 100644
--- a/.github/workflows/main.yml
+++ b/.github/workflows/main.yml
@@ -80,6 +80,7 @@ jobs:
         uses: pypa/hatch@install
 
       - run: hatch run unit-tests
+        working-directory: ./dbt-bigquery
 
   build:
     name: build packages
@@ -104,12 +105,15 @@ jobs:
 
       - name: Build distributions
         run: hatch build
+        working-directory: ./dbt-bigquery
 
       - name: Show distributions
         run: ls -lh dist/
+        working-directory: ./dbt-bigquery
 
       - name: Check distribution descriptions
         run: hatch run build:check-all
+        working-directory: ./dbt-bigquery
 
       - name: Check if this is an alpha version
         id: check-is-alpha
@@ -117,6 +121,7 @@ jobs:
           export is_alpha=0
           if [[ "$(ls -lh dist/)" == *"a1"* ]]; then export is_alpha=1; fi
           echo "is_alpha=$is_alpha" >> $GITHUB_OUTPUT
+        working-directory: ./dbt-bigquery
 
       - uses: actions/upload-artifact@v4
         with:
diff --git a/.github/workflows/nightly-release.yml b/.github/workflows/nightly-release.yml
index c986a4d0f..fad21907c 100644
--- a/.github/workflows/nightly-release.yml
+++ b/.github/workflows/nightly-release.yml
@@ -59,6 +59,7 @@ jobs:
 
       - id: version-number-sources
         run: echo "current_version=$(hatch version)" >> $GITHUB_OUTPUT
+        working-directory: ./dbt-bigquery
 
       - name: "Audit Version And Parse Into Parts"
         id: semver
diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml
index 15840e5ed..c09e05277 100644
--- a/.github/workflows/release.yml
+++ b/.github/workflows/release.yml
@@ -148,11 +148,13 @@ jobs:
           archive_name=${{ github.event.repository.name }}-${{ inputs.version }}-${{ inputs.deploy-to }}
           echo "name=$archive_name" >> $GITHUB_OUTPUT
       - run: hatch build && hatch run build:check-all
+        working-directory: ./dbt-bigquery
       - uses: actions/upload-artifact@v4
         with:
           name: ${{ steps.archive.outputs.name }}
           path: dist/
           retention-days: 3
+        working-directory: ./dbt-bigquery
 
   github-release:
     name: GitHub Release
diff --git a/.github/workflows/release_prep_hatch.yml b/.github/workflows/release_prep_hatch.yml
index ee092600d..a83663ae1 100644
--- a/.github/workflows/release_prep_hatch.yml
+++ b/.github/workflows/release_prep_hatch.yml
@@ -130,6 +130,7 @@ jobs:
             is_current=true
           fi
           echo "is-current=$is_current" >> $GITHUB_OUTPUT
+        working-directory: ./dbt-bigquery
 
       - name: "[INFO] Skip version bump"
         if: steps.version.outputs.is-current == 'true'
@@ -156,6 +157,7 @@ jobs:
             does_exist=true
           fi
           echo "exists=$does_exist">> $GITHUB_OUTPUT
+        working-directory: ./dbt-bigquery
 
       - name: "[INFO] Skip changelog generation"
         if: steps.changelog.outputs.exists == 'true'
@@ -255,11 +257,13 @@ jobs:
           changie merge
         env:
           CHANGIE_CORE_TEAM: ${{ needs.core-team.outputs.team_membership }}
+        working-directory: ./dbt-bigquery
 
       - name: "Remove trailing whitespace and missing new lines"
         # this step will fail on whitespace errors but also correct them
         continue-on-error: true
         run: hatch run code-quality
+        working-directory: ./dbt-bigquery
 
       - name: "Commit & push changes"
         run: |
@@ -310,6 +314,7 @@ jobs:
 
       - name: "Bump version to ${{ inputs.version }}"
         run: hatch version ${{ inputs.version }}
+        working-directory: ./dbt-bigquery
 
       - name: "Commit & push changes"
         run: |
@@ -355,6 +360,7 @@ jobs:
 
       - name: "Run unit tests"
         run: hatch run unit-tests
+        working-directory: ./dbt-bigquery
 
   integration-tests:
     runs-on: ubuntu-latest
@@ -381,6 +387,7 @@ jobs:
 
       - name: "Run integration tests"
         run: hatch run integration-tests
+        working-directory: ./dbt-bigquery
 
   merge-release-branch:
     runs-on: ubuntu-latest
diff --git a/.changes/0.0.0.md b/dbt-bigquery/.changes/0.0.0.md
similarity index 100%
rename from .changes/0.0.0.md
rename to dbt-bigquery/.changes/0.0.0.md
diff --git a/.changes/README.md b/dbt-bigquery/.changes/README.md
similarity index 100%
rename from .changes/README.md
rename to dbt-bigquery/.changes/README.md
diff --git a/.changes/header.tpl.md b/dbt-bigquery/.changes/header.tpl.md
similarity index 100%
rename from .changes/header.tpl.md
rename to dbt-bigquery/.changes/header.tpl.md
diff --git a/.changes/unreleased/.gitkeep b/dbt-bigquery/.changes/unreleased/.gitkeep
similarity index 100%
rename from .changes/unreleased/.gitkeep
rename to dbt-bigquery/.changes/unreleased/.gitkeep
diff --git a/.changes/unreleased/Features-20241202-223835.yaml b/dbt-bigquery/.changes/unreleased/Features-20241202-223835.yaml
similarity index 100%
rename from .changes/unreleased/Features-20241202-223835.yaml
rename to dbt-bigquery/.changes/unreleased/Features-20241202-223835.yaml
diff --git a/.changes/unreleased/Fixes-20241120-163101.yaml b/dbt-bigquery/.changes/unreleased/Fixes-20241120-163101.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20241120-163101.yaml
rename to dbt-bigquery/.changes/unreleased/Fixes-20241120-163101.yaml
diff --git a/.changes/unreleased/Fixes-20241204-105846.yaml b/dbt-bigquery/.changes/unreleased/Fixes-20241204-105846.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20241204-105846.yaml
rename to dbt-bigquery/.changes/unreleased/Fixes-20241204-105846.yaml
diff --git a/.changes/unreleased/Fixes-20241205-133606.yaml b/dbt-bigquery/.changes/unreleased/Fixes-20241205-133606.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20241205-133606.yaml
rename to dbt-bigquery/.changes/unreleased/Fixes-20241205-133606.yaml
diff --git a/.changes/unreleased/Fixes-20241211-144752.yaml b/dbt-bigquery/.changes/unreleased/Fixes-20241211-144752.yaml
similarity index 100%
rename from .changes/unreleased/Fixes-20241211-144752.yaml
rename to dbt-bigquery/.changes/unreleased/Fixes-20241211-144752.yaml
diff --git a/.changes/unreleased/Under the Hood-20241117-194746.yaml b/dbt-bigquery/.changes/unreleased/Under the Hood-20241117-194746.yaml
similarity index 100%
rename from .changes/unreleased/Under the Hood-20241117-194746.yaml
rename to dbt-bigquery/.changes/unreleased/Under the Hood-20241117-194746.yaml
diff --git a/.changie.yaml b/dbt-bigquery/.changie.yaml
similarity index 100%
rename from .changie.yaml
rename to dbt-bigquery/.changie.yaml
diff --git a/CHANGELOG.md b/dbt-bigquery/CHANGELOG.md
similarity index 100%
rename from CHANGELOG.md
rename to dbt-bigquery/CHANGELOG.md
diff --git a/CONTRIBUTING.md b/dbt-bigquery/CONTRIBUTING.md
similarity index 100%
rename from CONTRIBUTING.md
rename to dbt-bigquery/CONTRIBUTING.md
diff --git a/LICENSE.md b/dbt-bigquery/LICENSE.md
similarity index 100%
rename from LICENSE.md
rename to dbt-bigquery/LICENSE.md
diff --git a/README.md b/dbt-bigquery/README.md
similarity index 100%
rename from README.md
rename to dbt-bigquery/README.md
diff --git a/docker/Dockerfile b/dbt-bigquery/docker/Dockerfile
similarity index 100%
rename from docker/Dockerfile
rename to dbt-bigquery/docker/Dockerfile
diff --git a/docker/README.md b/dbt-bigquery/docker/README.md
similarity index 100%
rename from docker/README.md
rename to dbt-bigquery/docker/README.md
diff --git a/docker/dev.Dockerfile b/dbt-bigquery/docker/dev.Dockerfile
similarity index 100%
rename from docker/dev.Dockerfile
rename to dbt-bigquery/docker/dev.Dockerfile
diff --git a/hatch.toml b/dbt-bigquery/hatch.toml
similarity index 93%
rename from hatch.toml
rename to dbt-bigquery/hatch.toml
index 4e0baf9f4..eb972b66a 100644
--- a/hatch.toml
+++ b/dbt-bigquery/hatch.toml
@@ -1,11 +1,13 @@
 [version]
-path = "dbt/adapters/bigquery/__version__.py"
+path = "src/dbt/adapters/bigquery/__version__.py"
 
 [build.targets.sdist]
-packages = ["dbt"]
+packages = ["src/dbt"]
+sources = ["src"]
 
 [build.targets.wheel]
-packages = ["dbt"]
+packages = ["src/dbt"]
+sources = ["src"]
 
 [envs.default]
 python = "3.9"
diff --git a/pyproject.toml b/dbt-bigquery/pyproject.toml
similarity index 100%
rename from pyproject.toml
rename to dbt-bigquery/pyproject.toml
diff --git a/.github/scripts/update_dev_dependency_branches.sh b/dbt-bigquery/scripts/update_dev_dependency_branches.sh
similarity index 100%
rename from .github/scripts/update_dev_dependency_branches.sh
rename to dbt-bigquery/scripts/update_dev_dependency_branches.sh
diff --git a/dbt/__init__.py b/dbt-bigquery/src/dbt/__init__.py
similarity index 100%
rename from dbt/__init__.py
rename to dbt-bigquery/src/dbt/__init__.py
diff --git a/dbt/adapters/bigquery/__init__.py b/dbt-bigquery/src/dbt/adapters/bigquery/__init__.py
similarity index 100%
rename from dbt/adapters/bigquery/__init__.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/__init__.py
diff --git a/dbt/adapters/bigquery/__version__.py b/dbt-bigquery/src/dbt/adapters/bigquery/__version__.py
similarity index 100%
rename from dbt/adapters/bigquery/__version__.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/__version__.py
diff --git a/dbt/adapters/bigquery/clients.py b/dbt-bigquery/src/dbt/adapters/bigquery/clients.py
similarity index 100%
rename from dbt/adapters/bigquery/clients.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/clients.py
diff --git a/dbt/adapters/bigquery/column.py b/dbt-bigquery/src/dbt/adapters/bigquery/column.py
similarity index 100%
rename from dbt/adapters/bigquery/column.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/column.py
diff --git a/dbt/adapters/bigquery/connections.py b/dbt-bigquery/src/dbt/adapters/bigquery/connections.py
similarity index 100%
rename from dbt/adapters/bigquery/connections.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/connections.py
diff --git a/dbt/adapters/bigquery/credentials.py b/dbt-bigquery/src/dbt/adapters/bigquery/credentials.py
similarity index 100%
rename from dbt/adapters/bigquery/credentials.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/credentials.py
diff --git a/dbt/adapters/bigquery/dataset.py b/dbt-bigquery/src/dbt/adapters/bigquery/dataset.py
similarity index 100%
rename from dbt/adapters/bigquery/dataset.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/dataset.py
diff --git a/dbt/adapters/bigquery/impl.py b/dbt-bigquery/src/dbt/adapters/bigquery/impl.py
similarity index 100%
rename from dbt/adapters/bigquery/impl.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/impl.py
diff --git a/dbt/adapters/bigquery/python_submissions.py b/dbt-bigquery/src/dbt/adapters/bigquery/python_submissions.py
similarity index 100%
rename from dbt/adapters/bigquery/python_submissions.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/python_submissions.py
diff --git a/dbt/adapters/bigquery/relation.py b/dbt-bigquery/src/dbt/adapters/bigquery/relation.py
similarity index 100%
rename from dbt/adapters/bigquery/relation.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/relation.py
diff --git a/dbt/adapters/bigquery/relation_configs/__init__.py b/dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/__init__.py
similarity index 100%
rename from dbt/adapters/bigquery/relation_configs/__init__.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/__init__.py
diff --git a/dbt/adapters/bigquery/relation_configs/_base.py b/dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/_base.py
similarity index 100%
rename from dbt/adapters/bigquery/relation_configs/_base.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/_base.py
diff --git a/dbt/adapters/bigquery/relation_configs/_cluster.py b/dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/_cluster.py
similarity index 100%
rename from dbt/adapters/bigquery/relation_configs/_cluster.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/_cluster.py
diff --git a/dbt/adapters/bigquery/relation_configs/_materialized_view.py b/dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/_materialized_view.py
similarity index 100%
rename from dbt/adapters/bigquery/relation_configs/_materialized_view.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/_materialized_view.py
diff --git a/dbt/adapters/bigquery/relation_configs/_options.py b/dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/_options.py
similarity index 100%
rename from dbt/adapters/bigquery/relation_configs/_options.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/_options.py
diff --git a/dbt/adapters/bigquery/relation_configs/_partition.py b/dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/_partition.py
similarity index 100%
rename from dbt/adapters/bigquery/relation_configs/_partition.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/_partition.py
diff --git a/dbt/adapters/bigquery/relation_configs/_policies.py b/dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/_policies.py
similarity index 100%
rename from dbt/adapters/bigquery/relation_configs/_policies.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/relation_configs/_policies.py
diff --git a/dbt/adapters/bigquery/retry.py b/dbt-bigquery/src/dbt/adapters/bigquery/retry.py
similarity index 100%
rename from dbt/adapters/bigquery/retry.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/retry.py
diff --git a/dbt/adapters/bigquery/utility.py b/dbt-bigquery/src/dbt/adapters/bigquery/utility.py
similarity index 100%
rename from dbt/adapters/bigquery/utility.py
rename to dbt-bigquery/src/dbt/adapters/bigquery/utility.py
diff --git a/dbt/include/bigquery/__init__.py b/dbt-bigquery/src/dbt/include/bigquery/__init__.py
similarity index 100%
rename from dbt/include/bigquery/__init__.py
rename to dbt-bigquery/src/dbt/include/bigquery/__init__.py
diff --git a/dbt/include/bigquery/dbt_project.yml b/dbt-bigquery/src/dbt/include/bigquery/dbt_project.yml
similarity index 100%
rename from dbt/include/bigquery/dbt_project.yml
rename to dbt-bigquery/src/dbt/include/bigquery/dbt_project.yml
diff --git a/dbt/include/bigquery/macros/adapters.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/adapters.sql
similarity index 100%
rename from dbt/include/bigquery/macros/adapters.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/adapters.sql
diff --git a/dbt/include/bigquery/macros/adapters/apply_grants.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/adapters/apply_grants.sql
similarity index 100%
rename from dbt/include/bigquery/macros/adapters/apply_grants.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/adapters/apply_grants.sql
diff --git a/dbt/include/bigquery/macros/adapters/columns.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/adapters/columns.sql
similarity index 100%
rename from dbt/include/bigquery/macros/adapters/columns.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/adapters/columns.sql
diff --git a/dbt/include/bigquery/macros/catalog/by_relation.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/catalog/by_relation.sql
similarity index 100%
rename from dbt/include/bigquery/macros/catalog/by_relation.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/catalog/by_relation.sql
diff --git a/dbt/include/bigquery/macros/catalog/by_schema.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/catalog/by_schema.sql
similarity index 100%
rename from dbt/include/bigquery/macros/catalog/by_schema.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/catalog/by_schema.sql
diff --git a/dbt/include/bigquery/macros/catalog/catalog.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/catalog/catalog.sql
similarity index 100%
rename from dbt/include/bigquery/macros/catalog/catalog.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/catalog/catalog.sql
diff --git a/dbt/include/bigquery/macros/etc.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/etc.sql
similarity index 100%
rename from dbt/include/bigquery/macros/etc.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/etc.sql
diff --git a/dbt/include/bigquery/macros/materializations/clone.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/materializations/clone.sql
similarity index 100%
rename from dbt/include/bigquery/macros/materializations/clone.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/materializations/clone.sql
diff --git a/dbt/include/bigquery/macros/materializations/copy.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/materializations/copy.sql
similarity index 100%
rename from dbt/include/bigquery/macros/materializations/copy.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/materializations/copy.sql
diff --git a/dbt/include/bigquery/macros/materializations/incremental.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/materializations/incremental.sql
similarity index 100%
rename from dbt/include/bigquery/macros/materializations/incremental.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/materializations/incremental.sql
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
similarity index 100%
rename from dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/materializations/incremental_strategy/common.sql
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
similarity index 100%
rename from dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/materializations/incremental_strategy/insert_overwrite.sql
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
similarity index 100%
rename from dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/materializations/incremental_strategy/merge.sql
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/microbatch.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/materializations/incremental_strategy/microbatch.sql
similarity index 100%
rename from dbt/include/bigquery/macros/materializations/incremental_strategy/microbatch.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/materializations/incremental_strategy/microbatch.sql
diff --git a/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
similarity index 100%
rename from dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/materializations/incremental_strategy/time_ingestion_tables.sql
diff --git a/dbt/include/bigquery/macros/materializations/seed.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/materializations/seed.sql
similarity index 100%
rename from dbt/include/bigquery/macros/materializations/seed.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/materializations/seed.sql
diff --git a/dbt/include/bigquery/macros/materializations/snapshot.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/materializations/snapshot.sql
similarity index 100%
rename from dbt/include/bigquery/macros/materializations/snapshot.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/materializations/snapshot.sql
diff --git a/dbt/include/bigquery/macros/materializations/table.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/materializations/table.sql
similarity index 100%
rename from dbt/include/bigquery/macros/materializations/table.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/materializations/table.sql
diff --git a/dbt/include/bigquery/macros/materializations/view.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/materializations/view.sql
similarity index 100%
rename from dbt/include/bigquery/macros/materializations/view.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/materializations/view.sql
diff --git a/dbt/include/bigquery/macros/python_model/python.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/python_model/python.sql
similarity index 100%
rename from dbt/include/bigquery/macros/python_model/python.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/python_model/python.sql
diff --git a/dbt/include/bigquery/macros/relations/cluster.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/cluster.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/cluster.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/cluster.sql
diff --git a/dbt/include/bigquery/macros/relations/drop.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/drop.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/drop.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/drop.sql
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/alter.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/materialized_view/alter.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/materialized_view/alter.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/materialized_view/alter.sql
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/create.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/materialized_view/create.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/materialized_view/create.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/materialized_view/create.sql
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/drop.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/materialized_view/drop.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/materialized_view/drop.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/materialized_view/drop.sql
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/materialized_view/refresh.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/materialized_view/refresh.sql
diff --git a/dbt/include/bigquery/macros/relations/materialized_view/replace.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/materialized_view/replace.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/materialized_view/replace.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/materialized_view/replace.sql
diff --git a/dbt/include/bigquery/macros/relations/options.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/options.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/options.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/options.sql
diff --git a/dbt/include/bigquery/macros/relations/partition.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/partition.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/partition.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/partition.sql
diff --git a/dbt/include/bigquery/macros/relations/rename.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/rename.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/rename.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/rename.sql
diff --git a/dbt/include/bigquery/macros/relations/table/drop.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/table/drop.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/table/drop.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/table/drop.sql
diff --git a/dbt/include/bigquery/macros/relations/table/options.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/table/options.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/table/options.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/table/options.sql
diff --git a/dbt/include/bigquery/macros/relations/table/rename.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/table/rename.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/table/rename.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/table/rename.sql
diff --git a/dbt/include/bigquery/macros/relations/view/drop.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/view/drop.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/view/drop.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/view/drop.sql
diff --git a/dbt/include/bigquery/macros/relations/view/options.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/view/options.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/view/options.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/view/options.sql
diff --git a/dbt/include/bigquery/macros/relations/view/rename.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/view/rename.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/view/rename.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/view/rename.sql
diff --git a/dbt/include/bigquery/macros/relations/view/replace.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/relations/view/replace.sql
similarity index 100%
rename from dbt/include/bigquery/macros/relations/view/replace.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/relations/view/replace.sql
diff --git a/dbt/include/bigquery/macros/utils/array_append.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/array_append.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/array_append.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/array_append.sql
diff --git a/dbt/include/bigquery/macros/utils/array_concat.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/array_concat.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/array_concat.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/array_concat.sql
diff --git a/dbt/include/bigquery/macros/utils/array_construct.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/array_construct.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/array_construct.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/array_construct.sql
diff --git a/dbt/include/bigquery/macros/utils/bool_or.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/bool_or.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/bool_or.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/bool_or.sql
diff --git a/dbt/include/bigquery/macros/utils/date.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/date.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/date.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/date.sql
diff --git a/dbt/include/bigquery/macros/utils/date_trunc.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/date_trunc.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/date_trunc.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/date_trunc.sql
diff --git a/dbt/include/bigquery/macros/utils/dateadd.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/dateadd.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/dateadd.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/dateadd.sql
diff --git a/dbt/include/bigquery/macros/utils/datediff.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/datediff.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/datediff.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/datediff.sql
diff --git a/dbt/include/bigquery/macros/utils/escape_single_quotes.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/escape_single_quotes.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/escape_single_quotes.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/escape_single_quotes.sql
diff --git a/dbt/include/bigquery/macros/utils/except.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/except.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/except.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/except.sql
diff --git a/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/get_columns_spec_ddl.sql
diff --git a/dbt/include/bigquery/macros/utils/hash.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/hash.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/hash.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/hash.sql
diff --git a/dbt/include/bigquery/macros/utils/intersect.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/intersect.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/intersect.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/intersect.sql
diff --git a/dbt/include/bigquery/macros/utils/listagg.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/listagg.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/listagg.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/listagg.sql
diff --git a/dbt/include/bigquery/macros/utils/position.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/position.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/position.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/position.sql
diff --git a/dbt/include/bigquery/macros/utils/right.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/right.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/right.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/right.sql
diff --git a/dbt/include/bigquery/macros/utils/safe_cast.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/safe_cast.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/safe_cast.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/safe_cast.sql
diff --git a/dbt/include/bigquery/macros/utils/split_part.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/split_part.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/split_part.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/split_part.sql
diff --git a/dbt/include/bigquery/macros/utils/string_literal.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/string_literal.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/string_literal.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/string_literal.sql
diff --git a/dbt/include/bigquery/macros/utils/timestamps.sql b/dbt-bigquery/src/dbt/include/bigquery/macros/utils/timestamps.sql
similarity index 100%
rename from dbt/include/bigquery/macros/utils/timestamps.sql
rename to dbt-bigquery/src/dbt/include/bigquery/macros/utils/timestamps.sql
diff --git a/dbt/include/bigquery/profile_template.yml b/dbt-bigquery/src/dbt/include/bigquery/profile_template.yml
similarity index 100%
rename from dbt/include/bigquery/profile_template.yml
rename to dbt-bigquery/src/dbt/include/bigquery/profile_template.yml
diff --git a/test.env.example b/dbt-bigquery/test.env.example
similarity index 100%
rename from test.env.example
rename to dbt-bigquery/test.env.example
diff --git a/tests/__init__.py b/dbt-bigquery/tests/__init__.py
similarity index 100%
rename from tests/__init__.py
rename to dbt-bigquery/tests/__init__.py
diff --git a/tests/boundary/test_bigquery_sdk.py b/dbt-bigquery/tests/boundary/test_bigquery_sdk.py
similarity index 100%
rename from tests/boundary/test_bigquery_sdk.py
rename to dbt-bigquery/tests/boundary/test_bigquery_sdk.py
diff --git a/tests/conftest.py b/dbt-bigquery/tests/conftest.py
similarity index 100%
rename from tests/conftest.py
rename to dbt-bigquery/tests/conftest.py
diff --git a/tests/functional/adapter/catalog_tests/files.py b/dbt-bigquery/tests/functional/adapter/catalog_tests/files.py
similarity index 100%
rename from tests/functional/adapter/catalog_tests/files.py
rename to dbt-bigquery/tests/functional/adapter/catalog_tests/files.py
diff --git a/tests/functional/adapter/catalog_tests/test_relation_types.py b/dbt-bigquery/tests/functional/adapter/catalog_tests/test_relation_types.py
similarity index 100%
rename from tests/functional/adapter/catalog_tests/test_relation_types.py
rename to dbt-bigquery/tests/functional/adapter/catalog_tests/test_relation_types.py
diff --git a/tests/functional/adapter/column_types/fixtures.py b/dbt-bigquery/tests/functional/adapter/column_types/fixtures.py
similarity index 100%
rename from tests/functional/adapter/column_types/fixtures.py
rename to dbt-bigquery/tests/functional/adapter/column_types/fixtures.py
diff --git a/tests/functional/adapter/column_types/test_alter_column_types.py b/dbt-bigquery/tests/functional/adapter/column_types/test_alter_column_types.py
similarity index 100%
rename from tests/functional/adapter/column_types/test_alter_column_types.py
rename to dbt-bigquery/tests/functional/adapter/column_types/test_alter_column_types.py
diff --git a/tests/functional/adapter/column_types/test_column_types.py b/dbt-bigquery/tests/functional/adapter/column_types/test_column_types.py
similarity index 100%
rename from tests/functional/adapter/column_types/test_column_types.py
rename to dbt-bigquery/tests/functional/adapter/column_types/test_column_types.py
diff --git a/tests/functional/adapter/constraints/fixtures.py b/dbt-bigquery/tests/functional/adapter/constraints/fixtures.py
similarity index 100%
rename from tests/functional/adapter/constraints/fixtures.py
rename to dbt-bigquery/tests/functional/adapter/constraints/fixtures.py
diff --git a/tests/functional/adapter/constraints/test_constraints.py b/dbt-bigquery/tests/functional/adapter/constraints/test_constraints.py
similarity index 100%
rename from tests/functional/adapter/constraints/test_constraints.py
rename to dbt-bigquery/tests/functional/adapter/constraints/test_constraints.py
diff --git a/tests/functional/adapter/dbt_clone/test_dbt_clone.py b/dbt-bigquery/tests/functional/adapter/dbt_clone/test_dbt_clone.py
similarity index 100%
rename from tests/functional/adapter/dbt_clone/test_dbt_clone.py
rename to dbt-bigquery/tests/functional/adapter/dbt_clone/test_dbt_clone.py
diff --git a/tests/functional/adapter/dbt_show/test_dbt_show.py b/dbt-bigquery/tests/functional/adapter/dbt_show/test_dbt_show.py
similarity index 100%
rename from tests/functional/adapter/dbt_show/test_dbt_show.py
rename to dbt-bigquery/tests/functional/adapter/dbt_show/test_dbt_show.py
diff --git a/tests/functional/adapter/describe_relation/_files.py b/dbt-bigquery/tests/functional/adapter/describe_relation/_files.py
similarity index 100%
rename from tests/functional/adapter/describe_relation/_files.py
rename to dbt-bigquery/tests/functional/adapter/describe_relation/_files.py
diff --git a/tests/functional/adapter/describe_relation/test_describe_relation.py b/dbt-bigquery/tests/functional/adapter/describe_relation/test_describe_relation.py
similarity index 100%
rename from tests/functional/adapter/describe_relation/test_describe_relation.py
rename to dbt-bigquery/tests/functional/adapter/describe_relation/test_describe_relation.py
diff --git a/tests/functional/adapter/empty/test_empty.py b/dbt-bigquery/tests/functional/adapter/empty/test_empty.py
similarity index 100%
rename from tests/functional/adapter/empty/test_empty.py
rename to dbt-bigquery/tests/functional/adapter/empty/test_empty.py
diff --git a/tests/functional/adapter/expected_stats.py b/dbt-bigquery/tests/functional/adapter/expected_stats.py
similarity index 100%
rename from tests/functional/adapter/expected_stats.py
rename to dbt-bigquery/tests/functional/adapter/expected_stats.py
diff --git a/tests/functional/adapter/hooks/data/seed_model.sql b/dbt-bigquery/tests/functional/adapter/hooks/data/seed_model.sql
similarity index 100%
rename from tests/functional/adapter/hooks/data/seed_model.sql
rename to dbt-bigquery/tests/functional/adapter/hooks/data/seed_model.sql
diff --git a/tests/functional/adapter/hooks/data/seed_run.sql b/dbt-bigquery/tests/functional/adapter/hooks/data/seed_run.sql
similarity index 100%
rename from tests/functional/adapter/hooks/data/seed_run.sql
rename to dbt-bigquery/tests/functional/adapter/hooks/data/seed_run.sql
diff --git a/tests/functional/adapter/hooks/test_model_hooks.py b/dbt-bigquery/tests/functional/adapter/hooks/test_model_hooks.py
similarity index 100%
rename from tests/functional/adapter/hooks/test_model_hooks.py
rename to dbt-bigquery/tests/functional/adapter/hooks/test_model_hooks.py
diff --git a/tests/functional/adapter/hooks/test_run_hooks.py b/dbt-bigquery/tests/functional/adapter/hooks/test_run_hooks.py
similarity index 100%
rename from tests/functional/adapter/hooks/test_run_hooks.py
rename to dbt-bigquery/tests/functional/adapter/hooks/test_run_hooks.py
diff --git a/tests/functional/adapter/incremental/incremental_strategy_fixtures.py b/dbt-bigquery/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
similarity index 100%
rename from tests/functional/adapter/incremental/incremental_strategy_fixtures.py
rename to dbt-bigquery/tests/functional/adapter/incremental/incremental_strategy_fixtures.py
diff --git a/tests/functional/adapter/incremental/seeds.py b/dbt-bigquery/tests/functional/adapter/incremental/seeds.py
similarity index 100%
rename from tests/functional/adapter/incremental/seeds.py
rename to dbt-bigquery/tests/functional/adapter/incremental/seeds.py
diff --git a/tests/functional/adapter/incremental/test_incremental_merge_exclude_columns.py b/dbt-bigquery/tests/functional/adapter/incremental/test_incremental_merge_exclude_columns.py
similarity index 100%
rename from tests/functional/adapter/incremental/test_incremental_merge_exclude_columns.py
rename to dbt-bigquery/tests/functional/adapter/incremental/test_incremental_merge_exclude_columns.py
diff --git a/tests/functional/adapter/incremental/test_incremental_microbatch.py b/dbt-bigquery/tests/functional/adapter/incremental/test_incremental_microbatch.py
similarity index 100%
rename from tests/functional/adapter/incremental/test_incremental_microbatch.py
rename to dbt-bigquery/tests/functional/adapter/incremental/test_incremental_microbatch.py
diff --git a/tests/functional/adapter/incremental/test_incremental_on_schema_change.py b/dbt-bigquery/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
similarity index 100%
rename from tests/functional/adapter/incremental/test_incremental_on_schema_change.py
rename to dbt-bigquery/tests/functional/adapter/incremental/test_incremental_on_schema_change.py
diff --git a/tests/functional/adapter/incremental/test_incremental_predicates.py b/dbt-bigquery/tests/functional/adapter/incremental/test_incremental_predicates.py
similarity index 100%
rename from tests/functional/adapter/incremental/test_incremental_predicates.py
rename to dbt-bigquery/tests/functional/adapter/incremental/test_incremental_predicates.py
diff --git a/tests/functional/adapter/incremental/test_incremental_strategies.py b/dbt-bigquery/tests/functional/adapter/incremental/test_incremental_strategies.py
similarity index 100%
rename from tests/functional/adapter/incremental/test_incremental_strategies.py
rename to dbt-bigquery/tests/functional/adapter/incremental/test_incremental_strategies.py
diff --git a/tests/functional/adapter/incremental/test_incremental_unique_id.py b/dbt-bigquery/tests/functional/adapter/incremental/test_incremental_unique_id.py
similarity index 100%
rename from tests/functional/adapter/incremental/test_incremental_unique_id.py
rename to dbt-bigquery/tests/functional/adapter/incremental/test_incremental_unique_id.py
diff --git a/tests/functional/adapter/materialized_view_tests/_files.py b/dbt-bigquery/tests/functional/adapter/materialized_view_tests/_files.py
similarity index 100%
rename from tests/functional/adapter/materialized_view_tests/_files.py
rename to dbt-bigquery/tests/functional/adapter/materialized_view_tests/_files.py
diff --git a/tests/functional/adapter/materialized_view_tests/_mixin.py b/dbt-bigquery/tests/functional/adapter/materialized_view_tests/_mixin.py
similarity index 100%
rename from tests/functional/adapter/materialized_view_tests/_mixin.py
rename to dbt-bigquery/tests/functional/adapter/materialized_view_tests/_mixin.py
diff --git a/tests/functional/adapter/materialized_view_tests/test_materialized_view.py b/dbt-bigquery/tests/functional/adapter/materialized_view_tests/test_materialized_view.py
similarity index 100%
rename from tests/functional/adapter/materialized_view_tests/test_materialized_view.py
rename to dbt-bigquery/tests/functional/adapter/materialized_view_tests/test_materialized_view.py
diff --git a/tests/functional/adapter/materialized_view_tests/test_materialized_view_changes.py b/dbt-bigquery/tests/functional/adapter/materialized_view_tests/test_materialized_view_changes.py
similarity index 100%
rename from tests/functional/adapter/materialized_view_tests/test_materialized_view_changes.py
rename to dbt-bigquery/tests/functional/adapter/materialized_view_tests/test_materialized_view_changes.py
diff --git a/tests/functional/adapter/materialized_view_tests/test_materialized_view_cluster_changes.py b/dbt-bigquery/tests/functional/adapter/materialized_view_tests/test_materialized_view_cluster_changes.py
similarity index 100%
rename from tests/functional/adapter/materialized_view_tests/test_materialized_view_cluster_changes.py
rename to dbt-bigquery/tests/functional/adapter/materialized_view_tests/test_materialized_view_cluster_changes.py
diff --git a/tests/functional/adapter/materialized_view_tests/test_materialized_view_partition_changes.py b/dbt-bigquery/tests/functional/adapter/materialized_view_tests/test_materialized_view_partition_changes.py
similarity index 100%
rename from tests/functional/adapter/materialized_view_tests/test_materialized_view_partition_changes.py
rename to dbt-bigquery/tests/functional/adapter/materialized_view_tests/test_materialized_view_partition_changes.py
diff --git a/tests/functional/adapter/query_comment_test/test_job_label.py b/dbt-bigquery/tests/functional/adapter/query_comment_test/test_job_label.py
similarity index 100%
rename from tests/functional/adapter/query_comment_test/test_job_label.py
rename to dbt-bigquery/tests/functional/adapter/query_comment_test/test_job_label.py
diff --git a/tests/functional/adapter/query_comment_test/test_query_comment.py b/dbt-bigquery/tests/functional/adapter/query_comment_test/test_query_comment.py
similarity index 100%
rename from tests/functional/adapter/query_comment_test/test_query_comment.py
rename to dbt-bigquery/tests/functional/adapter/query_comment_test/test_query_comment.py
diff --git a/tests/functional/adapter/simple_bigquery_view/fixtures.py b/dbt-bigquery/tests/functional/adapter/simple_bigquery_view/fixtures.py
similarity index 100%
rename from tests/functional/adapter/simple_bigquery_view/fixtures.py
rename to dbt-bigquery/tests/functional/adapter/simple_bigquery_view/fixtures.py
diff --git a/tests/functional/adapter/simple_bigquery_view/seeds.py b/dbt-bigquery/tests/functional/adapter/simple_bigquery_view/seeds.py
similarity index 100%
rename from tests/functional/adapter/simple_bigquery_view/seeds.py
rename to dbt-bigquery/tests/functional/adapter/simple_bigquery_view/seeds.py
diff --git a/tests/functional/adapter/simple_bigquery_view/test_simple_bigquery_view.py b/dbt-bigquery/tests/functional/adapter/simple_bigquery_view/test_simple_bigquery_view.py
similarity index 100%
rename from tests/functional/adapter/simple_bigquery_view/test_simple_bigquery_view.py
rename to dbt-bigquery/tests/functional/adapter/simple_bigquery_view/test_simple_bigquery_view.py
diff --git a/tests/functional/adapter/simple_copy/fixtures.py b/dbt-bigquery/tests/functional/adapter/simple_copy/fixtures.py
similarity index 100%
rename from tests/functional/adapter/simple_copy/fixtures.py
rename to dbt-bigquery/tests/functional/adapter/simple_copy/fixtures.py
diff --git a/tests/functional/adapter/simple_copy/test_simple_copy.py b/dbt-bigquery/tests/functional/adapter/simple_copy/test_simple_copy.py
similarity index 100%
rename from tests/functional/adapter/simple_copy/test_simple_copy.py
rename to dbt-bigquery/tests/functional/adapter/simple_copy/test_simple_copy.py
diff --git a/tests/functional/adapter/sources_freshness_tests/files.py b/dbt-bigquery/tests/functional/adapter/sources_freshness_tests/files.py
similarity index 100%
rename from tests/functional/adapter/sources_freshness_tests/files.py
rename to dbt-bigquery/tests/functional/adapter/sources_freshness_tests/files.py
diff --git a/tests/functional/adapter/sources_freshness_tests/test_get_relation_last_modified.py b/dbt-bigquery/tests/functional/adapter/sources_freshness_tests/test_get_relation_last_modified.py
similarity index 100%
rename from tests/functional/adapter/sources_freshness_tests/test_get_relation_last_modified.py
rename to dbt-bigquery/tests/functional/adapter/sources_freshness_tests/test_get_relation_last_modified.py
diff --git a/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py b/dbt-bigquery/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
similarity index 100%
rename from tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
rename to dbt-bigquery/tests/functional/adapter/store_test_failures_tests/test_store_test_failures.py
diff --git a/tests/functional/adapter/test_aliases.py b/dbt-bigquery/tests/functional/adapter/test_aliases.py
similarity index 100%
rename from tests/functional/adapter/test_aliases.py
rename to dbt-bigquery/tests/functional/adapter/test_aliases.py
diff --git a/tests/functional/adapter/test_basic.py b/dbt-bigquery/tests/functional/adapter/test_basic.py
similarity index 100%
rename from tests/functional/adapter/test_basic.py
rename to dbt-bigquery/tests/functional/adapter/test_basic.py
diff --git a/tests/functional/adapter/test_changing_relation_type.py b/dbt-bigquery/tests/functional/adapter/test_changing_relation_type.py
similarity index 100%
rename from tests/functional/adapter/test_changing_relation_type.py
rename to dbt-bigquery/tests/functional/adapter/test_changing_relation_type.py
diff --git a/tests/functional/adapter/test_copy_materialization.py b/dbt-bigquery/tests/functional/adapter/test_copy_materialization.py
similarity index 100%
rename from tests/functional/adapter/test_copy_materialization.py
rename to dbt-bigquery/tests/functional/adapter/test_copy_materialization.py
diff --git a/tests/functional/adapter/test_dbt_debug.py b/dbt-bigquery/tests/functional/adapter/test_dbt_debug.py
similarity index 100%
rename from tests/functional/adapter/test_dbt_debug.py
rename to dbt-bigquery/tests/functional/adapter/test_dbt_debug.py
diff --git a/tests/functional/adapter/test_grant_access_to.py b/dbt-bigquery/tests/functional/adapter/test_grant_access_to.py
similarity index 100%
rename from tests/functional/adapter/test_grant_access_to.py
rename to dbt-bigquery/tests/functional/adapter/test_grant_access_to.py
diff --git a/tests/functional/adapter/test_grants.py b/dbt-bigquery/tests/functional/adapter/test_grants.py
similarity index 100%
rename from tests/functional/adapter/test_grants.py
rename to dbt-bigquery/tests/functional/adapter/test_grants.py
diff --git a/tests/functional/adapter/test_json_keyfile.py b/dbt-bigquery/tests/functional/adapter/test_json_keyfile.py
similarity index 100%
rename from tests/functional/adapter/test_json_keyfile.py
rename to dbt-bigquery/tests/functional/adapter/test_json_keyfile.py
diff --git a/tests/functional/adapter/test_persist_docs.py b/dbt-bigquery/tests/functional/adapter/test_persist_docs.py
similarity index 100%
rename from tests/functional/adapter/test_persist_docs.py
rename to dbt-bigquery/tests/functional/adapter/test_persist_docs.py
diff --git a/tests/functional/adapter/test_python_model.py b/dbt-bigquery/tests/functional/adapter/test_python_model.py
similarity index 100%
rename from tests/functional/adapter/test_python_model.py
rename to dbt-bigquery/tests/functional/adapter/test_python_model.py
diff --git a/tests/functional/adapter/test_simple_seed.py b/dbt-bigquery/tests/functional/adapter/test_simple_seed.py
similarity index 100%
rename from tests/functional/adapter/test_simple_seed.py
rename to dbt-bigquery/tests/functional/adapter/test_simple_seed.py
diff --git a/tests/functional/adapter/test_simple_snaphot.py b/dbt-bigquery/tests/functional/adapter/test_simple_snaphot.py
similarity index 100%
rename from tests/functional/adapter/test_simple_snaphot.py
rename to dbt-bigquery/tests/functional/adapter/test_simple_snaphot.py
diff --git a/tests/functional/adapter/test_string_literal_macro.py b/dbt-bigquery/tests/functional/adapter/test_string_literal_macro.py
similarity index 100%
rename from tests/functional/adapter/test_string_literal_macro.py
rename to dbt-bigquery/tests/functional/adapter/test_string_literal_macro.py
diff --git a/tests/functional/adapter/unit_testing/test_unit_testing.py b/dbt-bigquery/tests/functional/adapter/unit_testing/test_unit_testing.py
similarity index 100%
rename from tests/functional/adapter/unit_testing/test_unit_testing.py
rename to dbt-bigquery/tests/functional/adapter/unit_testing/test_unit_testing.py
diff --git a/tests/functional/adapter/upload_file_tests/data/csv/source.csv b/dbt-bigquery/tests/functional/adapter/upload_file_tests/data/csv/source.csv
similarity index 100%
rename from tests/functional/adapter/upload_file_tests/data/csv/source.csv
rename to dbt-bigquery/tests/functional/adapter/upload_file_tests/data/csv/source.csv
diff --git a/tests/functional/adapter/upload_file_tests/data/ndjson/source.ndjson b/dbt-bigquery/tests/functional/adapter/upload_file_tests/data/ndjson/source.ndjson
similarity index 100%
rename from tests/functional/adapter/upload_file_tests/data/ndjson/source.ndjson
rename to dbt-bigquery/tests/functional/adapter/upload_file_tests/data/ndjson/source.ndjson
diff --git a/tests/functional/adapter/upload_file_tests/data/parquet/source.parquet b/dbt-bigquery/tests/functional/adapter/upload_file_tests/data/parquet/source.parquet
similarity index 100%
rename from tests/functional/adapter/upload_file_tests/data/parquet/source.parquet
rename to dbt-bigquery/tests/functional/adapter/upload_file_tests/data/parquet/source.parquet
diff --git a/tests/functional/adapter/upload_file_tests/test_upload_file.py b/dbt-bigquery/tests/functional/adapter/upload_file_tests/test_upload_file.py
similarity index 100%
rename from tests/functional/adapter/upload_file_tests/test_upload_file.py
rename to dbt-bigquery/tests/functional/adapter/upload_file_tests/test_upload_file.py
diff --git a/tests/functional/adapter/utils/fixture_array_append.py b/dbt-bigquery/tests/functional/adapter/utils/fixture_array_append.py
similarity index 100%
rename from tests/functional/adapter/utils/fixture_array_append.py
rename to dbt-bigquery/tests/functional/adapter/utils/fixture_array_append.py
diff --git a/tests/functional/adapter/utils/fixture_array_concat.py b/dbt-bigquery/tests/functional/adapter/utils/fixture_array_concat.py
similarity index 100%
rename from tests/functional/adapter/utils/fixture_array_concat.py
rename to dbt-bigquery/tests/functional/adapter/utils/fixture_array_concat.py
diff --git a/tests/functional/adapter/utils/fixture_array_construct.py b/dbt-bigquery/tests/functional/adapter/utils/fixture_array_construct.py
similarity index 100%
rename from tests/functional/adapter/utils/fixture_array_construct.py
rename to dbt-bigquery/tests/functional/adapter/utils/fixture_array_construct.py
diff --git a/tests/functional/adapter/utils/fixture_get_intervals_between.py b/dbt-bigquery/tests/functional/adapter/utils/fixture_get_intervals_between.py
similarity index 100%
rename from tests/functional/adapter/utils/fixture_get_intervals_between.py
rename to dbt-bigquery/tests/functional/adapter/utils/fixture_get_intervals_between.py
diff --git a/tests/functional/adapter/utils/test_data_types.py b/dbt-bigquery/tests/functional/adapter/utils/test_data_types.py
similarity index 100%
rename from tests/functional/adapter/utils/test_data_types.py
rename to dbt-bigquery/tests/functional/adapter/utils/test_data_types.py
diff --git a/tests/functional/adapter/utils/test_timestamps.py b/dbt-bigquery/tests/functional/adapter/utils/test_timestamps.py
similarity index 100%
rename from tests/functional/adapter/utils/test_timestamps.py
rename to dbt-bigquery/tests/functional/adapter/utils/test_timestamps.py
diff --git a/tests/functional/adapter/utils/test_utils.py b/dbt-bigquery/tests/functional/adapter/utils/test_utils.py
similarity index 100%
rename from tests/functional/adapter/utils/test_utils.py
rename to dbt-bigquery/tests/functional/adapter/utils/test_utils.py
diff --git a/tests/functional/python_model_tests/__init__.py b/dbt-bigquery/tests/functional/python_model_tests/__init__.py
similarity index 100%
rename from tests/functional/python_model_tests/__init__.py
rename to dbt-bigquery/tests/functional/python_model_tests/__init__.py
diff --git a/tests/functional/python_model_tests/files.py b/dbt-bigquery/tests/functional/python_model_tests/files.py
similarity index 100%
rename from tests/functional/python_model_tests/files.py
rename to dbt-bigquery/tests/functional/python_model_tests/files.py
diff --git a/tests/functional/python_model_tests/test_list_inference.py b/dbt-bigquery/tests/functional/python_model_tests/test_list_inference.py
similarity index 100%
rename from tests/functional/python_model_tests/test_list_inference.py
rename to dbt-bigquery/tests/functional/python_model_tests/test_list_inference.py
diff --git a/tests/functional/test_cancel.py b/dbt-bigquery/tests/functional/test_cancel.py
similarity index 100%
rename from tests/functional/test_cancel.py
rename to dbt-bigquery/tests/functional/test_cancel.py
diff --git a/tests/functional/test_changing_partitions.py b/dbt-bigquery/tests/functional/test_changing_partitions.py
similarity index 100%
rename from tests/functional/test_changing_partitions.py
rename to dbt-bigquery/tests/functional/test_changing_partitions.py
diff --git a/tests/functional/test_delete_column_policy.py b/dbt-bigquery/tests/functional/test_delete_column_policy.py
similarity index 100%
rename from tests/functional/test_delete_column_policy.py
rename to dbt-bigquery/tests/functional/test_delete_column_policy.py
diff --git a/tests/functional/test_drop_temp_relation.py b/dbt-bigquery/tests/functional/test_drop_temp_relation.py
similarity index 100%
rename from tests/functional/test_drop_temp_relation.py
rename to dbt-bigquery/tests/functional/test_drop_temp_relation.py
diff --git a/tests/functional/test_get_columns_incomplete_database.py b/dbt-bigquery/tests/functional/test_get_columns_incomplete_database.py
similarity index 100%
rename from tests/functional/test_get_columns_incomplete_database.py
rename to dbt-bigquery/tests/functional/test_get_columns_incomplete_database.py
diff --git a/tests/functional/test_hours_to_expiration.py b/dbt-bigquery/tests/functional/test_hours_to_expiration.py
similarity index 100%
rename from tests/functional/test_hours_to_expiration.py
rename to dbt-bigquery/tests/functional/test_hours_to_expiration.py
diff --git a/tests/functional/test_incremental_materialization.py b/dbt-bigquery/tests/functional/test_incremental_materialization.py
similarity index 100%
rename from tests/functional/test_incremental_materialization.py
rename to dbt-bigquery/tests/functional/test_incremental_materialization.py
diff --git a/tests/functional/test_job_timeout.py b/dbt-bigquery/tests/functional/test_job_timeout.py
similarity index 100%
rename from tests/functional/test_job_timeout.py
rename to dbt-bigquery/tests/functional/test_job_timeout.py
diff --git a/tests/functional/test_location_change.py b/dbt-bigquery/tests/functional/test_location_change.py
similarity index 100%
rename from tests/functional/test_location_change.py
rename to dbt-bigquery/tests/functional/test_location_change.py
diff --git a/tests/functional/test_override_database/fixtures.py b/dbt-bigquery/tests/functional/test_override_database/fixtures.py
similarity index 100%
rename from tests/functional/test_override_database/fixtures.py
rename to dbt-bigquery/tests/functional/test_override_database/fixtures.py
diff --git a/tests/functional/test_override_database/test_override_database.py b/dbt-bigquery/tests/functional/test_override_database/test_override_database.py
similarity index 100%
rename from tests/functional/test_override_database/test_override_database.py
rename to dbt-bigquery/tests/functional/test_override_database/test_override_database.py
diff --git a/tests/functional/test_quota_project.py b/dbt-bigquery/tests/functional/test_quota_project.py
similarity index 100%
rename from tests/functional/test_quota_project.py
rename to dbt-bigquery/tests/functional/test_quota_project.py
diff --git a/tests/functional/test_update_column_policy.py b/dbt-bigquery/tests/functional/test_update_column_policy.py
similarity index 100%
rename from tests/functional/test_update_column_policy.py
rename to dbt-bigquery/tests/functional/test_update_column_policy.py
diff --git a/tests/functional/test_update_field_description.py b/dbt-bigquery/tests/functional/test_update_field_description.py
similarity index 100%
rename from tests/functional/test_update_field_description.py
rename to dbt-bigquery/tests/functional/test_update_field_description.py
diff --git a/tests/unit/__init__.py b/dbt-bigquery/tests/unit/__init__.py
similarity index 100%
rename from tests/unit/__init__.py
rename to dbt-bigquery/tests/unit/__init__.py
diff --git a/tests/unit/mock_adapter.py b/dbt-bigquery/tests/unit/mock_adapter.py
similarity index 100%
rename from tests/unit/mock_adapter.py
rename to dbt-bigquery/tests/unit/mock_adapter.py
diff --git a/tests/unit/test_bigquery_adapter.py b/dbt-bigquery/tests/unit/test_bigquery_adapter.py
similarity index 100%
rename from tests/unit/test_bigquery_adapter.py
rename to dbt-bigquery/tests/unit/test_bigquery_adapter.py
diff --git a/tests/unit/test_bigquery_connection_manager.py b/dbt-bigquery/tests/unit/test_bigquery_connection_manager.py
similarity index 100%
rename from tests/unit/test_bigquery_connection_manager.py
rename to dbt-bigquery/tests/unit/test_bigquery_connection_manager.py
diff --git a/tests/unit/test_column.py b/dbt-bigquery/tests/unit/test_column.py
similarity index 100%
rename from tests/unit/test_column.py
rename to dbt-bigquery/tests/unit/test_column.py
diff --git a/tests/unit/test_configure_dataproc_batch.py b/dbt-bigquery/tests/unit/test_configure_dataproc_batch.py
similarity index 100%
rename from tests/unit/test_configure_dataproc_batch.py
rename to dbt-bigquery/tests/unit/test_configure_dataproc_batch.py
diff --git a/tests/unit/test_dataset.py b/dbt-bigquery/tests/unit/test_dataset.py
similarity index 100%
rename from tests/unit/test_dataset.py
rename to dbt-bigquery/tests/unit/test_dataset.py
diff --git a/tests/unit/test_renamed_relations.py b/dbt-bigquery/tests/unit/test_renamed_relations.py
similarity index 100%
rename from tests/unit/test_renamed_relations.py
rename to dbt-bigquery/tests/unit/test_renamed_relations.py
diff --git a/tests/unit/utils.py b/dbt-bigquery/tests/unit/utils.py
similarity index 100%
rename from tests/unit/utils.py
rename to dbt-bigquery/tests/unit/utils.py
diff --git a/third-party-stubs/agate/__init__.pyi b/dbt-bigquery/third-party-stubs/agate/__init__.pyi
similarity index 100%
rename from third-party-stubs/agate/__init__.pyi
rename to dbt-bigquery/third-party-stubs/agate/__init__.pyi
diff --git a/third-party-stubs/agate/data_types.pyi b/dbt-bigquery/third-party-stubs/agate/data_types.pyi
similarity index 100%
rename from third-party-stubs/agate/data_types.pyi
rename to dbt-bigquery/third-party-stubs/agate/data_types.pyi