def _read_cache(self, pipeline, pcoll, is_unbounded_source_output): """Reads a cached pvalue. A noop will cause the pipeline to execute the transform as it is and cache nothing from this transform for next run. Modifies: pipeline """ # Makes sure the pcoll belongs to the pipeline being instrumented. if pcoll.pipeline is not pipeline: return # The keyed cache is always valid within this instrumentation. key = self.cache_key(pcoll) # Can only read from cache when the cache with expected key exists and its # computation has been completed. is_cached = self._cache_manager.exists('full', key) is_computed = (pcoll in self._runner_pcoll_to_user_pcoll and self._runner_pcoll_to_user_pcoll[pcoll] in ie.current_env().computed_pcollections) if ((is_cached and is_computed) or is_unbounded_source_output): if key not in self._cached_pcoll_read: # Mutates the pipeline with cache read transform attached # to root of the pipeline. # To put the cached value into the correct window, simply return a # WindowedValue constructed from the element. pcoll_from_cache = unreify_from_cache( pipeline=pipeline, cache_key=key, cache_manager=self._cache_manager) self._cached_pcoll_read[key] = pcoll_from_cache
def _build_query_components( query: str, found: Dict[str, beam.PCollection], output_name: str, run: bool = True ) -> Tuple[str, Union[Dict[str, beam.PCollection], beam.PCollection, beam.Pipeline], SqlChain]: """Builds necessary components needed to apply the SqlTransform. Args: query: The SQL query to be executed by the magic. found: The PCollections with variable names found to be used by the query. output_name: The output variable name in __main__ module. run: Whether to prepare components for a local run or not. Returns: The processed query to be executed by the magic; a source to apply the SqlTransform to: a dictionary of tagged PCollections, or a single PCollection, or the pipeline to execute the query; the chain of applied beam_sql magics this one belongs to. """ if found: user_pipeline = ie.current_env().user_pipeline( next(iter(found.values())).pipeline) sql_pipeline = beam.Pipeline(options=user_pipeline._options) ie.current_env().add_derived_pipeline(user_pipeline, sql_pipeline) sql_source = {} if run: if has_source_to_cache(user_pipeline): sql_source = pcolls_from_streaming_cache( user_pipeline, sql_pipeline, found) else: cache_manager = ie.current_env().get_cache_manager( user_pipeline, create_if_absent=True) for pcoll_name, pcoll in found.items(): cache_key = CacheKey.from_pcoll(pcoll_name, pcoll).to_str() sql_source[pcoll_name] = unreify_from_cache( pipeline=sql_pipeline, cache_key=cache_key, cache_manager=cache_manager, element_type=pcoll.element_type) else: sql_source = found if len(sql_source) == 1: query = replace_single_pcoll_token(query, next(iter(sql_source.keys()))) sql_source = next(iter(sql_source.values())) node = SqlNode( output_name=output_name, source=set(found.keys()), query=query) chain = ie.current_env().get_sql_chain( user_pipeline, set_user_pipeline=True).append(node) else: # does not query any existing PCollection sql_source = beam.Pipeline() ie.current_env().add_user_pipeline(sql_source) # The node should be the root node of the chain created below. node = SqlNode(output_name=output_name, source=sql_source, query=query) chain = ie.current_env().get_sql_chain(sql_source).append(node) return query, sql_source, chain
def _build_query_components( query: str, found: Dict[str, beam.PCollection] ) -> Tuple[str, Union[Dict[str, beam.PCollection], beam.PCollection, beam.Pipeline]]: """Builds necessary components needed to apply the SqlTransform. Args: query: The SQL query to be executed by the magic. found: The PCollections with variable names found to be used by the query. Returns: The processed query to be executed by the magic and a source to apply the SqlTransform to: a dictionary of tagged PCollections, or a single PCollection, or the pipeline to execute the query. """ if found: user_pipeline = ie.current_env().user_pipeline( next(iter(found.values())).pipeline) sql_pipeline = beam.Pipeline(options=user_pipeline._options) ie.current_env().add_derived_pipeline(user_pipeline, sql_pipeline) sql_source = {} if has_source_to_cache(user_pipeline): sql_source = pcolls_from_streaming_cache(user_pipeline, sql_pipeline, found) else: cache_manager = ie.current_env().get_cache_manager( user_pipeline, create_if_absent=True) for pcoll_name, pcoll in found.items(): cache_key = CacheKey.from_pcoll(pcoll_name, pcoll).to_str() sql_source[pcoll_name] = unreify_from_cache( pipeline=sql_pipeline, cache_key=cache_key, cache_manager=cache_manager, element_type=pcoll.element_type) if len(sql_source) == 1: query = replace_single_pcoll_token(query, next(iter(sql_source.keys()))) sql_source = next(iter(sql_source.values())) else: sql_source = beam.Pipeline() ie.current_env().add_user_pipeline(sql_source) return query, sql_source
def expand(self, pcoll: beam.pvalue.PCollection) -> beam.pvalue.PCollection: return unreify_from_cache(pipeline=pcoll.pipeline, cache_key=self._key, cache_manager=self._cache_manager)