Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Parse information returned by list_relations_without_caching macro to speed up catalog generation #160

Merged
merged 10 commits into from
Apr 17, 2021
Merged
Show file tree
Hide file tree
Changes from 8 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down
51 changes: 43 additions & 8 deletions dbt/adapters/spark/impl.py
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
import re
from concurrent.futures import Future
from dataclasses import dataclass
from typing import Optional, List, Dict, Any, Union, Iterable
Expand Down Expand Up @@ -60,6 +61,9 @@ 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)
Comment on lines +64 to +66
Copy link
Contributor

@jtcohen6 jtcohen6 Apr 14, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

While we're here... any chance I could convince you to regex for stats as well? :)

INFORMATION_STATISTICS_REGEX = re.compile('^Statistics: (.*)$", re.MULTILINE)

From some (anecdotal) testing, delta tables include a line like Statistics: 1109049927 bytes, whereas (e.g.) parquet tables include a line like Statistics: 1109049927 bytes, 14093476 rows.

The SparkColumn object takes a table_stats argument; we'd just need to adjust convert_table_stats to handle the delta table case, which is missing the rows bit after the comma split.

Alternatively, if you wanted to do it all in regex, I guess you could:

INFORMATION_STATISTICS_REGEX = re.compile('^Statistics: ([0-9]+) bytes(, ([0-9]+) rows)?', re.MULTILINE)
bytes, _, rows = re.findall(self.INFORMATION_OWNER_REGEX, relation.information)

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jtcohen6 This is a great idea! This is useful information and I think it's worth trying to parse it. I'll give it a try and add an extra test for parquet tables. I'll let you know when I have this implemented. (Switching back to WIP)


Relation = SparkRelation
Column = SparkColumn
Expand Down Expand Up @@ -139,7 +143,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)

Expand Down Expand Up @@ -197,19 +202,49 @@ 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_relation is None:
rows: List[agate.Row] = super().get_columns_in_relation(relation)
columns = self.parse_describe_extended(relation, rows)
else:
columns = self.parse_columns_from_information(cached_relation)
return columns

def parse_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):
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.parse_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)
Expand Down
1 change: 1 addition & 0 deletions dbt/adapters/spark/relation.py
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
111 changes: 111 additions & 0 deletions test/unit/test_adapter.py
Original file line number Diff line number Diff line change
Expand Up @@ -446,3 +446,114 @@ 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
})

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
})