def migrate_asset_key_data(event_log_storage, print_fn=lambda _: None): """ Utility method to build an asset key index from the data in existing event log records. Takes in event_log_storage, and a print_fn to keep track of progress. """ from dagster.core.storage.event_log.sql_event_log import AssetAwareSqlEventLogStorage from .schema import AssetKeyTable, SqlEventLogStorageTable if not isinstance(event_log_storage, AssetAwareSqlEventLogStorage): return query = (db.select([ SqlEventLogStorageTable.c.asset_key ]).where(SqlEventLogStorageTable.c.asset_key != None).group_by( SqlEventLogStorageTable.c.asset_key)) with event_log_storage.connect() as conn: print_fn("Querying event logs.") to_insert = conn.execute(query).fetchall() print_fn("Found {} records to index".format(len(to_insert))) for (asset_key, ) in tqdm(to_insert): try: conn.execute(AssetKeyTable.insert().values( # pylint: disable=no-value-for-parameter asset_key=AssetKey.from_db_string(asset_key).to_string())) except db.exc.IntegrityError: # asset key already present pass
def migrate_asset_keys_index_columns(event_log_storage, print_fn=None): from dagster.core.storage.event_log.sql_event_log import SqlEventLogStorage from dagster.serdes import serialize_dagster_namedtuple from .schema import AssetKeyTable, SqlEventLogStorageTable if not isinstance(event_log_storage, SqlEventLogStorage): return with event_log_storage.index_connection() as conn: if print_fn: print_fn("Querying asset keys.") results = conn.execute( db.select( [ AssetKeyTable.c.asset_key, AssetKeyTable.c.asset_details, AssetKeyTable.c.last_materialization, ] ) ).fetchall() if print_fn: print_fn(f"Found {len(results)} assets to reindex.") results = tqdm(results) for row in results: asset_key_str, asset_details_str, last_materialization_str = row wipe_timestamp = None event = None asset_key = AssetKey.from_db_string(asset_key_str) if asset_details_str: asset_details = deserialize_json_to_dagster_namedtuple(asset_details_str) wipe_timestamp = asset_details.last_wipe_timestamp if asset_details else None if last_materialization_str: event_or_materialization = deserialize_json_to_dagster_namedtuple( last_materialization_str ) if isinstance(event_or_materialization, EventLogEntry): event = event_or_materialization if not event: materialization_query = ( db.select([SqlEventLogStorageTable.c.event]) .where( db.or_( SqlEventLogStorageTable.c.asset_key == asset_key.to_string(), SqlEventLogStorageTable.c.asset_key == asset_key.to_string(legacy=True), ) ) .order_by(SqlEventLogStorageTable.c.timestamp.desc()) .limit(1) ) row = conn.execute(materialization_query).fetchone() if row: event = deserialize_json_to_dagster_namedtuple(row[0]) if not event: # this must be a wiped asset conn.execute( AssetKeyTable.update() .values( # pylint: disable=no-value-for-parameter last_materialization=None, last_materialization_timestamp=None, wipe_timestamp=utc_datetime_from_timestamp(wipe_timestamp) if wipe_timestamp else None, tags=None, ) .where( AssetKeyTable.c.asset_key == asset_key.to_string(), ) ) else: tags = event.dagster_event.step_materialization_data.materialization.tags conn.execute( AssetKeyTable.update() .values( # pylint: disable=no-value-for-parameter last_materialization=serialize_dagster_namedtuple(event), last_materialization_timestamp=utc_datetime_from_timestamp(event.timestamp), wipe_timestamp=utc_datetime_from_timestamp(wipe_timestamp) if wipe_timestamp else None, tags=seven.json.dumps(tags) if tags else None, ) .where( AssetKeyTable.c.asset_key == asset_key.to_string(), ) )