From 6c8156203fd25a3c2e0bdb49e4709b9f0903fafb Mon Sep 17 00:00:00 2001 From: Jacob Beck Date: Thu, 19 Sep 2019 15:49:35 -0600 Subject: [PATCH] Include the database when deciding if two tables are the same in catalog generation Convert catalog intermediate structure into something more useful Make comparing manifests to catalogs faster by generating an explicit identifier to id mapping Make the identifier to unique ID mapping include databases Convert catalog to use dataclasses/hologram types Fix unit tests to test what we actually care about No changes to integration tests means no need to change dbt docs, hooray --- core/dbt/contracts/graph/compiled.py | 4 - core/dbt/contracts/graph/manifest.py | 32 --- core/dbt/contracts/graph/parsed.py | 4 + core/dbt/helper_types.py | 51 +++- core/dbt/task/generate.py | 338 ++++++++++++++----------- test/unit/test_docs_generate.py | 361 ++++++++++++++++++--------- 6 files changed, 488 insertions(+), 302 deletions(-) diff --git a/core/dbt/contracts/graph/compiled.py b/core/dbt/contracts/graph/compiled.py index 2dc9a700ab9..c4c0fe5ce0d 100644 --- a/core/dbt/contracts/graph/compiled.py +++ b/core/dbt/contracts/graph/compiled.py @@ -1,8 +1,6 @@ from dbt.contracts.graph.parsed import ( ParsedNode, ParsedAnalysisNode, - ParsedDocumentation, - ParsedMacro, ParsedModelNode, ParsedHookNode, ParsedRPCNode, @@ -216,8 +214,6 @@ def parsed_instance_for(compiled: CompiledNode) -> ParsedNode: CompiledSnapshotNode, CompiledTestNode, ParsedAnalysisNode, - ParsedDocumentation, - ParsedMacro, ParsedModelNode, ParsedHookNode, ParsedRPCNode, diff --git a/core/dbt/contracts/graph/manifest.py b/core/dbt/contracts/graph/manifest.py index 652865ed006..fc6d63d7d7b 100644 --- a/core/dbt/contracts/graph/manifest.py +++ b/core/dbt/contracts/graph/manifest.py @@ -367,38 +367,6 @@ def get_resource_fqns(self): return resource_fqns - def _filter_subgraph(self, subgraph, predicate): - """ - Given a subgraph of the manifest, and a predicate, filter - the subgraph using that predicate. Generates a list of nodes. - """ - to_return = [] - - for unique_id, item in subgraph.items(): - if predicate(item): - to_return.append(item) - - return to_return - - def _model_matches_schema_and_table(self, schema, table, model): - if model.resource_type == NodeType.Source: - return (model.schema.lower() == schema.lower() and - model.identifier.lower() == table.lower()) - return (model.schema.lower() == schema.lower() and - model.alias.lower() == table.lower()) - - def get_unique_ids_for_schema_and_table(self, schema, table): - """ - Given a schema and table, find matching models, and return - their unique_ids. A schema and table may have more than one - match if the relation matches both a source and a seed, for instance. - """ - def predicate(model): - return self._model_matches_schema_and_table(schema, table, model) - - matching = list(self._filter_subgraph(self.nodes, predicate)) - return [match.unique_id for match in matching] - def add_nodes(self, new_nodes): """Add the given dict of new nodes to the manifest.""" for unique_id, node in new_nodes.items(): diff --git a/core/dbt/contracts/graph/parsed.py b/core/dbt/contracts/graph/parsed.py index 49dd454ba79..52c05c3403e 100644 --- a/core/dbt/contracts/graph/parsed.py +++ b/core/dbt/contracts/graph/parsed.py @@ -219,6 +219,10 @@ class ParsedNodeMandatory( ): alias: str + @property + def identifier(self): + return self.alias + @dataclass class ParsedNodeDefaults(ParsedNodeMandatory): diff --git a/core/dbt/helper_types.py b/core/dbt/helper_types.py index 1a3120b9c15..6a511f1874f 100644 --- a/core/dbt/helper_types.py +++ b/core/dbt/helper_types.py @@ -1,13 +1,16 @@ # never name this package "types", or mypy will crash in ugly ways +from datetime import timedelta +from typing import NewType + from hologram import ( FieldEncoder, JsonSchemaMixin, JsonDict, ValidationError ) -from datetime import timedelta -from typing import NewType +from dbt.utils import DECIMALS Port = NewType('Port', int) +Bool = NewType('Bool', bool) class PortEncoder(FieldEncoder): @@ -37,7 +40,45 @@ def json_schema(self) -> JsonDict: return {'type': 'number'} -JsonSchemaMixin.register_field_encoders({ +class DecimalEncoder(FieldEncoder): + def to_wire(self, value) -> float: + return float(value) + + def to_python(self, value): + if isinstance(value, DECIMALS): + return value + try: + return DECIMALS[0](value) + except TypeError: + raise ValidationError( + 'cannot encode {} into decimal'.format(value) + ) from None + + @property + def json_schema(self) -> JsonDict: + return {'type': 'number'} + + +class BoolEncoder(FieldEncoder): + def to_wire(self, value) -> bool: + return value + + def to_python(self, value) -> bool: + return value + + @property + def json_schema(self) -> JsonDict: + return {'type': 'boolean'} + + +_ENCODERS = { Port: PortEncoder(), - timedelta: TimeDeltaFieldEncoder() -}) + timedelta: TimeDeltaFieldEncoder(), + Bool: BoolEncoder(), +} + +for decimal_type in DECIMALS: + _ENCODERS[decimal_type] = DecimalEncoder() + + +JsonSchemaMixin.register_field_encoders(_ENCODERS) diff --git a/core/dbt/task/generate.py b/core/dbt/task/generate.py index 51e90790948..2883f8280b5 100644 --- a/core/dbt/task/generate.py +++ b/core/dbt/task/generate.py @@ -1,9 +1,18 @@ import os import shutil +from dataclasses import dataclass +from datetime import datetime +from typing import Union, Dict, List, Optional, Any, NamedTuple + +from hologram import JsonSchemaMixin, ValidationError from dbt.adapters.factory import get_adapter -from dbt.clients.system import write_json +from dbt.contracts.graph.compiled import CompileResultNode +from dbt.contracts.graph.manifest import Manifest +from dbt.contracts.util import Writable, Replaceable from dbt.include.global_project import DOCS_INDEX_FILE_PATH +# this import also ensures we can encode decimals +from dbt.helper_types import Bool import dbt.ui.printer import dbt.utils import dbt.compilation @@ -15,7 +24,7 @@ CATALOG_FILENAME = 'catalog.json' -def get_stripped_prefix(source, prefix): +def get_stripped_prefix(source: Dict[str, Any], prefix: str) -> Dict[str, Any]: """Go through source, extracting every key/value pair where the key starts with the given prefix. """ @@ -26,7 +35,130 @@ def get_stripped_prefix(source, prefix): } -def format_stats(stats): +Primitive = Union[Bool, str, float, None] +PrimitiveDict = Dict[str, Primitive] + + +Key = NamedTuple( + 'Key', + [('database', str), ('schema', str), ('name', str)] +) + + +@dataclass +class StatsItem(JsonSchemaMixin): + id: str + label: str + value: Primitive + description: str + include: bool + + +StatsDict = Dict[str, StatsItem] + + +@dataclass +class ColumnMetadata(JsonSchemaMixin): + type: str + comment: Optional[str] + index: int + name: str + + +ColumnMap = Dict[str, ColumnMetadata] + + +@dataclass +class TableMetadata(JsonSchemaMixin): + type: str + database: str + schema: str + name: str + comment: Optional[str] + owner: Optional[str] + + +@dataclass +class Table(JsonSchemaMixin, Replaceable): + metadata: TableMetadata + columns: ColumnMap + stats: StatsDict + # the same table with two unique IDs will just be listed two times + unique_id: Optional[str] = None + + @classmethod + def from_query_result(cls, data) -> 'Table': + # build the new table's metadata + stats + metadata = TableMetadata.from_dict(get_stripped_prefix(data, 'table_')) + stats = format_stats(get_stripped_prefix(data, 'stats:')) + + return cls( + metadata=metadata, + stats=stats, + columns={}, + ) + + def key(self) -> Key: + return Key( + self.metadata.database.lower(), + self.metadata.schema.lower(), + self.metadata.name.lower(), + ) + + +# keys are database name, schema name, table name +class Catalog(Dict[Key, Table]): + def __init__(self, columns: List[PrimitiveDict]): + super().__init__() + for col in columns: + self.add_column(col) + + def get_table(self, data: PrimitiveDict) -> Table: + try: + key = Key( + str(data['table_database']), + str(data['table_schema']), + str(data['table_name']), + ) + except KeyError as exc: + raise dbt.exceptions.CompilationException( + 'Catalog information missing required key {} (got {})' + .format(exc, data) + ) + if key in self: + table = self[key] + else: + table = Table.from_query_result(data) + self[key] = table + return table + + def add_column(self, data: PrimitiveDict): + table = self.get_table(data) + column_data = get_stripped_prefix(data, 'column_') + # the index should really never be that big so it's ok to end up + # serializing this to JSON (2^53 is the max safe value there) + column_data['index'] = int(column_data['index']) + + column = ColumnMetadata.from_dict(column_data) + table.columns[column.name] = column + + def make_unique_id_map(self, manifest: Manifest) -> Dict[str, Table]: + nodes: Dict[str, Table] = {} + + manifest_mapping = get_unique_id_mapping(manifest) + for table in self.values(): + unique_ids = manifest_mapping.get(table.key(), []) + for unique_id in unique_ids: + if unique_id in nodes: + dbt.exceptions.raise_ambiguous_catalog_match( + unique_id, nodes[unique_id].to_dict(), table.to_dict() + ) + else: + nodes[unique_id] = table.replace(unique_id=unique_id) + return nodes + + +def format_stats(stats: PrimitiveDict) -> StatsDict: """Given a dictionary following this layout: { @@ -41,148 +173,71 @@ def format_stats(stats): 'size:include': True, } - format_stats will convert the dict into this structure: - - { - 'encoded': { - 'id': 'encoded', - 'label': 'Encoded', - 'value': 'Yes', - 'description': 'Indicates if the column is encoded', - 'include': True - }, - 'size': { - 'id': 'size', - 'label': 'Size', - 'value': 128, - 'description': 'Size of the table in MB', - 'include': True - } - } + format_stats will convert the dict into a StatsDict with keys of 'encoded' + and 'size'. """ - stats_collector = {} - for stat_key, stat_value in stats.items(): - stat_id, stat_field = stat_key.split(":") - - stats_collector.setdefault(stat_id, {"id": stat_id}) - stats_collector[stat_id][stat_field] = stat_value - - # strip out all the stats we don't want - stats_collector = { - stat_id: stats - for stat_id, stats in stats_collector.items() - if stats.get('include', False) - } + stats_collector: StatsDict = {} + + base_keys = {k.split(':')[0] for k in stats} + for key in base_keys: + dct: PrimitiveDict = {'id': key} + for subkey in ('label', 'value', 'description', 'include'): + dct[subkey] = stats['{}:{}'.format(key, subkey)] + + try: + stats_item = StatsItem.from_dict(dct) + except ValidationError: + continue + if stats_item.include: + stats_collector[key] = stats_item # we always have a 'has_stats' field, it's never included - has_stats = { - 'id': 'has_stats', - 'label': 'Has Stats?', - 'value': len(stats_collector) > 0, - 'description': 'Indicates whether there are statistics for this table', - 'include': False, - } + has_stats = StatsItem( + id='has_stats', + label='Has Stats?', + value=len(stats_collector) > 0, + description='Indicates whether there are statistics for this table', + include=False, + ) stats_collector['has_stats'] = has_stats return stats_collector -def unflatten(columns): - """Given a list of column dictionaries following this layout: - - [{ - 'column_comment': None, - 'column_index': Decimal('1'), - 'column_name': 'id', - 'column_type': 'integer', - 'table_comment': None, - 'table_name': 'test_table', - 'table_schema': 'test_schema', - 'table_type': 'BASE TABLE' - }] - - unflatten will convert them into a dict with this nested structure: - - { - 'test_schema': { - 'test_table': { - 'metadata': { - 'comment': None, - 'name': 'test_table', - 'type': 'BASE TABLE', - 'schema': 'test_schema', - }, - 'columns': { - "id": { - 'type': 'integer', - 'comment': None, - 'index': 1, - 'name': 'id' - } - } - } - } - } - - Required keys in each column: table_schema, table_name, column_index - - Keys prefixed with 'column_' end up in per-column data and keys prefixed - with 'table_' end up in table metadata. Keys without either prefix are - ignored. - """ - structured = {} - for entry in columns: - schema_name = entry['table_schema'] - table_name = entry['table_name'] - - if schema_name not in structured: - structured[schema_name] = {} - schema = structured[schema_name] +def mapping_key(node: CompileResultNode) -> Key: + return Key( + node.database.lower(), node.schema.lower(), node.identifier.lower() + ) - if table_name not in schema: - metadata = get_stripped_prefix(entry, 'table_') - stats = get_stripped_prefix(entry, 'stats:') - stats_dict = format_stats(stats) - schema[table_name] = { - 'metadata': metadata, - 'stats': stats_dict, - 'columns': {} - } +def get_unique_id_mapping(manifest: Manifest) -> Dict[Key, List[str]]: + # A single relation could have multiple unique IDs pointing to it if a + # source were also a node. + ident_map: Dict[Key, List[str]] = {} + for unique_id, node in manifest.nodes.items(): + key = mapping_key(node) - table = schema[table_name] + if key not in ident_map: + ident_map[key] = [] - column = get_stripped_prefix(entry, 'column_') + ident_map[key].append(unique_id) + return ident_map - # the index should really never be that big so it's ok to end up - # serializing this to JSON (2^53 is the max safe value there) - column['index'] = int(column['index']) - table['columns'][column['name']] = column - return structured +@dataclass +class CatalogResults(JsonSchemaMixin, Writable): + nodes: Dict[str, Table] + generated_at: datetime + _compile_results: Optional[Any] = None -def incorporate_catalog_unique_ids(catalog, manifest): - nodes = {} - for schema, tables in catalog.items(): - for table_name, table_def in tables.items(): - unique_ids = manifest.get_unique_ids_for_schema_and_table( - schema, table_name) - - for unique_id in unique_ids: - if unique_id in nodes: - dbt.exceptions.raise_ambiguous_catalog_match( - unique_id, nodes[unique_id], table_def) - - else: - table_def_copy = table_def.copy() - table_def_copy['unique_id'] = unique_id - nodes[unique_id] = table_def_copy - - return nodes +def _coerce_decimal(value): + if isinstance(value, dbt.utils.DECIMALS): + return float(value) + return value class GenerateTask(CompileTask): - def _get_manifest(self): + def _get_manifest(self) -> Manifest: manifest = dbt.loader.GraphLoader.load_all(self.config) return manifest @@ -194,7 +249,7 @@ def run(self): dbt.ui.printer.print_timestamped_line( 'compile failed, cannot generate docs' ) - return {'compile_results': compile_results} + return CatalogResults({}, datetime.utcnow(), compile_results) shutil.copyfile( DOCS_INDEX_FILE_PATH, @@ -205,33 +260,30 @@ def run(self): manifest = self._get_manifest() dbt.ui.printer.print_timestamped_line("Building catalog") - results = adapter.get_catalog(manifest) + catalog_table = adapter.get_catalog(manifest) - results = [ - dict(zip(results.column_names, row)) - for row in results + catalog_data: List[PrimitiveDict] = [ + dict(zip(catalog_table.column_names, map(_coerce_decimal, row))) + for row in catalog_table ] - nested_results = unflatten(results) - results = { - 'nodes': incorporate_catalog_unique_ids(nested_results, manifest), - 'generated_at': dbt.utils.timestring(), - } + catalog = Catalog(catalog_data) + results = CatalogResults( + nodes=catalog.make_unique_id_map(manifest), + generated_at=datetime.utcnow(), + _compile_results=compile_results, + ) path = os.path.join(self.config.target_path, CATALOG_FILENAME) - write_json(path, results) + results.write(path) dbt.ui.printer.print_timestamped_line( 'Catalog written to {}'.format(os.path.abspath(path)) ) - # now that we've serialized the data we can add compile_results in to - # make interpret_results happy. - results['compile_results'] = compile_results - return results def interpret_results(self, results): - compile_results = results.get('compile_results') + compile_results = results._compile_results if compile_results is None: return True diff --git a/test/unit/test_docs_generate.py b/test/unit/test_docs_generate.py index 88062e426d0..d3afce97bf1 100644 --- a/test/unit/test_docs_generate.py +++ b/test/unit/test_docs_generate.py @@ -1,6 +1,7 @@ +from datetime import datetime from decimal import Decimal +from unittest import mock import unittest -import os import dbt.flags from dbt.task import generate @@ -10,11 +11,35 @@ class GenerateTest(unittest.TestCase): def setUp(self): dbt.flags.STRICT_MODE = True self.maxDiff = None + self.manifest = mock.MagicMock() + self.patcher = mock.patch('dbt.task.generate.get_unique_id_mapping') + self.mock_get_unique_id_mapping = self.patcher.start() + + def tearDown(self): + self.patcher.stop() + + def map_uids(self, effects): + results = { + generate.Key(db, sch, tbl): [uid] + for db, sch, tbl, uid in effects + } + self.mock_get_unique_id_mapping.return_value = results + + def generate_catalog_dict(self, columns): + result = generate.CatalogResults( + nodes=generate.Catalog(columns).make_unique_id_map(self.manifest), + generated_at=datetime.utcnow(), + ) + return result.to_dict(omit_none=False)['nodes'] def test__unflatten_empty(self): columns = {} expected = {} - result = generate.unflatten(columns) + self.map_uids([]) + + result = self.generate_catalog_dict(columns) + + self.mock_get_unique_id_mapping.assert_called_once_with(self.manifest) self.assertEqual(result, expected) def test__unflatten_one_column(self): @@ -26,42 +51,47 @@ def test__unflatten_one_column(self): 'table_comment': None, 'table_name': 'test_table', 'table_schema': 'test_schema', - 'table_type': 'BASE TABLE' + 'table_type': 'BASE TABLE', + 'table_database': 'test_database', }] - expected = { - 'test_schema': { - 'test_table': { - 'metadata': { + 'test.model.test_table': { + 'metadata': { + 'owner': None, + 'comment': None, + 'name': 'test_table', + 'type': 'BASE TABLE', + 'schema': 'test_schema', + 'database': 'test_database', + }, + 'columns': { + 'id': { + 'type': 'integer', 'comment': None, - 'name': 'test_table', - 'type': 'BASE TABLE', - 'schema': 'test_schema', + 'index': 1, + 'name': 'id' }, - 'columns': { - 'id': { - 'type': 'integer', - 'comment': None, - 'index': 1, - 'name': 'id' - }, - }, - 'stats': { - 'has_stats': { - 'id': 'has_stats', - 'label': 'Has Stats?', - 'value': False, - 'description': 'Indicates whether there are statistics for this table', - 'include': False, - }, + }, + 'stats': { + 'has_stats': { + 'id': 'has_stats', + 'label': 'Has Stats?', + 'value': False, + 'description': 'Indicates whether there are statistics for this table', + 'include': False, }, - } - } + }, + 'unique_id': 'test.model.test_table', + }, } - result = generate.unflatten(columns) + self.map_uids([('test_database', 'test_schema', 'test_table', 'test.model.test_table')]) + + result = self.generate_catalog_dict(columns) + + self.mock_get_unique_id_mapping.assert_called_once_with(self.manifest) self.assertEqual(result, expected) - def test__unflatten_multiple_schemas(self): + def test__unflatten_multiple_schemas_dbs(self): columns = [ { 'column_comment': None, @@ -71,7 +101,9 @@ def test__unflatten_multiple_schemas(self): 'table_comment': None, 'table_name': 'test_table', 'table_schema': 'test_schema', - 'table_type': 'BASE TABLE' + 'table_type': 'BASE TABLE', + 'table_database': 'test_database', + 'table_owner': None, }, { 'column_comment': None, @@ -81,7 +113,9 @@ def test__unflatten_multiple_schemas(self): 'table_comment': None, 'table_name': 'test_table', 'table_schema': 'test_schema', - 'table_type': 'BASE TABLE' + 'table_type': 'BASE TABLE', + 'table_database': 'test_database', + 'table_owner': None, }, { 'column_comment': None, @@ -92,6 +126,8 @@ def test__unflatten_multiple_schemas(self): 'table_name': 'other_test_table', 'table_schema': 'test_schema', 'table_type': 'BASE TABLE', + 'table_database': 'test_database', + 'table_owner': None, }, { 'column_comment': None, @@ -102,6 +138,8 @@ def test__unflatten_multiple_schemas(self): 'table_name': 'other_test_table', 'table_schema': 'test_schema', 'table_type': 'BASE TABLE', + 'table_database': 'test_database', + 'table_owner': None, }, { 'column_comment': None, @@ -111,7 +149,9 @@ def test__unflatten_multiple_schemas(self): 'table_comment': None, 'table_name': 'test_table', 'table_schema': 'other_test_schema', - 'table_type': 'BASE TABLE' + 'table_type': 'BASE TABLE', + 'table_database': 'test_database', + 'table_owner': None, }, { 'column_comment': None, @@ -121,108 +161,193 @@ def test__unflatten_multiple_schemas(self): 'table_comment': None, 'table_name': 'test_table', 'table_schema': 'other_test_schema', - 'table_type': 'BASE TABLE' + 'table_type': 'BASE TABLE', + 'table_database': 'test_database', + 'table_owner': None, + }, + { + 'column_comment': None, + 'column_index': Decimal('1'), + 'column_name': 'id', + 'column_type': 'integer', + 'table_comment': None, + 'table_name': 'test_table', + 'table_schema': 'test_schema', + 'table_type': 'BASE TABLE', + 'table_database': 'other_test_database', + 'table_owner': None, + }, + { + 'column_comment': None, + 'column_index': Decimal('2'), + 'column_name': 'name', + 'column_type': 'text', + 'table_comment': None, + 'table_name': 'test_table', + 'table_schema': 'test_schema', + 'table_type': 'BASE TABLE', + 'table_database': 'other_test_database', + 'table_owner': None, }, ] - expected = { - 'test_schema': { - 'test_table': { - 'metadata': { + 'test.model.test_table': { + 'metadata': { + 'owner': None, + 'comment': None, + 'name': 'test_table', + 'type': 'BASE TABLE', + 'schema': 'test_schema', + 'database': 'test_database', + }, + 'columns': { + 'id': { + 'type': 'integer', 'comment': None, - 'name': 'test_table', - 'type': 'BASE TABLE', - 'schema': 'test_schema', + 'index': 1, + 'name': 'id' }, - 'columns': { - 'id': { - 'type': 'integer', - 'comment': None, - 'index': 1, - 'name': 'id' - }, - 'name': { - 'type': 'text', - 'comment': None, - 'index': Decimal('2'), - 'name': 'name', - } - }, - 'stats': { - 'has_stats': { - 'id': 'has_stats', - 'label': 'Has Stats?', - 'value': False, - 'description': 'Indicates whether there are statistics for this table', - 'include': False, - }, + 'name': { + 'type': 'text', + 'comment': None, + 'index': 2, + 'name': 'name', + } + }, + 'stats': { + 'has_stats': { + 'id': 'has_stats', + 'label': 'Has Stats?', + 'value': False, + 'description': 'Indicates whether there are statistics for this table', + 'include': False, }, }, - 'other_test_table': { - 'metadata': { + 'unique_id': 'test.model.test_table', + }, + 'test.model.other_test_table': { + 'metadata': { + 'owner': None, + 'comment': None, + 'name': 'other_test_table', + 'type': 'BASE TABLE', + 'schema': 'test_schema', + 'database': 'test_database', + }, + 'columns': { + 'id': { + 'type': 'integer', 'comment': None, - 'name': 'other_test_table', - 'type': 'BASE TABLE', - 'schema': 'test_schema', + 'index': 1, + 'name': 'id' }, - 'columns': { - 'id': { - 'type': 'integer', - 'comment': None, - 'index': 1, - 'name': 'id' - }, - 'email': { - 'type': 'character varying', - 'comment': None, - 'index': Decimal('2'), - 'name': 'email', - } - }, - 'stats': { - 'has_stats': { - 'id': 'has_stats', - 'label': 'Has Stats?', - 'value': False, - 'description': 'Indicates whether there are statistics for this table', - 'include': False, - }, + 'email': { + 'type': 'character varying', + 'comment': None, + 'index': 2, + 'name': 'email', + } + }, + 'stats': { + 'has_stats': { + 'id': 'has_stats', + 'label': 'Has Stats?', + 'value': False, + 'description': 'Indicates whether there are statistics for this table', + 'include': False, }, }, + 'unique_id': 'test.model.other_test_table', }, - 'other_test_schema': { - 'test_table': { - 'metadata': { + 'test.model.test_table_otherschema': { + 'metadata': { + 'owner': None, + 'comment': None, + 'name': 'test_table', + 'type': 'BASE TABLE', + 'schema': 'other_test_schema', + 'database': 'test_database', + }, + 'columns': { + 'id': { + 'type': 'integer', 'comment': None, - 'name': 'test_table', - 'type': 'BASE TABLE', - 'schema': 'other_test_schema', + 'index': 1, + 'name': 'id' }, - 'columns': { - 'id': { - 'type': 'integer', - 'comment': None, - 'index': 1, - 'name': 'id' - }, - 'name': { - 'type': 'text', - 'comment': None, - 'index': Decimal('2'), - 'name': 'name', - } + 'name': { + 'type': 'text', + 'comment': None, + 'index': 2, + 'name': 'name', + } + }, + 'stats': { + 'has_stats': { + 'id': 'has_stats', + 'label': 'Has Stats?', + 'value': False, + 'description': 'Indicates whether there are statistics for this table', + 'include': False, }, - 'stats': { - 'has_stats': { - 'id': 'has_stats', - 'label': 'Has Stats?', - 'value': False, - 'description': 'Indicates whether there are statistics for this table', - 'include': False, - }, + }, + 'unique_id': 'test.model.test_table_otherschema', + }, + 'test.model.test_table_otherdb': { + 'metadata': { + 'owner': None, + 'comment': None, + 'name': 'test_table', + 'type': 'BASE TABLE', + 'schema': 'test_schema', + 'database': 'other_test_database', + }, + 'columns': { + 'id': { + 'type': 'integer', + 'comment': None, + 'index': 1, + 'name': 'id' }, + 'name': { + 'type': 'text', + 'comment': None, + 'index': 2, + 'name': 'name', + } }, + 'stats': { + 'has_stats': { + 'id': 'has_stats', + 'label': 'Has Stats?', + 'value': False, + 'description': 'Indicates whether there are statistics for this table', + 'include': False, + }, + }, + 'unique_id': 'test.model.test_table_otherdb', } } - result = generate.unflatten(columns) + self.map_uids([ + ( + 'test_database', 'test_schema', 'test_table', + 'test.model.test_table' + ), + ( + 'test_database', 'test_schema', 'other_test_table', + 'test.model.other_test_table' + ), + ( + 'test_database', 'other_test_schema', 'test_table', + 'test.model.test_table_otherschema' + ), + ( + 'other_test_database', 'test_schema', 'test_table', + 'test.model.test_table_otherdb' + ), + ]) + + result = self.generate_catalog_dict(columns) + + self.mock_get_unique_id_mapping.assert_called_once_with(self.manifest) self.assertEqual(result, expected)