dagster-sling 0.28.8__py3-none-any.whl

This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
@@ -0,0 +1,646 @@
1
+ import contextlib
2
+ import json
3
+ import os
4
+ import re
5
+ import subprocess
6
+ import tempfile
7
+ import time
8
+ import uuid
9
+ from collections.abc import Generator, Iterator, Sequence
10
+ from enum import Enum
11
+ from subprocess import PIPE, STDOUT, Popen
12
+ from typing import IO, Any, AnyStr, Optional, Union
13
+
14
+ import sling
15
+ from dagster import (
16
+ AssetExecutionContext,
17
+ AssetMaterialization,
18
+ ConfigurableResource,
19
+ EnvVar,
20
+ MaterializeResult,
21
+ OpExecutionContext,
22
+ PermissiveConfig,
23
+ get_dagster_logger,
24
+ )
25
+ from dagster._annotations import public
26
+ from dagster._core.definitions.metadata import TableMetadataSet
27
+ from dagster._utils.env import environ
28
+ from pydantic import Field
29
+
30
+ from dagster_sling.asset_decorator import (
31
+ METADATA_KEY_REPLICATION_CONFIG,
32
+ METADATA_KEY_TRANSLATOR,
33
+ get_streams_from_replication,
34
+ streams_with_default_dagster_meta,
35
+ )
36
+ from dagster_sling.dagster_sling_translator import DagsterSlingTranslator
37
+ from dagster_sling.sling_event_iterator import SlingEventIterator, SlingEventType
38
+ from dagster_sling.sling_replication import SlingReplicationParam, validate_replication
39
+
40
+ logger = get_dagster_logger()
41
+
42
+ ANSI_ESCAPE = re.compile(r"\x1B(?:[@-Z\\-_]|\[[0-?]*[ -/]*[@-~])")
43
+
44
+
45
+ @public
46
+ class SlingMode(str, Enum):
47
+ """The mode to use when syncing.
48
+
49
+ See the Sling docs for more information: https://docs.slingdata.io/sling-cli/run/configuration#modes
50
+ """
51
+
52
+ INCREMENTAL = "incremental"
53
+ TRUNCATE = "truncate"
54
+ FULL_REFRESH = "full-refresh"
55
+ SNAPSHOT = "snapshot"
56
+ BACKFILL = "backfill"
57
+
58
+
59
+ @public
60
+ class SlingConnectionResource(PermissiveConfig):
61
+ """A representation of a connection to a database or file to be used by Sling. This resource can be used as a source or a target for a Sling syncs.
62
+
63
+ Reference the Sling docs for more information on possible connection types and parameters: https://docs.slingdata.io/connections
64
+
65
+ The name of the connection is passed to Sling and must match the name of the connection provided in the replication configuration: https://docs.slingdata.io/sling-cli/run/configuration/replication
66
+ You may provide either a connection string or keyword arguments for the connection.
67
+
68
+ Examples:
69
+ Creating a Sling Connection for a file, such as CSV or JSON:
70
+
71
+ .. code-block:: python
72
+
73
+ source = SlingConnectionResource(name="MY_FILE", type="file")
74
+
75
+ Create a Sling Connection for a Postgres database, using a connection string:
76
+
77
+ .. code-block:: python
78
+
79
+ postgres_conn = SlingConnectionResource(name="MY_POSTGRES", type="postgres", connection_string=EnvVar("POSTGRES_CONNECTION_STRING"))
80
+ mysql_conn = SlingConnectionResource(name="MY_MYSQL", type="mysql", connection_string="mysql://user:password@host:port/schema")
81
+
82
+ Create a Sling Connection for a Postgres or Snowflake database, using keyword arguments:
83
+
84
+ .. code-block::python
85
+
86
+ postgres_conn = SlingConnectionResource(
87
+ name="MY_OTHER_POSRGRES",
88
+ type="postgres",
89
+ host="host",
90
+ user="hunter42",
91
+ password=EnvVar("POSTGRES_PASSWORD")
92
+ )
93
+
94
+ snowflake_conn = SlingConnectionResource(
95
+ name="MY_SNOWFLAKE",
96
+ type="snowflake",
97
+ host=EnvVar("SNOWFLAKE_HOST"),
98
+ user=EnvVar("SNOWFLAKE_USER"),
99
+ database=EnvVar("SNOWFLAKE_DATABASE"),
100
+ password=EnvVar("SNOWFLAKE_PASSWORD"),
101
+ role=EnvVar("SNOWFLAKE_ROLE")
102
+ )
103
+ """
104
+
105
+ name: str = Field(
106
+ description="The name of the connection, must match the name in your Sling replication configuration."
107
+ )
108
+ type: str = Field(
109
+ description="Type of the source connection, must match the Sling connection types. Use 'file' for local storage."
110
+ )
111
+ connection_string: Optional[str] = Field(
112
+ description="The optional connection string for the source database, if not using keyword arguments.",
113
+ default=None,
114
+ )
115
+
116
+
117
+ class SlingResource(ConfigurableResource):
118
+ """Resource for interacting with the Sling package. This resource can be used to run Sling replications.
119
+
120
+ Args:
121
+ connections (List[SlingConnectionResource]): A list of connections to use for the replication.
122
+
123
+ Examples:
124
+ .. code-block:: python
125
+
126
+ from dagster_etl.sling import SlingResource, SlingConnectionResource
127
+
128
+ sling_resource = SlingResource(
129
+ connections=[
130
+ SlingConnectionResource(
131
+ name="MY_POSTGRES",
132
+ type="postgres",
133
+ connection_string=EnvVar("POSTGRES_CONNECTION_STRING"),
134
+ ),
135
+ SlingConnectionResource(
136
+ name="MY_SNOWFLAKE",
137
+ type="snowflake",
138
+ host=EnvVar("SNOWFLAKE_HOST"),
139
+ user=EnvVar("SNOWFLAKE_USER"),
140
+ database=EnvVar("SNOWFLAKE_DATABASE"),
141
+ password=EnvVar("SNOWFLAKE_PASSWORD"),
142
+ role=EnvVar("SNOWFLAKE_ROLE"),
143
+ ),
144
+ ]
145
+ )
146
+ """
147
+
148
+ connections: list[SlingConnectionResource] = []
149
+ _stdout: list[str] = []
150
+
151
+ @staticmethod
152
+ def _get_replication_streams_for_context(
153
+ context: Union[OpExecutionContext, AssetExecutionContext],
154
+ ) -> dict[str, Any]:
155
+ """Computes the sling replication streams config for a given execution context with an
156
+ assets def, possibly involving a subset selection of sling assets.
157
+ """
158
+ if not context.has_assets_def:
159
+ no_assets_def_message = """
160
+ The current execution context has no backing AssetsDefinition object. Therefore, no
161
+ sling assets subsetting will be performed...
162
+ """
163
+ logger.warn(no_assets_def_message)
164
+ return {}
165
+ context_streams = {}
166
+ assets_def = context.assets_def
167
+ run_config = context.run_config
168
+ if run_config: # triggered via sensor
169
+ run_config_ops = run_config.get("ops", {})
170
+ if isinstance(run_config_ops, dict):
171
+ assets_op_config = run_config_ops.get(assets_def.op.name, {}).get("config", {})
172
+ else:
173
+ assets_op_config = {}
174
+ context_streams = assets_op_config.get("context_streams", {})
175
+ if not context_streams:
176
+ no_context_streams_config_message = f"""
177
+ It was expected that your `run_config` would provide a `context_streams` config for
178
+ the op {assets_def.op.name}. Instead, the received value for this op config was
179
+ {assets_op_config}.
180
+
181
+ NO ASSET SUBSETTING WILL BE PERFORMED!
182
+
183
+ If that was your intention, you can safely ignore this message. Otherwise, provide
184
+ the mentioned `context_streams` config for executing only your desired asset subset.
185
+ """
186
+ logger.warn(no_context_streams_config_message)
187
+ else:
188
+ metadata_by_key = assets_def.metadata_by_key
189
+ first_asset_metadata = next(iter(metadata_by_key.values()))
190
+ replication_config: dict[str, Any] = first_asset_metadata.get(
191
+ METADATA_KEY_REPLICATION_CONFIG, {}
192
+ )
193
+ dagster_sling_translator: DagsterSlingTranslator = first_asset_metadata.get(
194
+ METADATA_KEY_TRANSLATOR, DagsterSlingTranslator()
195
+ )
196
+ raw_streams = get_streams_from_replication(replication_config)
197
+ streams = streams_with_default_dagster_meta(raw_streams, replication_config)
198
+ selected_asset_keys = context.selected_asset_keys
199
+ for stream in streams:
200
+ asset_key = dagster_sling_translator.get_asset_spec(stream).key
201
+ if asset_key in selected_asset_keys:
202
+ context_streams.update({stream["name"]: stream["config"]})
203
+
204
+ return context_streams
205
+
206
+ @classmethod
207
+ def _is_dagster_maintained(cls) -> bool:
208
+ return True
209
+
210
+ def _clean_connection_dict(self, d: dict[str, Any]) -> dict[str, Any]:
211
+ d = _process_env_vars(d)
212
+ if d["connection_string"]:
213
+ d["url"] = d["connection_string"]
214
+ if "connection_string" in d:
215
+ del d["connection_string"]
216
+ return d
217
+
218
+ def _query_metadata(
219
+ self, metadata_string: str, start_time: float, base_metadata: Union[list, None] = None
220
+ ):
221
+ """Metadata quering using regular expression from standard sling log.
222
+
223
+ Args:
224
+ metadata_string (str): raw log string containing log/metadata from sling cli run
225
+ start_time (float): start time that will be assign to calculate elapse
226
+ base_metadata (list, Null): list of metadata to be query from string
227
+
228
+ Return:
229
+ final_dict (dict): Final metadata idct contain metadata query from string
230
+ """
231
+ if base_metadata is None:
232
+ base_metadata = [
233
+ "stream_name",
234
+ "row_count",
235
+ "destination_table",
236
+ "destination_file",
237
+ "elapsed_time",
238
+ ]
239
+
240
+ tmp = None
241
+ tmp_metadata = {}
242
+ end_time = time.time()
243
+ target_type = re.findall(r"writing to target ([\w\s]*) ", metadata_string)
244
+ if target_type and target_type[0] == "database":
245
+ tmp = re.findall(r"inserted ([0-9]*) rows .*into ([\w.:/;-_\"\'{}]*)", metadata_string)
246
+ elif target_type and target_type[0] == "file system":
247
+ tmp = re.findall(r"wrote ([0-9]*) rows .*to ([\w.:/;-_\"\'{}]*)", metadata_string)
248
+ else:
249
+ tmp = re.findall(r"inserted ([0-9]*) rows .*into ([\w.:/;-_\"\'{}]*)", metadata_string)
250
+
251
+ if tmp:
252
+ if target_type and target_type[0] == "database":
253
+ tmp_metadata["destination_table"] = re.sub(r"[^\w\s.]", "", tmp[0][1])
254
+ if target_type and target_type[0] == "file system":
255
+ tmp_metadata["destination_file"] = re.sub(r"[^\w\s.]", "", tmp[0][1])
256
+ tmp_metadata["elapsed_time"] = end_time - start_time
257
+ tmp_metadata["row_count"] = tmp[0][0]
258
+
259
+ final_dict = {}
260
+ for k in base_metadata:
261
+ if tmp_metadata.get(k):
262
+ final_dict[k] = tmp_metadata.get(k)
263
+ return final_dict
264
+
265
+ def prepare_environment(self) -> dict[str, Any]:
266
+ env = {}
267
+
268
+ for conn in self.connections:
269
+ d = self._clean_connection_dict(dict(conn))
270
+ env[conn.name] = json.dumps(d)
271
+
272
+ return env
273
+
274
+ @contextlib.contextmanager
275
+ def _setup_config(self) -> Generator[None, None, None]:
276
+ """Uses environment variables to set the Sling source and target connections."""
277
+ prepared_environment = self.prepare_environment()
278
+ with environ(prepared_environment):
279
+ yield
280
+
281
+ def _clean_line(self, line: str) -> str:
282
+ """Removes ANSI escape sequences and Sling log prefixes from a line of output."""
283
+ line = ANSI_ESCAPE.sub("", line)
284
+ # Remove Sling log format prefix: "{timestamp} {LEVEL} " (e.g., "1:04PM INF ")
285
+ # Match pattern: optional timestamp followed by log level (INF, WRN, ERR, DBG) and space
286
+ line = re.sub(r"^\d{1,2}:\d{2}[AP]M\s+(INF|WRN|ERR|DBG)\s+", "", line)
287
+ return line
288
+
289
+ def _clean_timestamp_log(self, line: str):
290
+ """Remove timestamp from log gather from sling cli to reduce redundency in dagster log.
291
+
292
+ Args:
293
+ line (str): line of log gather from cli to be cleaned
294
+
295
+ Returns:
296
+ text: cleaned log consist only of log data
297
+
298
+ """
299
+ tmp = self._clean_line(line)
300
+ try:
301
+ text = tmp.split(" ")[1]
302
+ except:
303
+ text = tmp
304
+ return text
305
+
306
+ def _process_stdout(self, stdout: IO[AnyStr], encoding="utf8") -> Iterator[str]:
307
+ """Process stdout from the Sling CLI."""
308
+ for line in stdout:
309
+ assert isinstance(line, bytes)
310
+ fmt_line = bytes.decode(line, encoding=encoding, errors="replace")
311
+ yield self._clean_line(fmt_line)
312
+
313
+ def _exec_sling_cmd(
314
+ self, cmd, stdin=None, stdout=PIPE, stderr=STDOUT, encoding="utf8"
315
+ ) -> Generator[str, None, None]:
316
+ with Popen(cmd, shell=True, stdin=stdin, stdout=stdout, stderr=stderr) as proc:
317
+ if proc.stdout:
318
+ yield from self._process_stdout(proc.stdout, encoding=encoding)
319
+
320
+ proc.wait()
321
+ if proc.returncode != 0:
322
+ raise Exception("Sling command failed with error code %s", proc.returncode)
323
+
324
+ def _parse_json_table_output(self, table_output: dict[str, Any]) -> list[dict[str, str]]:
325
+ column_keys: list[str] = table_output["fields"]
326
+ column_values: list[list[str]] = table_output["rows"]
327
+
328
+ return [dict(zip(column_keys, column_values)) for column_values in column_values]
329
+
330
+ def get_column_info_for_table(self, target_name: str, table_name: str) -> list[dict[str, str]]:
331
+ """Fetches column metadata for a given table in a Sling target and parses it into a list of
332
+ dictionaries, keyed by column name.
333
+
334
+ Args:
335
+ target_name (str): The name of the target connection to use.
336
+ table_name (str): The name of the table to fetch column metadata for.
337
+
338
+ Returns:
339
+ List[Dict[str, str]]: A list of dictionaries, keyed by column name, containing column metadata.
340
+ """
341
+ output = self.run_sling_cli(
342
+ ["conns", "discover", target_name, "--pattern", table_name, "--columns"],
343
+ force_json=True,
344
+ )
345
+ return self._parse_json_table_output(json.loads(output.strip()))
346
+
347
+ def get_row_count_for_table(self, target_name: str, table_name: str) -> int:
348
+ """Queries the target connection to get the row count for a given table.
349
+
350
+ Args:
351
+ target_name (str): The name of the target connection to use.
352
+ table_name (str): The name of the table to fetch the row count for.
353
+
354
+ Returns:
355
+ int: The number of rows in the table.
356
+ """
357
+ select_stmt: str = f"select count(*) as ct from {table_name}"
358
+ output = self.run_sling_cli(
359
+ ["conns", "exec", target_name, select_stmt],
360
+ force_json=True,
361
+ )
362
+ return int(
363
+ next(iter(self._parse_json_table_output(json.loads(output.strip()))[0].values()))
364
+ )
365
+
366
+ def run_sling_cli(self, args: Sequence[str], force_json: bool = False) -> str:
367
+ """Runs the Sling CLI with the given arguments and returns the output.
368
+
369
+ Args:
370
+ args (Sequence[str]): The arguments to pass to the Sling CLI.
371
+
372
+ Returns:
373
+ str: The output from the Sling CLI.
374
+ """
375
+ with environ({"SLING_OUTPUT": "json"}) if force_json else contextlib.nullcontext():
376
+ return subprocess.check_output(args=[sling.SLING_BIN, *args], text=True)
377
+
378
+ def replicate(
379
+ self,
380
+ *,
381
+ context: Union[OpExecutionContext, AssetExecutionContext],
382
+ replication_config: Optional[SlingReplicationParam] = None,
383
+ dagster_sling_translator: Optional[DagsterSlingTranslator] = None,
384
+ debug: bool = False,
385
+ stream: bool = False,
386
+ ) -> SlingEventIterator[SlingEventType]:
387
+ """Runs a Sling replication from the given replication config.
388
+
389
+ Args:
390
+ context: Asset or Op execution context.
391
+ replication_config: The Sling replication config to use for the replication.
392
+ dagster_sling_translator: The translator to use for the replication.
393
+ debug: Whether to run the replication in debug mode.
394
+
395
+ Returns:
396
+ SlingEventIterator[MaterializeResult]: A generator of MaterializeResult
397
+ """
398
+ if not (replication_config or dagster_sling_translator):
399
+ metadata_by_key = context.assets_def.metadata_by_key
400
+ first_asset_metadata = next(iter(metadata_by_key.values()))
401
+ dagster_sling_translator = first_asset_metadata.get(METADATA_KEY_TRANSLATOR)
402
+ replication_config = first_asset_metadata.get(METADATA_KEY_REPLICATION_CONFIG)
403
+
404
+ dagster_sling_translator = dagster_sling_translator or DagsterSlingTranslator()
405
+ replication_config_dict = dict(validate_replication(replication_config))
406
+ return SlingEventIterator(
407
+ self._replicate(
408
+ context=context,
409
+ replication_config=replication_config_dict,
410
+ dagster_sling_translator=dagster_sling_translator,
411
+ debug=debug,
412
+ stream=stream,
413
+ ),
414
+ sling_cli=self,
415
+ replication_config=replication_config_dict,
416
+ context=context,
417
+ )
418
+
419
+ def _replicate(
420
+ self,
421
+ *,
422
+ context: Union[OpExecutionContext, AssetExecutionContext],
423
+ replication_config: dict[str, Any],
424
+ dagster_sling_translator: DagsterSlingTranslator,
425
+ debug: bool,
426
+ stream: bool = False,
427
+ ) -> Iterator[SlingEventType]:
428
+ # if translator has not been defined on metadata _or_ through param, then use the default constructor
429
+
430
+ with self._setup_config():
431
+ env = os.environ.copy()
432
+
433
+ if not stream:
434
+ ##### Old method use _run which is not streamable #####
435
+ generator = self._batch_sling_replicate(
436
+ context=context,
437
+ replication_config=replication_config,
438
+ dagster_sling_translator=dagster_sling_translator,
439
+ env=env,
440
+ debug=debug,
441
+ )
442
+
443
+ else:
444
+ #### New method use sling _exec_cmd to stream log from sling to dagster log
445
+ generator = self._stream_sling_replicate(
446
+ context=context,
447
+ replication_config=replication_config,
448
+ dagster_sling_translator=dagster_sling_translator,
449
+ env=env,
450
+ debug=debug,
451
+ )
452
+
453
+ yield from generator
454
+
455
+ def _batch_sling_replicate(
456
+ self,
457
+ context: Union[OpExecutionContext, AssetExecutionContext],
458
+ replication_config: dict[str, Any],
459
+ dagster_sling_translator: DagsterSlingTranslator,
460
+ env: dict,
461
+ debug: bool,
462
+ ) -> Generator[Union[MaterializeResult, AssetMaterialization], None, None]:
463
+ """Underlying function to run replication and fetch metadata in batch mode."""
464
+ # convert to dict to enable updating the index
465
+ context_streams = self._get_replication_streams_for_context(context)
466
+
467
+ if context_streams:
468
+ replication_config.update({"streams": context_streams})
469
+ stream_definitions = get_streams_from_replication(replication_config)
470
+
471
+ # extract the destination name from the replication config
472
+ destination_name = replication_config.get("target")
473
+
474
+ uid = uuid.uuid4()
475
+ temp_dir = tempfile.gettempdir()
476
+ temp_file = os.path.join(temp_dir, f"sling-replication-{uid}.json")
477
+
478
+ with open(temp_file, "w") as file:
479
+ json.dump(replication_config, file, cls=sling.JsonEncoder)
480
+
481
+ logger.debug(f"Replication config: {replication_config}")
482
+
483
+ debug_str = "-d" if debug else ""
484
+
485
+ cmd = f"{sling.SLING_BIN} run {debug_str} -r {temp_file}"
486
+
487
+ logger.debug(f"Running Sling replication with command: {cmd}")
488
+
489
+ # Get start time from wall clock
490
+ start_time = time.time()
491
+
492
+ results = sling._run( # noqa
493
+ cmd=cmd,
494
+ temp_file=temp_file,
495
+ return_output=True,
496
+ env=env,
497
+ )
498
+
499
+ end_time = time.time()
500
+
501
+ for row in results.split("\n"):
502
+ clean_line = self._clean_line(row)
503
+ logger.debug(clean_line + "\n")
504
+ self._stdout.append(clean_line)
505
+
506
+ for stream_definition in stream_definitions:
507
+ asset_key = dagster_sling_translator.get_asset_spec(stream_definition).key
508
+
509
+ object_key = (stream_definition.get("config") or {}).get("object")
510
+ destination_stream_name = object_key or stream_definition["name"]
511
+ table_name = None
512
+ if destination_name and destination_stream_name:
513
+ table_name = ".".join([destination_name, destination_stream_name])
514
+
515
+ metadata = {
516
+ "elapsed_time": end_time - start_time,
517
+ "stream_name": stream_definition["name"],
518
+ **TableMetadataSet(
519
+ table_name=table_name,
520
+ ),
521
+ }
522
+
523
+ if context.has_assets_def:
524
+ yield MaterializeResult(asset_key=asset_key, metadata=metadata)
525
+ else:
526
+ yield AssetMaterialization(asset_key=asset_key, metadata=metadata)
527
+
528
+ def _stream_sling_replicate(
529
+ self,
530
+ context: Union[OpExecutionContext, AssetExecutionContext],
531
+ replication_config: dict[str, Any],
532
+ dagster_sling_translator: DagsterSlingTranslator,
533
+ env: dict,
534
+ debug: bool,
535
+ ) -> Generator[Union[MaterializeResult, AssetMaterialization], None, None]:
536
+ """Underlying function to run replication and fetch metadata in stream mode."""
537
+ # define variable to use to compute metadata during run
538
+ current_stream = None
539
+ metadata_text = []
540
+ metadata = {}
541
+
542
+ # convert to dict to enable updating the index
543
+ context_streams = self._get_replication_streams_for_context(context)
544
+
545
+ if context_streams:
546
+ replication_config.update({"streams": context_streams})
547
+
548
+ uid = uuid.uuid4()
549
+ temp_dir = tempfile.gettempdir()
550
+ temp_file = os.path.join(temp_dir, f"sling-replication-{uid}.json")
551
+
552
+ with open(temp_file, "w") as file:
553
+ json.dump(replication_config, file, cls=sling.JsonEncoder)
554
+
555
+ logger.debug(f"Replication config: {replication_config}")
556
+
557
+ debug_str = "-d" if debug else ""
558
+
559
+ cmd = f"{sling.SLING_BIN} run {debug_str} -r {temp_file}"
560
+
561
+ logger.debug(f"Running Sling replication with command: {cmd}")
562
+
563
+ # Get start time from wall clock
564
+ start_time = time.time()
565
+
566
+ for line in sling._exec_cmd(cmd, env=env): # noqa
567
+ if line == "": # if empty line -- skipped
568
+ continue
569
+ text = self._clean_timestamp_log(line) # else clean timestamp
570
+ logger.info(text) # log info to dagster log
571
+
572
+ # if no current stream is chosen
573
+ if current_stream is None:
574
+ # Try to match stream name with stream keyword
575
+ matched = re.findall("stream (.*)$", text)
576
+
577
+ # If found, extract stream name, stream config, asset key
578
+ if matched:
579
+ current_stream = matched[0]
580
+ current_config = replication_config.get("streams", {}).get(current_stream, {})
581
+ asset_key = dagster_sling_translator.get_asset_spec(
582
+ {"name": current_stream, "config": current_config}
583
+ ).key
584
+ if debug:
585
+ logger.debug(current_stream)
586
+ logger.debug(current_config)
587
+ logger.debug(asset_key)
588
+ # Else search for single replication format
589
+ else:
590
+ # If found, extract stream name, stream config, asset key
591
+ matched = re.findall(r"Sling Replication [|] .* [|] (\S*)$", text)
592
+ if matched:
593
+ current_stream = matched[0]
594
+ current_config = replication_config.get("streams", {}).get(
595
+ current_stream, {}
596
+ )
597
+ asset_key = dagster_sling_translator.get_asset_spec(
598
+ {"name": current_stream, "config": current_config}
599
+ ).key
600
+ if debug:
601
+ logger.debug(current_stream)
602
+ logger.debug(current_config)
603
+ logger.debug(asset_key)
604
+ # Else log that no stream found. This is normal for a few line. But if multiple line come up, further evaluate might be needed for other pattern
605
+ else:
606
+ if debug:
607
+ logger.debug("no match stream name")
608
+ # If current stream is already choose
609
+ else:
610
+ # Search whether the current stream ended
611
+ matched = re.findall("execution succeeded", text)
612
+
613
+ if matched:
614
+ # If yes, query metadata and materialize asset
615
+ metadata = self._query_metadata("\n".join(metadata_text), start_time=start_time)
616
+ start_time = time.time()
617
+ metadata["stream_name"] = current_stream
618
+ logger.debug(metadata)
619
+ if context.has_assets_def:
620
+ yield MaterializeResult(asset_key=asset_key, metadata=metadata) # pyright: ignore[reportPossiblyUnboundVariable]
621
+ else:
622
+ yield AssetMaterialization(asset_key=asset_key, metadata=metadata) # pyright: ignore[reportPossiblyUnboundVariable]
623
+
624
+ current_stream = None
625
+ metadata_text = []
626
+
627
+ metadata_text.append(text)
628
+
629
+ # clean up unused file
630
+ os.remove(temp_file)
631
+
632
+ def stream_raw_logs(self) -> Generator[str, None, None]:
633
+ """Returns a generator of raw logs from the Sling CLI."""
634
+ yield from self._stdout
635
+
636
+
637
+ def _process_env_vars(config: dict[str, Any]) -> dict[str, Any]:
638
+ out = {}
639
+ for key, value in config.items():
640
+ if isinstance(value, dict) and len(value) == 1 and next(iter(value.keys())) == "env":
641
+ out[key] = EnvVar(next(iter(value.values()))).get_value()
642
+ elif isinstance(value, EnvVar):
643
+ out[key] = value.get_value()
644
+ else:
645
+ out[key] = value
646
+ return out