-
Notifications
You must be signed in to change notification settings - Fork 375
Expand file tree
/
Copy pathdatabricks.py
More file actions
368 lines (322 loc) · 14.3 KB
/
databricks.py
File metadata and controls
368 lines (322 loc) · 14.3 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
from __future__ import annotations
import logging
import typing as t
from functools import partial
from sqlglot import exp
from sqlmesh.core.dialect import to_schema
from sqlmesh.core.engine_adapter.shared import (
CatalogSupport,
DataObject,
DataObjectType,
InsertOverwriteStrategy,
SourceQuery,
)
from sqlmesh.core.engine_adapter.spark import SparkEngineAdapter
from sqlmesh.core.node import IntervalUnit
from sqlmesh.core.schema_diff import NestedSupport
from sqlmesh.engines.spark.db_api.spark_session import connection, SparkSessionConnection
from sqlmesh.utils.errors import SQLMeshError, MissingDefaultCatalogError
if t.TYPE_CHECKING:
import pandas as pd
from sqlmesh.core._typing import SchemaName, TableName, SessionProperties
from sqlmesh.core.engine_adapter._typing import DF, PySparkSession, Query
logger = logging.getLogger(__name__)
class DatabricksEngineAdapter(SparkEngineAdapter):
DIALECT = "databricks"
INSERT_OVERWRITE_STRATEGY = InsertOverwriteStrategy.REPLACE_WHERE
SUPPORTS_CLONING = True
SUPPORTS_MATERIALIZED_VIEWS = True
SUPPORTS_MATERIALIZED_VIEW_SCHEMA = True
SCHEMA_DIFFER_KWARGS = {
"support_positional_add": True,
"nested_support": NestedSupport.ALL,
"array_element_selector": "element",
"parameterized_type_defaults": {
exp.DataType.build("DECIMAL", dialect=DIALECT).this: [(10, 0), (0,)],
},
}
def __init__(self, *args: t.Any, **kwargs: t.Any) -> None:
super().__init__(*args, **kwargs)
self._set_spark_engine_adapter_if_needed()
@classmethod
def can_access_spark_session(cls, disable_spark_session: bool) -> bool:
from sqlmesh import RuntimeEnv
if disable_spark_session:
return False
return RuntimeEnv.get().is_databricks
@classmethod
def can_access_databricks_connect(cls, disable_databricks_connect: bool) -> bool:
if disable_databricks_connect:
return False
try:
from databricks.connect import DatabricksSession # noqa
return True
except ImportError:
return False
@property
def _use_spark_session(self) -> bool:
if self.can_access_spark_session(bool(self._extra_config.get("disable_spark_session"))):
return True
return (
self.can_access_databricks_connect(
bool(self._extra_config.get("disable_databricks_connect"))
)
and (
{
"databricks_connect_server_hostname",
"databricks_connect_access_token",
}.issubset(self._extra_config)
)
and (
"databricks_connect_cluster_id" in self._extra_config
or "databricks_connect_use_serverless" in self._extra_config
)
)
@property
def is_spark_session_connection(self) -> bool:
return isinstance(self.connection, SparkSessionConnection)
def _set_spark_engine_adapter_if_needed(self) -> None:
self._spark_engine_adapter = None
if not self._use_spark_session or self.is_spark_session_connection:
return
from databricks.connect import DatabricksSession
connect_kwargs = dict(
host=self._extra_config["databricks_connect_server_hostname"],
token=self._extra_config["databricks_connect_access_token"],
)
if "databricks_connect_use_serverless" in self._extra_config:
connect_kwargs["serverless"] = True
else:
connect_kwargs["cluster_id"] = self._extra_config["databricks_connect_cluster_id"]
catalog = self._extra_config.get("catalog")
spark = (
DatabricksSession.builder.remote(**connect_kwargs).userAgent("sqlmesh").getOrCreate()
)
self._spark_engine_adapter = SparkEngineAdapter(
partial(connection, spark=spark, catalog=catalog),
default_catalog=catalog,
execute_log_level=self._execute_log_level,
multithreaded=self._multithreaded,
sql_gen_kwargs=self._sql_gen_kwargs,
register_comments=self._register_comments,
pre_ping=self._pre_ping,
pretty_sql=self._pretty_sql,
)
@property
def cursor(self) -> t.Any:
if (
self._connection_pool.get_attribute("use_spark_engine_adapter")
and not self.is_spark_session_connection
):
return self._spark_engine_adapter.cursor # type: ignore
return super().cursor
@property
def spark(self) -> PySparkSession:
if not self._use_spark_session:
raise SQLMeshError(
"SparkSession is not available. "
"Either run from a Databricks Notebook or "
"install `databricks-connect` and configure it to connect to your Databricks cluster."
)
if self.is_spark_session_connection:
return self.connection.spark
return self._spark_engine_adapter.spark # type: ignore
@property
def catalog_support(self) -> CatalogSupport:
return CatalogSupport.FULL_SUPPORT
def _begin_session(self, properties: SessionProperties) -> t.Any:
"""Begin a new session."""
# Align the different possible connectors to a single catalog
self.set_current_catalog(self.default_catalog) # type: ignore
def _end_session(self) -> None:
self._connection_pool.set_attribute("use_spark_engine_adapter", False)
def _df_to_source_queries(
self,
df: DF,
target_columns_to_types: t.Dict[str, exp.DataType],
batch_size: int,
target_table: TableName,
source_columns: t.Optional[t.List[str]] = None,
) -> t.List[SourceQuery]:
if not self._use_spark_session:
return super(SparkEngineAdapter, self)._df_to_source_queries(
df, target_columns_to_types, batch_size, target_table, source_columns=source_columns
)
pyspark_df = self._ensure_pyspark_df(
df, target_columns_to_types, source_columns=source_columns
)
def query_factory() -> Query:
temp_table = self._get_temp_table(target_table or "spark", table_only=True)
pyspark_df.createOrReplaceTempView(temp_table.sql(dialect=self.dialect))
self._connection_pool.set_attribute("use_spark_engine_adapter", True)
return exp.select(*self._select_columns(target_columns_to_types)).from_(temp_table)
return [SourceQuery(query_factory=query_factory)]
def _fetch_native_df(
self, query: t.Union[exp.Expression, str], quote_identifiers: bool = False
) -> DF:
"""Fetches a DataFrame that can be either Pandas or PySpark from the cursor"""
if self.is_spark_session_connection:
return super()._fetch_native_df(query, quote_identifiers=quote_identifiers)
if self._spark_engine_adapter:
return self._spark_engine_adapter._fetch_native_df( # type: ignore
query, quote_identifiers=quote_identifiers
)
self.execute(query)
return self.cursor.fetchall_arrow().to_pandas()
def fetchdf(
self, query: t.Union[exp.Expression, str], quote_identifiers: bool = False
) -> pd.DataFrame:
"""
Returns a Pandas DataFrame from a query or expression.
"""
import pandas as pd
df = self._fetch_native_df(query, quote_identifiers=quote_identifiers)
if not isinstance(df, pd.DataFrame):
return df.toPandas()
return df
def get_current_catalog(self) -> t.Optional[str]:
pyspark_catalog = None
sql_connector_catalog = None
if self._spark_engine_adapter:
from py4j.protocol import Py4JError
from pyspark.errors.exceptions.connect import SparkConnectGrpcException
try:
# Note: Spark 3.4+ Only API
pyspark_catalog = self._spark_engine_adapter.get_current_catalog()
except (Py4JError, SparkConnectGrpcException):
pass
elif self.is_spark_session_connection:
pyspark_catalog = self.connection.spark.catalog.currentCatalog()
if not self.is_spark_session_connection:
result = self.fetchone(exp.select(self.CURRENT_CATALOG_EXPRESSION))
sql_connector_catalog = result[0] if result else None
if self._spark_engine_adapter and pyspark_catalog != sql_connector_catalog:
logger.warning(
f"Current catalog mismatch between Databricks SQL Connector and Databricks-Connect: `{sql_connector_catalog}` != `{pyspark_catalog}`. Set `catalog` connection property to make them the same."
)
return pyspark_catalog or sql_connector_catalog
def set_current_catalog(self, catalog_name: str) -> None:
def _set_spark_session_current_catalog(spark: PySparkSession) -> None:
from py4j.protocol import Py4JError
from pyspark.errors.exceptions.connect import SparkConnectGrpcException
try:
# Note: Spark 3.4+ Only API
spark.catalog.setCurrentCatalog(catalog_name)
except (Py4JError, SparkConnectGrpcException):
pass
# Since Databricks splits commands across the Dataframe API and the SQL Connector
# (depending if databricks-connect is installed and a Dataframe is used) we need to ensure both
# are set to the same catalog since they maintain their default catalog separately
self.execute(exp.Use(this=exp.to_identifier(catalog_name), kind="CATALOG"))
if self.is_spark_session_connection:
_set_spark_session_current_catalog(self.connection.spark)
if self._spark_engine_adapter:
_set_spark_session_current_catalog(self._spark_engine_adapter.spark)
def _get_data_objects(
self, schema_name: SchemaName, object_names: t.Optional[t.Set[str]] = None
) -> t.List[DataObject]:
"""
Returns all the data objects that exist in the given schema and catalog.
"""
schema = to_schema(schema_name)
catalog_name = schema.catalog or self.get_current_catalog()
query = (
exp.select(
exp.column("table_name").as_("name"),
exp.column("table_schema").as_("schema"),
exp.column("table_catalog").as_("catalog"),
exp.case(exp.column("table_type"))
.when(exp.Literal.string("VIEW"), exp.Literal.string("view"))
.when(
exp.Literal.string("MATERIALIZED_VIEW"), exp.Literal.string("materialized_view")
)
.else_(exp.Literal.string("table"))
.as_("type"),
)
.from_(
# always query `system` information_schema
exp.table_("tables", "information_schema", "system")
)
.where(exp.column("table_catalog").eq(catalog_name))
.where(exp.column("table_schema").eq(schema.db))
)
if object_names:
query = query.where(exp.column("table_name").isin(*object_names))
df = self.fetchdf(query)
return [
DataObject(
catalog=row.catalog, # type: ignore
schema=row.schema, # type: ignore
name=row.name, # type: ignore
type=DataObjectType.from_str(row.type), # type: ignore
)
for row in df.itertuples()
]
def clone_table(
self,
target_table_name: TableName,
source_table_name: TableName,
replace: bool = False,
exists: bool = True,
clone_kwargs: t.Optional[t.Dict[str, t.Any]] = None,
**kwargs: t.Any,
) -> None:
clone_kwargs = clone_kwargs or {}
clone_kwargs["shallow"] = True
super().clone_table(
target_table_name,
source_table_name,
replace=replace,
clone_kwargs=clone_kwargs,
**kwargs,
)
def wap_supported(self, table_name: TableName) -> bool:
return False
def close(self) -> t.Any:
"""Closes all open connections and releases all allocated resources."""
super().close()
if self._spark_engine_adapter:
self._spark_engine_adapter.close()
@property
def default_catalog(self) -> t.Optional[str]:
try:
return super().default_catalog
except MissingDefaultCatalogError as e:
raise MissingDefaultCatalogError(
"Could not determine default catalog. Define the connection property `catalog` since it can't be inferred from your connection. See SQLMesh Databricks documentation for details"
) from e
def _build_table_properties_exp(
self,
catalog_name: t.Optional[str] = None,
table_format: t.Optional[str] = None,
storage_format: t.Optional[str] = None,
partitioned_by: t.Optional[t.List[exp.Expression]] = None,
partition_interval_unit: t.Optional[IntervalUnit] = None,
clustered_by: t.Optional[t.List[exp.Expression]] = None,
table_properties: t.Optional[t.Dict[str, exp.Expression]] = None,
target_columns_to_types: t.Optional[t.Dict[str, exp.DataType]] = None,
table_description: t.Optional[str] = None,
table_kind: t.Optional[str] = None,
**kwargs: t.Any,
) -> t.Optional[exp.Properties]:
properties = super()._build_table_properties_exp(
catalog_name=catalog_name,
table_format=table_format,
storage_format=storage_format,
partitioned_by=partitioned_by,
partition_interval_unit=partition_interval_unit,
clustered_by=clustered_by,
table_properties=table_properties,
target_columns_to_types=target_columns_to_types,
table_description=table_description,
table_kind=table_kind,
)
if clustered_by:
# Databricks expects wrapped CLUSTER BY expressions
clustered_by_exp = exp.Cluster(
expressions=[exp.Tuple(expressions=[c.copy() for c in clustered_by])]
)
expressions = properties.expressions if properties else []
expressions.append(clustered_by_exp)
properties = exp.Properties(expressions=expressions)
return properties