def test_dataframes_with_multi_index_get_result(self): p = beam.Pipeline(runner=interactive_runner.InteractiveRunner( direct_runner.DirectRunner())) data = [ Record('a', 20, 170), Record('a', 30, 170), Record('b', 22, 180), Record('c', 18, 150) ] aggregate = lambda df: df.groupby(['name', 'height']).mean()['age'] deferred_df = aggregate(to_dataframe(p | beam.Create(data))) df_expected = aggregate(pd.DataFrame(data)) # Watch the local scope for Interactive Beam so that values will be cached. ib.watch(locals()) # This is normally done in the interactive_utils when a transform is # applied but needs an IPython environment. So we manually run this here. ie.current_env().track_user_pipelines() pd.testing.assert_series_equal(df_expected, ib.collect(deferred_df, n=10))
def find_pcolls( sql: str, pcolls: Dict[str, beam.PCollection]) -> Dict[str, beam.PCollection]: """Finds all PCollections used in the given sql query. It does a simple word by word match and calls ib.collect for each PCollection found. """ found = {} for word in sql.split(): if word in pcolls: found[word] = pcolls[word] if found: _LOGGER.info('Found PCollections used in the magic: %s.', found) _LOGGER.info('Collecting data...') for name, pcoll in found.items(): try: _ = ib.collect(pcoll) except (KeyboardInterrupt, SystemExit): raise except: _LOGGER.error( 'Cannot collect data for PCollection %s. Please make sure the ' 'PCollections queried in the sql "%s" are all from a single ' 'pipeline using an InteractiveRunner. Make sure there is no ' 'ambiguity, for example, same named PCollections from multiple ' 'pipelines or notebook re-executions.', name, sql) raise _LOGGER.info('Done collecting data.') return found
def test_recordings_clear(self): """Tests that clearing the pipeline is correctly forwarded.""" # Create a basic pipeline to store something in the cache. p = beam.Pipeline(ir.InteractiveRunner()) elem = p | beam.Create([1]) ib.watch(locals()) ie.current_env().track_user_pipelines() # This records the pipeline so that the cache size is > 0. ib.collect(elem) self.assertGreater(ib.recordings.describe(p)['size'], 0) # After clearing, the cache should be empty. ib.recordings.clear(p) self.assertEqual(ib.recordings.describe(p)['size'], 0)
def test_dataframes(self): p = beam.Pipeline(runner=interactive_runner.InteractiveRunner( direct_runner.DirectRunner())) data = p | beam.Create([ 1, 2, 3 ]) | beam.Map(lambda x: beam.Row(square=x * x, cube=x * x * x)) df = to_dataframe(data) pcoll = to_pcollection(df) # Watch the local scope for Interactive Beam so that values will be cached. ib.watch(locals()) # This is normally done in the interactive_utils when a transform is # applied but needs an IPython environment. So we manually run this here. ie.current_env().track_user_pipelines() df_expected = pd.DataFrame({'square': [1, 4, 9], 'cube': [1, 8, 27]}) pd.testing.assert_frame_equal(df_expected, ib.collect(data, n=10)) pd.testing.assert_frame_equal(df_expected, ib.collect(df, n=10)) pd.testing.assert_frame_equal(df_expected, ib.collect(pcoll, n=10))
def test_get_pcoll_data(self): pipeline = beam.Pipeline(ir.InteractiveRunner()) # pylint: disable=range-builtin-not-iterating pcoll = pipeline | 'Create' >> beam.Create(range(10)) counts = pcoll | beam.combiners.Count.PerElement() ib.watch(locals()) counts_identifier = obfuscate(inspector.meta('counts', counts)) ins = inspector.InteractiveEnvironmentInspector() _ = ins.list_inspectables() actual_counts_pcoll_data = ins.get_pcoll_data(counts_identifier) expected_counts_pcoll_data = ib.collect(counts).to_json(orient='table') self.assertEqual(actual_counts_pcoll_data, expected_counts_pcoll_data) actual_counts_with_window_info = ins.get_pcoll_data(counts_identifier, True) expected_counts_with_window_info = ib.collect(counts, True).to_json(orient='table') self.assertEqual( actual_counts_with_window_info, expected_counts_with_window_info)
def get_pcoll_data(self, identifier, include_window_info=False): """Retrieves the json formatted PCollection data. If no PCollection value can be retieved from the given identifier, an empty json string will be returned. """ value = self.get_val(identifier) if isinstance(value, beam.pvalue.PCollection): from apache_beam.runners.interactive import interactive_beam as ib dataframe = ib.collect(value, include_window_info) return dataframe.to_json(orient='table') return {}
def test_get_pcoll_data(self): pipeline = beam.Pipeline(ir.InteractiveRunner()) # pylint: disable=bad-option-value pcoll = pipeline | 'Create' >> beam.Create(list(range(10))) counts = pcoll | beam.combiners.Count.PerElement() ib.watch(locals()) ie.current_env().track_user_pipelines() counts_identifier = obfuscate(inspector.meta('counts', counts)) ins = inspector.InteractiveEnvironmentInspector() _ = ins.list_inspectables() actual_counts_pcoll_data = ins.get_pcoll_data(counts_identifier) expected_counts_pcoll_data = ib.collect( counts, n=10).to_json(orient='table') self.assertEqual(actual_counts_pcoll_data, expected_counts_pcoll_data) actual_counts_with_window_info = ins.get_pcoll_data(counts_identifier, True) expected_counts_with_window_info = ib.collect( counts, include_window_info=True).to_json(orient='table') self.assertEqual( actual_counts_with_window_info, expected_counts_with_window_info)
def collect_data_for_local_run(query: str, found: Dict[str, beam.PCollection]): from apache_beam.runners.interactive import interactive_beam as ib for name, pcoll in found.items(): try: _ = ib.collect(pcoll) except (KeyboardInterrupt, SystemExit): raise except: _LOGGER.error( 'Cannot collect data for PCollection %s. Please make sure the ' 'PCollections queried in the sql "%s" are all from a single ' 'pipeline using an InteractiveRunner. Make sure there is no ' 'ambiguity, for example, same named PCollections from multiple ' 'pipelines or notebook re-executions.', name, query) raise
def test_wordcount(self): class WordExtractingDoFn(beam.DoFn): def process(self, element): text_line = element.strip() words = text_line.split() return words p = beam.Pipeline(runner=interactive_runner.InteractiveRunner( direct_runner.DirectRunner())) # Count the occurrences of each word. counts = (p | beam.Create(['to be or not to be that is the question']) | 'split' >> beam.ParDo(WordExtractingDoFn()) | 'pair_with_one' >> beam.Map(lambda x: (x, 1)) | 'group' >> beam.GroupByKey() | 'count' >> beam.Map(lambda wordones: (wordones[0], sum(wordones[1])))) # Watch the local scope for Interactive Beam so that counts will be cached. ib.watch(locals()) result = p.run() result.wait_until_finish() actual = list(result.get(counts)) self.assertSetEqual( set(actual), set([ ('or', 1), ('that', 1), ('be', 2), ('is', 1), ('question', 1), ('to', 2), ('the', 1), ('not', 1), ])) # Truncate the precision to millis because the window coder uses millis # as units then gets upcast to micros. end_of_window = (GlobalWindow().max_timestamp().micros // 1000) * 1000 df_counts = ib.collect(counts, include_window_info=True) df_expected = pd.DataFrame( { 0: [e[0] for e in actual], 1: [e[1] for e in actual], 'event_time': [end_of_window for _ in actual], 'windows': [[GlobalWindow()] for _ in actual], 'pane_info': [ PaneInfo(True, True, PaneInfoTiming.ON_TIME, 0, 0) for _ in actual ] }, columns=[0, 1, 'event_time', 'windows', 'pane_info']) pd.testing.assert_frame_equal(df_expected, df_counts) actual_reified = result.get(counts, include_window_info=True) expected_reified = [ WindowedValue(e, Timestamp(micros=end_of_window), [GlobalWindow()], PaneInfo(True, True, PaneInfoTiming.ON_TIME, 0, 0)) for e in actual ] self.assertEqual(actual_reified, expected_reified)
def test_streaming_wordcount(self): class WordExtractingDoFn(beam.DoFn): def process(self, element): text_line = element.strip() words = text_line.split() return words # Add the TestStream so that it can be cached. ib.options.capturable_sources.add(TestStream) p = beam.Pipeline(runner=interactive_runner.InteractiveRunner(), options=StandardOptions(streaming=True)) data = ( p | TestStream() .advance_watermark_to(0) .advance_processing_time(1) .add_elements(['to', 'be', 'or', 'not', 'to', 'be']) .advance_watermark_to(20) .advance_processing_time(1) .add_elements(['that', 'is', 'the', 'question']) | beam.WindowInto(beam.window.FixedWindows(10))) # yapf: disable counts = (data | 'split' >> beam.ParDo(WordExtractingDoFn()) | 'pair_with_one' >> beam.Map(lambda x: (x, 1)) | 'group' >> beam.GroupByKey() | 'count' >> beam.Map(lambda wordones: (wordones[0], sum(wordones[1])))) # Watch the local scope for Interactive Beam so that referenced PCollections # will be cached. ib.watch(locals()) # This is normally done in the interactive_utils when a transform is # applied but needs an IPython environment. So we manually run this here. ie.current_env().track_user_pipelines() # Create a fake limiter that cancels the BCJ once the main job receives the # expected amount of results. class FakeLimiter: def __init__(self, p, pcoll): self.p = p self.pcoll = pcoll def is_triggered(self): result = ie.current_env().pipeline_result(self.p) if result: try: results = result.get(self.pcoll) except ValueError: return False return len(results) >= 10 return False # This sets the limiters to stop reading when the test receives 10 elements. ie.current_env().options.capture_control.set_limiters_for_test( [FakeLimiter(p, data)]) # This tests that the data was correctly cached. pane_info = PaneInfo(True, True, PaneInfoTiming.UNKNOWN, 0, 0) expected_data_df = pd.DataFrame([ ('to', 0, [IntervalWindow(0, 10)], pane_info), ('be', 0, [IntervalWindow(0, 10)], pane_info), ('or', 0, [IntervalWindow(0, 10)], pane_info), ('not', 0, [IntervalWindow(0, 10)], pane_info), ('to', 0, [IntervalWindow(0, 10)], pane_info), ('be', 0, [IntervalWindow(0, 10)], pane_info), ('that', 20000000, [IntervalWindow(20, 30)], pane_info), ('is', 20000000, [IntervalWindow(20, 30)], pane_info), ('the', 20000000, [IntervalWindow(20, 30)], pane_info), ('question', 20000000, [IntervalWindow(20, 30)], pane_info) ], columns=[0, 'event_time', 'windows', 'pane_info']) # yapf: disable data_df = ib.collect(data, include_window_info=True) pd.testing.assert_frame_equal(expected_data_df, data_df) # This tests that the windowing was passed correctly so that all the data # is aggregated also correctly. pane_info = PaneInfo(True, False, PaneInfoTiming.ON_TIME, 0, 0) expected_counts_df = pd.DataFrame([ ('be', 2, 9999999, [IntervalWindow(0, 10)], pane_info), ('not', 1, 9999999, [IntervalWindow(0, 10)], pane_info), ('or', 1, 9999999, [IntervalWindow(0, 10)], pane_info), ('to', 2, 9999999, [IntervalWindow(0, 10)], pane_info), ('is', 1, 29999999, [IntervalWindow(20, 30)], pane_info), ('question', 1, 29999999, [IntervalWindow(20, 30)], pane_info), ('that', 1, 29999999, [IntervalWindow(20, 30)], pane_info), ('the', 1, 29999999, [IntervalWindow(20, 30)], pane_info), ], columns=[0, 1, 'event_time', 'windows', 'pane_info']) # yapf: disable counts_df = ib.collect(counts, include_window_info=True) # The group by key has no guarantee of order. So we post-process the DF by # sorting so we can test equality. sorted_counts_df = (counts_df .sort_values(['event_time', 0], ascending=True) .reset_index(drop=True)) # yapf: disable pd.testing.assert_frame_equal(expected_counts_df, sorted_counts_df)
def test_dataframe_caching(self, cell): # Create a pipeline that exercises the DataFrame API. This will also use # caching in the background. with cell: # Cell 1 p = beam.Pipeline(interactive_runner.InteractiveRunner()) ib.watch({'p': p}) with cell: # Cell 2 data = p | beam.Create([ 1, 2, 3 ]) | beam.Map(lambda x: beam.Row(square=x * x, cube=x * x * x)) with beam.dataframe.allow_non_parallel_operations(): df = to_dataframe(data).reset_index(drop=True) ib.collect(df) with cell: # Cell 3 df['output'] = df['square'] * df['cube'] ib.collect(df) with cell: # Cell 4 df['output'] = 0 ib.collect(df) # We use a trace through the graph to perform an isomorphism test. The end # output should look like a linear graph. This indicates that the dataframe # transform was correctly broken into separate pieces to cache. If caching # isn't enabled, all the dataframe computation nodes are connected to a # single shared node. trace = [] # Only look at the top-level transforms for the isomorphism. The test # doesn't care about the transform implementations, just the overall shape. class TopLevelTracer(beam.pipeline.PipelineVisitor): def _find_root_producer(self, node: beam.pipeline.AppliedPTransform): if node is None or not node.full_label: return None parent = self._find_root_producer(node.parent) if parent is None: return node return parent def _add_to_trace(self, node, trace): if '/' not in str(node): if node.inputs: producer = self._find_root_producer( node.inputs[0].producer) producer_name = producer.full_label if producer else '' trace.append((producer_name, node.full_label)) def visit_transform(self, node: beam.pipeline.AppliedPTransform): self._add_to_trace(node, trace) def enter_composite_transform( self, node: beam.pipeline.AppliedPTransform): self._add_to_trace(node, trace) p.visit(TopLevelTracer()) # Do the isomorphism test which states that the topological sort of the # graph yields a linear graph. trace_string = '\n'.join(str(t) for t in trace) prev_producer = '' for producer, consumer in trace: self.assertEqual(producer, prev_producer, trace_string) prev_producer = consumer
def test_streaming_wordcount(self): self.skipTest('[BEAM-9601] Test is breaking PreCommits') class WordExtractingDoFn(beam.DoFn): def process(self, element): text_line = element.strip() words = text_line.split() return words # Add the TestStream so that it can be cached. ib.options.capturable_sources.add(TestStream) ib.options.capture_duration = timedelta(seconds=1) p = beam.Pipeline(runner=interactive_runner.InteractiveRunner(), options=StandardOptions(streaming=True)) data = ( p | TestStream() .advance_watermark_to(0) .advance_processing_time(1) .add_elements(['to', 'be', 'or', 'not', 'to', 'be']) .advance_watermark_to(20) .advance_processing_time(1) .add_elements(['that', 'is', 'the', 'question']) | beam.WindowInto(beam.window.FixedWindows(10))) # yapf: disable counts = (data | 'split' >> beam.ParDo(WordExtractingDoFn()) | 'pair_with_one' >> beam.Map(lambda x: (x, 1)) | 'group' >> beam.GroupByKey() | 'count' >> beam.Map(lambda wordones: (wordones[0], sum(wordones[1])))) # Watch the local scope for Interactive Beam so that referenced PCollections # will be cached. ib.watch(locals()) # This is normally done in the interactive_utils when a transform is # applied but needs an IPython environment. So we manually run this here. ie.current_env().track_user_pipelines() # This tests that the data was correctly cached. pane_info = PaneInfo(True, True, PaneInfoTiming.UNKNOWN, 0, 0) expected_data_df = pd.DataFrame( [('to', 0, [beam.window.IntervalWindow(0, 10)], pane_info), ('be', 0, [beam.window.IntervalWindow(0, 10)], pane_info), ('or', 0, [beam.window.IntervalWindow(0, 10)], pane_info), ('not', 0, [beam.window.IntervalWindow(0, 10)], pane_info), ('to', 0, [beam.window.IntervalWindow(0, 10)], pane_info), ('be', 0, [beam.window.IntervalWindow(0, 10)], pane_info), ('that', 20000000, [beam.window.IntervalWindow(20, 30) ], pane_info), ('is', 20000000, [beam.window.IntervalWindow(20, 30)], pane_info), ('the', 20000000, [beam.window.IntervalWindow(20, 30) ], pane_info), ('question', 20000000, [beam.window.IntervalWindow(20, 30) ], pane_info)], columns=[0, 'event_time', 'windows', 'pane_info']) data_df = ib.collect(data, include_window_info=True) pd.testing.assert_frame_equal(expected_data_df, data_df) # This tests that the windowing was passed correctly so that all the data # is aggregated also correctly. pane_info = PaneInfo(True, False, PaneInfoTiming.ON_TIME, 0, 0) expected_counts_df = pd.DataFrame([ ('to', 2, 9999999, [beam.window.IntervalWindow(0, 10)], pane_info), ('be', 2, 9999999, [beam.window.IntervalWindow(0, 10)], pane_info), ('or', 1, 9999999, [beam.window.IntervalWindow(0, 10)], pane_info), ('not', 1, 9999999, [beam.window.IntervalWindow(0, 10) ], pane_info), ('that', 1, 29999999, [beam.window.IntervalWindow(20, 30) ], pane_info), ('is', 1, 29999999, [beam.window.IntervalWindow(20, 30) ], pane_info), ('the', 1, 29999999, [beam.window.IntervalWindow(20, 30) ], pane_info), ('question', 1, 29999999, [beam.window.IntervalWindow(20, 30) ], pane_info) ], columns=[ 0, 1, 'event_time', 'windows', 'pane_info' ]) counts_df = ib.collect(counts, include_window_info=True) pd.testing.assert_frame_equal(expected_counts_df, counts_df)
def test_read_in_interactive_runner(self): p = beam.Pipeline(InteractiveRunner(), argv=self.args) pcoll = p | beam.io.ReadFromBigQuery(query="SELECT 1") result = interactive_beam.collect(pcoll) assert result.iloc[0, 0] == 1