def __init__(self,
                 sources,
                 max_reader_threads=MAX_SOURCE_READER_THREADS,
                 read_counter=None,
                 element_counter=None):
        self.sources = sources
        self.num_reader_threads = min(max_reader_threads, len(self.sources))

        # Queue for sources that are to be read.
        self.sources_queue = queue.Queue()
        for source in sources:
            self.sources_queue.put(source)
        # Queue for elements that have been read.
        self.element_queue = queue.Queue(ELEMENT_QUEUE_SIZE)
        # Queue for exceptions encountered in reader threads; to be rethrown.
        self.reader_exceptions = queue.Queue()
        # Whether we have already iterated; this iterable can only be used once.
        self.already_iterated = False
        # Whether an error was encountered in any source reader.
        self.has_errored = False

        self.read_counter = read_counter or opcounters.NoOpTransformIOCounter()
        self.element_counter = element_counter
        self.reader_threads = []
        self._start_reader_threads()
Exemple #2
0
def run_benchmark(num_runs=50, input_per_source=4000, num_sources=4):
  print("Number of runs:", num_runs)
  print("Input size:", num_sources * input_per_source)
  print("Sources:", num_sources)

  times = []
  for i in range(num_runs):
    counter_factory = CounterFactory()
    state_sampler = statesampler.StateSampler('basic', counter_factory)
    with state_sampler.scoped_state('step1', 'state'):
      si_counter = opcounters.SideInputReadCounter(
          counter_factory, state_sampler, 'step1', 1)
      si_counter = opcounters.NoOpTransformIOCounter()
      sources = [
          FakeSource(long_generator(i, input_per_source))
          for i in range(num_sources)]
      iterator_fn = sideinputs.get_iterator_fn_for_sources(
          sources, read_counter=si_counter)
      start = time.time()
      list(iterator_fn())
      time_cost = time.time() - start
      times.append(time_cost)

  print("Runtimes:", times)

  avg_runtime = sum(times) // len(times)
  print("Average runtime:", avg_runtime)
  print("Time per element:", avg_runtime // (input_per_source *
                                             num_sources))
    def _read_side_inputs(self, tags_and_types):
        """Generator reading side inputs in the order prescribed by tags_and_types.

    Args:
      tags_and_types: List of tuples (tag, type). Each side input has a string
        tag that is specified in the worker instruction. The type is actually
        a boolean which is True for singleton input (read just first value)
        and False for collection input (read all values).

    Yields:
      With each iteration it yields the result of reading an entire side source
      either in singleton or collection mode according to the tags_and_types
      argument.
    """
        # Only call this on the old path where side_input_maps was not
        # provided directly.
        assert self.side_input_maps is None

        # We will read the side inputs in the order prescribed by the
        # tags_and_types argument because this is exactly the order needed to
        # replace the ArgumentPlaceholder objects in the args/kwargs of the DoFn
        # getting the side inputs.
        #
        # Note that for each tag there could be several read operations in the
        # specification. This can happen for instance if the source has been
        # sharded into several files.
        for i, (side_tag, view_class,
                view_options) in enumerate(tags_and_types):
            sources = []
            # Using the side_tag in the lambda below will trigger a pylint warning.
            # However in this case it is fine because the lambda is used right away
            # while the variable has the value assigned by the current iteration of
            # the for loop.
            # pylint: disable=cell-var-from-loop
            for si in itertools.ifilter(lambda o: o.tag == side_tag,
                                        self.spec.side_inputs):
                if not isinstance(si, operation_specs.WorkerSideInputSource):
                    raise NotImplementedError('Unknown side input type: %r' %
                                              si)
                sources.append(si.source)
                # The tracking of time spend reading and bytes read from side inputs is
                # behind an experiment flag to test its performance impact.
                if 'sideinput_io_metrics' in RuntimeValueProvider.experiments:
                    si_counter = opcounters.SideInputReadCounter(
                        self.counter_factory,
                        self.state_sampler,
                        declaring_step=self.name_context.step_name,
                        # Inputs are 1-indexed, so we add 1 to i in the side input id
                        input_index=i + 1)
                else:
                    si_counter = opcounters.NoOpTransformIOCounter()
            iterator_fn = sideinputs.get_iterator_fn_for_sources(
                sources, read_counter=si_counter)

            # Backwards compatibility for pre BEAM-733 SDKs.
            if isinstance(view_options, tuple):
                if view_class == pvalue.AsSingleton:
                    has_default, default = view_options
                    view_options = {'default': default} if has_default else {}
                else:
                    view_options = {}

            yield apache_sideinputs.SideInputMap(
                view_class, view_options,
                sideinputs.EmulatedIterable(iterator_fn))