def __init__( self, solid_config: Any, resources_dict: Optional[Dict[str, Any]], instance: Optional[DagsterInstance], ): # pylint: disable=super-init-not-called from dagster.core.execution.context_creation_pipeline import initialize_console_manager from dagster.core.execution.api import ephemeral_instance_if_missing self._solid_config = solid_config self._instance_provided = ( check.opt_inst_param(instance, "instance", DagsterInstance) is not None ) # Construct ephemeral instance if missing self._instance_cm = ephemeral_instance_if_missing(instance) # Pylint can't infer that the ephemeral_instance context manager has an __enter__ method, # so ignore lint error self._instance = self._instance_cm.__enter__() # pylint: disable=no-member # Open resource context manager self._resources_cm = build_resources( check.opt_dict_param(resources_dict, "resources_dict", key_type=str), instance ) self._resources = self._resources_cm.__enter__() # pylint: disable=no-member self._resources_contain_cm = isinstance(self._resources, IContainsGenerator) self._log = initialize_console_manager(None) self._pdb: Optional[ForkedPdb] = None self._cm_scope_entered = False
def __init__( self, resource_config: Any, resources: Optional[Union[Resources, Dict[str, Any]]], instance: Optional[DagsterInstance], ): from dagster.core.execution.build_resources import build_resources from dagster.core.execution.api import ephemeral_instance_if_missing from dagster.core.execution.context_creation_pipeline import initialize_console_manager self._instance_provided = (check.opt_inst_param( instance, "instance", DagsterInstance) is not None) # Construct ephemeral instance if missing self._instance_cm = ephemeral_instance_if_missing(instance) # Pylint can't infer that the ephemeral_instance context manager has an __enter__ method, # so ignore lint error instance = self._instance_cm.__enter__() # pylint: disable=no-member # If we are provided with a Resources instance, then we do not need to initialize if isinstance(resources, Resources): self._resources_cm = None else: self._resources_cm = build_resources(check.opt_dict_param( resources, "resources", key_type=str), instance=instance) resources = self._resources_cm.__enter__() # pylint: disable=no-member self._resources_contain_cm = isinstance(resources, IContainsGenerator) self._cm_scope_entered = False super(UnboundInitResourceContext, self).__init__( resource_config=resource_config, resources=resources, resource_def=None, instance=instance, pipeline_run=None, log_manager=initialize_console_manager(None), pipeline_def_for_backwards_compat=None, )
def build_input_context( name: Optional[str] = None, config: Optional[Any] = None, metadata: Optional[Dict[str, Any]] = None, upstream_output: Optional["OutputContext"] = None, dagster_type: Optional["DagsterType"] = None, resource_config: Optional[Dict[str, Any]] = None, resources: Optional[Dict[str, Any]] = None, ) -> "InputContext": """Builds input context from provided parameters. ``build_input_context`` can be used as either a function, or a context manager. If resources that are also context managers are provided, then ``build_input_context`` must be used as a context manager. Args: name (Optional[str]): The name of the input that we're loading. config (Optional[Any]): The config attached to the input that we're loading. metadata (Optional[Dict[str, Any]]): A dict of metadata that is assigned to the InputDefinition that we're loading for. upstream_output (Optional[OutputContext]): Info about the output that produced the object we're loading. dagster_type (Optional[DagsterType]): The type of this input. resource_config (Optional[Dict[str, Any]]): The resource config to make available from the input context. This usually corresponds to the config provided to the resource that loads the input manager. resources (Optional[Dict[str, Any]]): The resources to make available from the context. For a given key, you can provide either an actual instance of an object, or a resource definition. Examples: .. code-block:: python build_input_context(step_key, name) with build_input_context( step_key, name, resources={"foo": context_manager_resource} ) as context: do_something """ from dagster.core.execution.context.output import OutputContext from dagster.core.types.dagster_type import DagsterType from dagster.core.execution.context_creation_pipeline import initialize_console_manager experimental_fn_warning("build_input_context") name = check.opt_str_param(name, "name") metadata = check.opt_dict_param(metadata, "metadata", key_type=str) upstream_output = check.opt_inst_param(upstream_output, "upstream_output", OutputContext) dagster_type = check.opt_inst_param(dagster_type, "dagster_type", DagsterType) resource_config = check.opt_dict_param(resource_config, "resource_config", key_type=str) resources = check.opt_dict_param(resources, "resources", key_type=str) return InputContext( name=name, pipeline_name=None, solid_def=None, config=config, metadata=metadata, upstream_output=upstream_output, dagster_type=dagster_type, log_manager=initialize_console_manager(None), resource_config=resource_config, resources=resources, step_context=None, )
def build_output_context( step_key: Optional[str] = None, name: Optional[str] = None, metadata: Optional[Dict[str, Any]] = None, run_id: Optional[str] = None, mapping_key: Optional[str] = None, config: Optional[Any] = None, dagster_type: Optional["DagsterType"] = None, version: Optional[str] = None, resource_config: Optional[Dict[str, Any]] = None, resources: Optional[Dict[str, Any]] = None, solid_def: Optional[SolidDefinition] = None, op_def: Optional[OpDefinition] = None, ) -> "OutputContext": """Builds output context from provided parameters. ``build_output_context`` can be used as either a function, or a context manager. If resources that are also context managers are provided, then ``build_output_context`` must be used as a context manager. Args: step_key (Optional[str]): The step_key for the compute step that produced the output. name (Optional[str]): The name of the output that produced the output. metadata (Optional[Dict[str, Any]]): A dict of the metadata that is assigned to the OutputDefinition that produced the output. mapping_key (Optional[str]): The key that identifies a unique mapped output. None for regular outputs. config (Optional[Any]): The configuration for the output. dagster_type (Optional[DagsterType]): The type of this output. version (Optional[str]): (Experimental) The version of the output. resource_config (Optional[Dict[str, Any]]): The resource config to make available from the input context. This usually corresponds to the config provided to the resource that loads the output manager. resources (Optional[Resources]): The resources to make available from the context. For a given key, you can provide either an actual instance of an object, or a resource definition. solid_def (Optional[SolidDefinition]): The definition of the solid that produced the output. op_def (Optional[OpDefinition]): The definition of the solid that produced the output. Examples: .. code-block:: python build_output_context() with build_output_context(resources={"foo": context_manager_resource}) as context: do_something """ from dagster.core.types.dagster_type import DagsterType from dagster.core.execution.context_creation_pipeline import initialize_console_manager step_key = check.opt_str_param(step_key, "step_key") name = check.opt_str_param(name, "name") metadata = check.opt_dict_param(metadata, "metadata", key_type=str) run_id = check.opt_str_param(run_id, "run_id", default=RUN_ID_PLACEHOLDER) mapping_key = check.opt_str_param(mapping_key, "mapping_key") dagster_type = check.opt_inst_param(dagster_type, "dagster_type", DagsterType) version = check.opt_str_param(version, "version") resource_config = check.opt_dict_param(resource_config, "resource_config", key_type=str) resources = check.opt_dict_param(resources, "resources", key_type=str) solid_def = check.opt_inst_param(solid_def, "solid_def", SolidDefinition) op_def = check.opt_inst_param(op_def, "op_def", OpDefinition) return OutputContext( step_key=step_key, name=name, pipeline_name=None, run_id=run_id, metadata=metadata, mapping_key=mapping_key, config=config, solid_def=solid_def, dagster_type=dagster_type, log_manager=initialize_console_manager(None), version=version, resource_config=resource_config, resources=resources, step_context=None, op_def=op_def, )