From 07e4c5549896ddb9c3c82b938d29a9d11ff6cda2 Mon Sep 17 00:00:00 2001 From: Fran Lozano Date: Tue, 13 Apr 2021 23:28:26 +0200 Subject: [PATCH 01/10] Parse information returned by show table extended --- dbt/adapters/spark/impl.py | 44 +++++++++++++++++++++++++++------- dbt/adapters/spark/relation.py | 1 + 2 files changed, 37 insertions(+), 8 deletions(-) diff --git a/dbt/adapters/spark/impl.py b/dbt/adapters/spark/impl.py index a12e7a940..76ab27a01 100644 --- a/dbt/adapters/spark/impl.py +++ b/dbt/adapters/spark/impl.py @@ -1,3 +1,4 @@ +import re from concurrent.futures import Future from dataclasses import dataclass from typing import Optional, List, Dict, Any, Union, Iterable @@ -60,6 +61,8 @@ class SparkAdapter(SQLAdapter): 'stats:rows:description', 'stats:rows:include', ) + INFORMATION_COLUMNS_REGEX = re.compile(r"\|-- (.*): (.*) \(nullable = (.*)\b", re.MULTILINE) + INFORMATION_OWNER_REGEX = re.compile(r"^Owner: (.*)$", re.MULTILINE) Relation = SparkRelation Column = SparkColumn @@ -139,7 +142,8 @@ def list_relations_without_caching( schema=_schema, identifier=name, type=rel_type, - is_delta=is_delta + information=information, + is_delta=is_delta, ) relations.append(relation) @@ -197,19 +201,43 @@ def find_table_information_separator(rows: List[dict]) -> int: return pos def get_columns_in_relation(self, relation: Relation) -> List[SparkColumn]: - rows: List[agate.Row] = super().get_columns_in_relation(relation) - return self.parse_describe_extended(relation, rows) + cached_relations = self.cache.get_relations(relation.database, relation.schema) + cached_relation = next((cached_relation + for cached_relation in cached_relations + if str(cached_relation) == str(relation)), None) + if cached_relations is None: + rows: List[agate.Row] = super().get_columns_in_relation(relation) + columns = self.parse_describe_extended(relation, rows) + else: + columns = self.get_columns_from_information(cached_relation) + return columns + + def get_columns_from_information(self, relation: SparkRelation) -> List[SparkColumn]: + owner_match = re.findall(self.INFORMATION_OWNER_REGEX, relation.information) + owner = owner_match[0] if owner_match else None + matches = re.finditer(self.INFORMATION_COLUMNS_REGEX, relation.information) + columns = [] + for match_num, match in enumerate(matches, start=1): + column_name, column_type, nullable = match.groups() + column = SparkColumn( + table_database=None, + table_schema=relation.schema, + table_name=relation.table, + table_type=relation.type, + column_index=match_num, + table_owner=owner, + column=column_name, + dtype=column_type + ) + columns.append(column) + return columns def _get_columns_for_catalog( self, relation: SparkRelation ) -> Iterable[Dict[str, Any]]: - properties = self.get_properties(relation) - columns = self.get_columns_in_relation(relation) - owner = properties.get(KEY_TABLE_OWNER) + columns = self.get_columns_from_information(relation) for column in columns: - if owner: - column.table_owner = owner # convert SparkColumns into catalog dicts as_dict = column.to_column_dict() as_dict['column_name'] = as_dict.pop('column', None) diff --git a/dbt/adapters/spark/relation.py b/dbt/adapters/spark/relation.py index 507f51d3b..5fc096550 100644 --- a/dbt/adapters/spark/relation.py +++ b/dbt/adapters/spark/relation.py @@ -26,6 +26,7 @@ class SparkRelation(BaseRelation): include_policy: SparkIncludePolicy = SparkIncludePolicy() quote_character: str = '`' is_delta: Optional[bool] = None + information: str = None def __post_init__(self): if self.database != self.schema and self.database: From d98588d768ca97f9df339ccb48679c19afca2e66 Mon Sep 17 00:00:00 2001 From: Fran Lozano Date: Tue, 13 Apr 2021 23:42:17 +0200 Subject: [PATCH 02/10] Fix linter errors --- dbt/adapters/spark/impl.py | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/dbt/adapters/spark/impl.py b/dbt/adapters/spark/impl.py index 76ab27a01..433dbbbbc 100644 --- a/dbt/adapters/spark/impl.py +++ b/dbt/adapters/spark/impl.py @@ -61,7 +61,8 @@ class SparkAdapter(SQLAdapter): 'stats:rows:description', 'stats:rows:include', ) - INFORMATION_COLUMNS_REGEX = re.compile(r"\|-- (.*): (.*) \(nullable = (.*)\b", re.MULTILINE) + INFORMATION_COLUMNS_REGEX = re.compile( + r"\|-- (.*): (.*) \(nullable = (.*)\b", re.MULTILINE) INFORMATION_OWNER_REGEX = re.compile(r"^Owner: (.*)$", re.MULTILINE) Relation = SparkRelation @@ -201,10 +202,12 @@ def find_table_information_separator(rows: List[dict]) -> int: return pos def get_columns_in_relation(self, relation: Relation) -> List[SparkColumn]: - cached_relations = self.cache.get_relations(relation.database, relation.schema) + cached_relations = self.cache.get_relations( + relation.database, relation.schema) cached_relation = next((cached_relation for cached_relation in cached_relations - if str(cached_relation) == str(relation)), None) + if str(cached_relation) == str(relation)), + None) if cached_relations is None: rows: List[agate.Row] = super().get_columns_in_relation(relation) columns = self.parse_describe_extended(relation, rows) @@ -212,10 +215,14 @@ def get_columns_in_relation(self, relation: Relation) -> List[SparkColumn]: columns = self.get_columns_from_information(cached_relation) return columns - def get_columns_from_information(self, relation: SparkRelation) -> List[SparkColumn]: - owner_match = re.findall(self.INFORMATION_OWNER_REGEX, relation.information) + def get_columns_from_information( + self, relation: SparkRelation + ) -> List[SparkColumn]: + owner_match = re.findall( + self.INFORMATION_OWNER_REGEX, relation.information) owner = owner_match[0] if owner_match else None - matches = re.finditer(self.INFORMATION_COLUMNS_REGEX, relation.information) + matches = re.finditer( + self.INFORMATION_COLUMNS_REGEX, relation.information) columns = [] for match_num, match in enumerate(matches, start=1): column_name, column_type, nullable = match.groups() From fe50b05a12f5f7da02c4b00343410252786c06e7 Mon Sep 17 00:00:00 2001 From: Fran Lozano Date: Wed, 14 Apr 2021 00:13:34 +0200 Subject: [PATCH 03/10] Add logic when relation is None --- dbt/adapters/spark/impl.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/spark/impl.py b/dbt/adapters/spark/impl.py index 433dbbbbc..5a386214e 100644 --- a/dbt/adapters/spark/impl.py +++ b/dbt/adapters/spark/impl.py @@ -242,11 +242,19 @@ def get_columns_from_information( def _get_columns_for_catalog( self, relation: SparkRelation ) -> Iterable[Dict[str, Any]]: - columns = self.get_columns_from_information(relation) + if relation and relation.information is not None: + columns = self.get_columns_from_information(relation) + owner = None + else: + properties = self.get_properties(relation) + columns = self.get_columns_in_relation(relation) + owner = properties.get(KEY_TABLE_OWNER) for column in columns: # convert SparkColumns into catalog dicts as_dict = column.to_column_dict() + if owner: + column.table_owner = owner as_dict['column_name'] = as_dict.pop('column', None) as_dict['column_type'] = as_dict.pop('dtype') as_dict['table_database'] = None From 2e307f6dee5568a7f7a5c952e7746d1023b11b10 Mon Sep 17 00:00:00 2001 From: Fran Lozano Date: Wed, 14 Apr 2021 00:18:31 +0200 Subject: [PATCH 04/10] Revert previous commit and fix bug --- dbt/adapters/spark/impl.py | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/dbt/adapters/spark/impl.py b/dbt/adapters/spark/impl.py index 5a386214e..aff6e5f4b 100644 --- a/dbt/adapters/spark/impl.py +++ b/dbt/adapters/spark/impl.py @@ -208,7 +208,7 @@ def get_columns_in_relation(self, relation: Relation) -> List[SparkColumn]: for cached_relation in cached_relations if str(cached_relation) == str(relation)), None) - if cached_relations is None: + if cached_relation is None: rows: List[agate.Row] = super().get_columns_in_relation(relation) columns = self.parse_describe_extended(relation, rows) else: @@ -242,19 +242,11 @@ def get_columns_from_information( def _get_columns_for_catalog( self, relation: SparkRelation ) -> Iterable[Dict[str, Any]]: - if relation and relation.information is not None: - columns = self.get_columns_from_information(relation) - owner = None - else: - properties = self.get_properties(relation) - columns = self.get_columns_in_relation(relation) - owner = properties.get(KEY_TABLE_OWNER) + columns = self.get_columns_from_information(relation) for column in columns: # convert SparkColumns into catalog dicts as_dict = column.to_column_dict() - if owner: - column.table_owner = owner as_dict['column_name'] = as_dict.pop('column', None) as_dict['column_type'] = as_dict.pop('dtype') as_dict['table_database'] = None From 3b54482267399355e322ebcefe6326418c415920 Mon Sep 17 00:00:00 2001 From: Fran Lozano Date: Wed, 14 Apr 2021 16:00:16 +0200 Subject: [PATCH 05/10] Rename method and add unit test --- dbt/adapters/spark/impl.py | 6 ++--- test/unit/test_adapter.py | 52 ++++++++++++++++++++++++++++++++++++++ 2 files changed, 55 insertions(+), 3 deletions(-) diff --git a/dbt/adapters/spark/impl.py b/dbt/adapters/spark/impl.py index aff6e5f4b..a0dc624d1 100644 --- a/dbt/adapters/spark/impl.py +++ b/dbt/adapters/spark/impl.py @@ -212,10 +212,10 @@ def get_columns_in_relation(self, relation: Relation) -> List[SparkColumn]: rows: List[agate.Row] = super().get_columns_in_relation(relation) columns = self.parse_describe_extended(relation, rows) else: - columns = self.get_columns_from_information(cached_relation) + columns = self.parse_columns_from_information(cached_relation) return columns - def get_columns_from_information( + def parse_columns_from_information( self, relation: SparkRelation ) -> List[SparkColumn]: owner_match = re.findall( @@ -242,7 +242,7 @@ def get_columns_from_information( def _get_columns_for_catalog( self, relation: SparkRelation ) -> Iterable[Dict[str, Any]]: - columns = self.get_columns_from_information(relation) + columns = self.parse_columns_from_information(relation) for column in columns: # convert SparkColumns into catalog dicts diff --git a/test/unit/test_adapter.py b/test/unit/test_adapter.py index 5e50e3100..f0aa06b3a 100644 --- a/test/unit/test_adapter.py +++ b/test/unit/test_adapter.py @@ -446,3 +446,55 @@ def test_profile_with_cluster_and_sql_endpoint(self): } with self.assertRaises(RuntimeException): config_from_parts_or_dicts(self.project_cfg, profile) + + def test_parse_columns_from_information_with_table_type(self): + self.maxDiff = None + rel_type = SparkRelation.get_relation_type.Table + + # Mimics the output of Spark in the information column + information = ( + "Database: default_schema\n" + "Table: mytable\n" + "Owner: root\n" + "Created Time: Wed Feb 04 18:15:00 UTC 1815\n" + "Last Access: Wed May 20 19:25:00 UTC 1925\n" + "Created By: Spark 3.0.1\n" + "Type: MANAGED\n" + "Provider: delta\n" + "Statistics: 123456789 bytes\n" + "Location: /mnt/vo\n" + "Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe\n" + "InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat\n" + "OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat\n" + "Partition Provider: Catalog\n" + "Partition Columns: [`dt`]\n" + "Schema: root\n" + " |-- col1: decimal(22,0) (nullable = true)\n" + " |-- col2: string (nullable = true)\n" + " |-- dt: date (nullable = true)\n" + ) + relation = SparkRelation.create( + schema='default_schema', + identifier='mytable', + type=rel_type, + information=information + ) + + config = self._get_target_http(self.project_cfg) + columns = SparkAdapter(config).parse_columns_from_information( + relation) + self.assertEqual(len(columns), 3) + self.assertEqual(columns[0].to_column_dict(omit_none=False), { + 'table_database': None, + 'table_schema': relation.schema, + 'table_name': relation.name, + 'table_type': rel_type, + 'table_owner': 'root', + 'column': 'col1', + 'column_index': 0, + 'dtype': 'decimal(22,0)', + 'numeric_scale': None, + 'numeric_precision': None, + 'char_size': None + }) + From e1bf65437a81d2823dcdae2178c4ec1b30396721 Mon Sep 17 00:00:00 2001 From: Fran Lozano Date: Wed, 14 Apr 2021 16:04:26 +0200 Subject: [PATCH 06/10] Fix bug in column_index --- dbt/adapters/spark/impl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbt/adapters/spark/impl.py b/dbt/adapters/spark/impl.py index a0dc624d1..950fc6760 100644 --- a/dbt/adapters/spark/impl.py +++ b/dbt/adapters/spark/impl.py @@ -224,7 +224,7 @@ def parse_columns_from_information( matches = re.finditer( self.INFORMATION_COLUMNS_REGEX, relation.information) columns = [] - for match_num, match in enumerate(matches, start=1): + for match_num, match in enumerate(matches): column_name, column_type, nullable = match.groups() column = SparkColumn( table_database=None, From 64870e9f35c1de6ade4e612b38609d0855d9359a Mon Sep 17 00:00:00 2001 From: Fran Lozano Date: Wed, 14 Apr 2021 16:16:09 +0200 Subject: [PATCH 07/10] Add test with view --- test/unit/test_adapter.py | 59 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 59 insertions(+) diff --git a/test/unit/test_adapter.py b/test/unit/test_adapter.py index f0aa06b3a..523a5447d 100644 --- a/test/unit/test_adapter.py +++ b/test/unit/test_adapter.py @@ -498,3 +498,62 @@ def test_parse_columns_from_information_with_table_type(self): 'char_size': None }) + def test_parse_columns_from_information_with_view_type(self): + self.maxDiff = None + rel_type = SparkRelation.get_relation_type.View + information = ( + "Database: default_schema\n" + "Table: myview\n" + "Owner: root\n" + "Created Time: Wed Feb 04 18:15:00 UTC 1815\n" + "Last Access: UNKNOWN\n" + "Created By: Spark 3.0.1\n" + "Type: VIEW\n" + "View Text: WITH base (\n" + " SELECT * FROM source_table\n" + ")\n" + "SELECT col1, col2, dt FROM base\n" + "View Original Text: WITH base (\n" + " SELECT * FROM source_table\n" + ")\n" + "SELECT col1, col2, dt FROM base\n" + "View Catalog and Namespace: spark_catalog.default\n" + "View Query Output Columns: [col1, col2, dt]\n" + "Table Properties: [view.query.out.col.1=col1, view.query.out.col.2=col2, " + "transient_lastDdlTime=1618324324, view.query.out.col.3=dt, " + "view.catalogAndNamespace.part.0=spark_catalog, " + "view.catalogAndNamespace.part.1=default]\n" + "Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe\n" + "InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat\n" + "OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat\n" + "Storage Properties: [serialization.format=1]\n" + "Schema: root\n" + " |-- col1: decimal(22,0) (nullable = true)\n" + " |-- col2: string (nullable = true)\n" + " |-- dt: date (nullable = true)\n" + ) + relation = SparkRelation.create( + schema='default_schema', + identifier='myview', + type=rel_type, + information=information + ) + + config = self._get_target_http(self.project_cfg) + columns = SparkAdapter(config).parse_columns_from_information( + relation) + self.assertEqual(len(columns), 3) + self.assertEqual(columns[1].to_column_dict(omit_none=False), { + 'table_database': None, + 'table_schema': relation.schema, + 'table_name': relation.name, + 'table_type': rel_type, + 'table_owner': 'root', + 'column': 'col2', + 'column_index': 1, + 'dtype': 'string', + 'numeric_scale': None, + 'numeric_precision': None, + 'char_size': None + }) + From 2c6a5d871874616d4ba1badfc12083163b1f060c Mon Sep 17 00:00:00 2001 From: Fran Lozano Date: Wed, 14 Apr 2021 16:32:55 +0200 Subject: [PATCH 08/10] Update CHANGELOG.md --- CHANGELOG.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 135df3fbe..9075551c0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,8 +4,14 @@ - Cast `table_owner` to string to avoid errors generating docs ([#158](https://github.com/fishtown-analytics/dbt-spark/pull/158), [#159](https://github.com/fishtown-analytics/dbt-spark/pull/159)) +### Under the hood + +- Parse information returned by `list_relations_without_caching` macro to speed up catalog generation ([#93](https://github.com/fishtown-analytics/dbt-spark/issues/93), [#160](https://github.com/fishtown-analytics/dbt-spark/pull/160)) + ### Contributors - [@friendofasquid](https://github.com/friendofasquid) ([#159](https://github.com/fishtown-analytics/dbt-spark/pull/159)) +- [@franloza](https://github.com/franloza) ([#160](https://github.com/fishtown-analytics/dbt-spark/pull/160)) + ## dbt-spark 0.19.1 (Release TBD) From 7612fcb89c801732ccd4e958fb7d9fbd2908662a Mon Sep 17 00:00:00 2001 From: Fran Lozano Date: Thu, 15 Apr 2021 20:05:42 +0200 Subject: [PATCH 09/10] Parse statistics --- dbt/adapters/spark/impl.py | 8 ++++- test/unit/test_adapter.py | 67 ++++++++++++++++++++++++++++++++++++-- test/unit/test_column.py | 38 +++++++++++++++++++++ 3 files changed, 110 insertions(+), 3 deletions(-) create mode 100644 test/unit/test_column.py diff --git a/dbt/adapters/spark/impl.py b/dbt/adapters/spark/impl.py index 950fc6760..661fc162a 100644 --- a/dbt/adapters/spark/impl.py +++ b/dbt/adapters/spark/impl.py @@ -64,6 +64,7 @@ class SparkAdapter(SQLAdapter): INFORMATION_COLUMNS_REGEX = re.compile( r"\|-- (.*): (.*) \(nullable = (.*)\b", re.MULTILINE) INFORMATION_OWNER_REGEX = re.compile(r"^Owner: (.*)$", re.MULTILINE) + INFORMATION_STATISTICS_REGEX = re.compile(r"^Statistics: (.*)$", re.MULTILINE) Relation = SparkRelation Column = SparkColumn @@ -224,6 +225,10 @@ def parse_columns_from_information( matches = re.finditer( self.INFORMATION_COLUMNS_REGEX, relation.information) columns = [] + stats_match = re.findall( + self.INFORMATION_STATISTICS_REGEX, relation.information) + raw_table_stats = stats_match[0] if stats_match else None + table_stats = SparkColumn.convert_table_stats(raw_table_stats) for match_num, match in enumerate(matches): column_name, column_type, nullable = match.groups() column = SparkColumn( @@ -234,7 +239,8 @@ def parse_columns_from_information( column_index=match_num, table_owner=owner, column=column_name, - dtype=column_type + dtype=column_type, + table_stats=table_stats ) columns.append(column) return columns diff --git a/test/unit/test_adapter.py b/test/unit/test_adapter.py index 523a5447d..d886ddee3 100644 --- a/test/unit/test_adapter.py +++ b/test/unit/test_adapter.py @@ -447,7 +447,7 @@ def test_profile_with_cluster_and_sql_endpoint(self): with self.assertRaises(RuntimeException): config_from_parts_or_dicts(self.project_cfg, profile) - def test_parse_columns_from_information_with_table_type(self): + def test_parse_columns_from_information_with_table_type_and_delta_provider(self): self.maxDiff = None rel_type = SparkRelation.get_relation_type.Table @@ -495,7 +495,12 @@ def test_parse_columns_from_information_with_table_type(self): 'dtype': 'decimal(22,0)', 'numeric_scale': None, 'numeric_precision': None, - 'char_size': None + 'char_size': None, + + 'stats:bytes:description': '', + 'stats:bytes:include': True, + 'stats:bytes:label': 'bytes', + 'stats:bytes:value': 123456789, }) def test_parse_columns_from_information_with_view_type(self): @@ -557,3 +562,61 @@ def test_parse_columns_from_information_with_view_type(self): 'char_size': None }) + def test_parse_columns_from_information_with_table_type_and_parquet_provider(self): + self.maxDiff = None + rel_type = SparkRelation.get_relation_type.Table + + information = ( + "Database: default_schema\n" + "Table: mytable\n" + "Owner: root\n" + "Created Time: Wed Feb 04 18:15:00 UTC 1815\n" + "Last Access: Wed May 20 19:25:00 UTC 1925\n" + "Created By: Spark 3.0.1\n" + "Type: MANAGED\n" + "Provider: parquet\n" + "Statistics: 1234567890 bytes, 12345678 rows\n" + "Location: /mnt/vo\n" + "Serde Library: org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe\n" + "InputFormat: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat\n" + "OutputFormat: org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat\n" + "Schema: root\n" + " |-- col1: decimal(22,0) (nullable = true)\n" + " |-- col2: string (nullable = true)\n" + " |-- dt: date (nullable = true)\n" + ) + relation = SparkRelation.create( + schema='default_schema', + identifier='mytable', + type=rel_type, + information=information + ) + + config = self._get_target_http(self.project_cfg) + columns = SparkAdapter(config).parse_columns_from_information( + relation) + self.assertEqual(len(columns), 3) + self.assertEqual(columns[2].to_column_dict(omit_none=False), { + 'table_database': None, + 'table_schema': relation.schema, + 'table_name': relation.name, + 'table_type': rel_type, + 'table_owner': 'root', + 'column': 'dt', + 'column_index': 2, + 'dtype': 'date', + 'numeric_scale': None, + 'numeric_precision': None, + 'char_size': None, + + 'stats:bytes:description': '', + 'stats:bytes:include': True, + 'stats:bytes:label': 'bytes', + 'stats:bytes:value': 1234567890, + + 'stats:rows:description': '', + 'stats:rows:include': True, + 'stats:rows:label': 'rows', + 'stats:rows:value': 12345678 + }) + diff --git a/test/unit/test_column.py b/test/unit/test_column.py new file mode 100644 index 000000000..f7f8d8776 --- /dev/null +++ b/test/unit/test_column.py @@ -0,0 +1,38 @@ +import unittest + +from dbt.adapters.spark import SparkColumn + + +class TestSparkColumn(unittest.TestCase): + + def test_convert_table_stats_with_no_statistics(self): + self.assertDictEqual( + SparkColumn.convert_table_stats(None), + {} + ) + + def test_convert_table_stats_with_bytes(self): + self.assertDictEqual( + SparkColumn.convert_table_stats("123456789 bytes"), + { + 'stats:bytes:description': '', + 'stats:bytes:include': True, + 'stats:bytes:label': 'bytes', + 'stats:bytes:value': 123456789 + } + ) + + def test_convert_table_stats_with_bytes_and_rows(self): + self.assertDictEqual( + SparkColumn.convert_table_stats("1234567890 bytes, 12345678 rows"), + { + 'stats:bytes:description': '', + 'stats:bytes:include': True, + 'stats:bytes:label': 'bytes', + 'stats:bytes:value': 1234567890, + 'stats:rows:description': '', + 'stats:rows:include': True, + 'stats:rows:label': 'rows', + 'stats:rows:value': 12345678 + } + ) From 36367e6ccef3afcb76056dc02c95b8e248ec6ff9 Mon Sep 17 00:00:00 2001 From: Fran Lozano Date: Thu, 15 Apr 2021 22:38:14 +0200 Subject: [PATCH 10/10] Fix inter errors --- dbt/adapters/spark/impl.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbt/adapters/spark/impl.py b/dbt/adapters/spark/impl.py index 661fc162a..93cfba687 100644 --- a/dbt/adapters/spark/impl.py +++ b/dbt/adapters/spark/impl.py @@ -64,7 +64,8 @@ class SparkAdapter(SQLAdapter): INFORMATION_COLUMNS_REGEX = re.compile( r"\|-- (.*): (.*) \(nullable = (.*)\b", re.MULTILINE) INFORMATION_OWNER_REGEX = re.compile(r"^Owner: (.*)$", re.MULTILINE) - INFORMATION_STATISTICS_REGEX = re.compile(r"^Statistics: (.*)$", re.MULTILINE) + INFORMATION_STATISTICS_REGEX = re.compile( + r"^Statistics: (.*)$", re.MULTILINE) Relation = SparkRelation Column = SparkColumn