github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/examples/complete/distribopt.py (about)

     1  #
     2  # Licensed to the Apache Software Foundation (ASF) under one or more
     3  # contributor license agreements.  See the NOTICE file distributed with
     4  # this work for additional information regarding copyright ownership.
     5  # The ASF licenses this file to You under the Apache License, Version 2.0
     6  # (the "License"); you may not use this file except in compliance with
     7  # the License.  You may obtain a copy of the License at
     8  #
     9  #    http://www.apache.org/licenses/LICENSE-2.0
    10  #
    11  # Unless required by applicable law or agreed to in writing, software
    12  # distributed under the License is distributed on an "AS IS" BASIS,
    13  # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    14  # See the License for the specific language governing permissions and
    15  # limitations under the License.
    16  #
    17  
    18  """Example illustrating the use of Apache Beam for solving distributing
    19  optimization tasks.
    20  
    21  This example solves an optimization problem which consists of distributing a
    22  number of crops to grow in several greenhouses. The decision where to grow the
    23  crop has an impact on the production parameters associated with the greenhouse,
    24  which affects the total cost of production at the greenhouse. Additionally,
    25  each crop needs to be transported to a customer so the decision where to grow
    26  the crop has an impact on the transportation costs as well.
    27  
    28  This type of optimization problems are known as mixed-integer programs as they
    29  exist of discrete parameters (do we produce a crop in greenhouse A, B or C?)
    30  and continuous parameters (the greenhouse production parameters).
    31  
    32  Running this example requires NumPy and SciPy. The input consists of a CSV file
    33  with the following columns (Tx representing the transporation cost/unit if the
    34  crop is produced in greenhouse x): Crop name, Quantity, Ta, Tb, Tc, ....
    35  
    36  Example input file with 5 crops and 3 greenhouses (a transporation cost of 0
    37  forbids production of the crop in a greenhouse):
    38  OP01,8,12,0,12
    39  OP02,30,14,3,12
    40  OP03,25,7,3,14
    41  OP04,87,7,2,2
    42  OP05,19,1,7,10
    43  
    44  The pipeline consists of three phases:
    45    - Creating a grid of mappings (assignment of each crop to a greenhouse)
    46    - For each mapping and each greenhouse, optimization of the production
    47      parameters for cost, addition of the transporation costs, and aggregation
    48      of the costs for each mapping.
    49    - Selecting the mapping with the lowest cost.
    50  """
    51  
    52  # pytype: skip-file
    53  
    54  import argparse
    55  import logging
    56  import string
    57  import uuid
    58  from collections import defaultdict
    59  
    60  import numpy as np
    61  from scipy.optimize import minimize
    62  
    63  import apache_beam as beam
    64  from apache_beam import pvalue
    65  from apache_beam.options.pipeline_options import PipelineOptions
    66  from apache_beam.options.pipeline_options import SetupOptions
    67  
    68  
    69  class Simulator(object):
    70    """Greenhouse simulation for the optimization of greenhouse parameters."""
    71    def __init__(self, quantities):
    72      self.quantities = np.atleast_1d(quantities)
    73  
    74      self.A = np.array([[3.0, 10, 30], [0.1, 10, 35], [3.0, 10, 30],
    75                         [0.1, 10, 35]])
    76  
    77      self.P = 1e-4 * np.array([[3689, 1170, 2673], [4699, 4387, 7470],
    78                                [1091, 8732, 5547], [381, 5743, 8828]])
    79  
    80      a0 = np.array([[1.0, 1.2, 3.0, 3.2]])
    81      coeff = np.sum(np.cos(np.dot(a0.T, self.quantities[None, :])), axis=1)
    82      self.alpha = coeff / np.sum(coeff)
    83  
    84    def simulate(self, xc):
    85      # Map the input parameter to a cost for each crop.
    86      weighted_distance = np.sum(self.A * np.square(xc - self.P), axis=1)
    87      f = -np.sum(self.alpha * np.exp(-weighted_distance))
    88      return np.square(f) * np.log(self.quantities)
    89  
    90  
    91  class CreateGrid(beam.PTransform):
    92    """A transform for generating the mapping grid.
    93  
    94    Input: Formatted records of the input file, e.g.,
    95    {
    96      'crop': 'OP009',
    97      'quantity': 102,
    98      'transport_costs': [('A', None), ('B', 3), ('C', 8)]
    99    }
   100    Output: tuple (mapping_identifier, {crop -> greenhouse})
   101    """
   102    class PreGenerateMappings(beam.DoFn):
   103      """ParDo implementation forming based on two elements a small sub grid.
   104  
   105      This facilitates parallellization of the grid generation.
   106      Emits two PCollections: the subgrid represented as collection of lists of
   107      two tuples, and a list of remaining records. Both serve as an input to
   108      GenerateMappings.
   109      """
   110      def process(self, element):
   111        records = list(element[1])
   112        # Split of 2 crops and pre-generate the subgrid.
   113        # Select the crop with highest number of possible greenhouses:
   114        # in case two crops with only a single possible greenhouse were selected
   115        # the subgrid would consist of only 1 element.
   116        best_split = np.argsort([-len(r['transport_costs']) for r in records])[:2]
   117        rec1 = records[best_split[0]]
   118        rec2 = records[best_split[1]]
   119  
   120        # Generate & emit all combinations
   121        for a in rec1['transport_costs']:
   122          if a[1]:
   123            for b in rec2['transport_costs']:
   124              if b[1]:
   125                combination = [(rec1['crop'], a[0]), (rec2['crop'], b[0])]
   126                yield pvalue.TaggedOutput('splitted', combination)
   127  
   128        # Pass on remaining records
   129        remaining = [rec for i, rec in enumerate(records) if i not in best_split]
   130        yield pvalue.TaggedOutput('combine', remaining)
   131  
   132    class GenerateMappings(beam.DoFn):
   133      """ParDo implementation to generate all possible mappings.
   134  
   135      Input: output of PreGenerateMappings
   136      Output: tuples of the form (mapping_identifier, {crop -> greenhouse})
   137      """
   138      @staticmethod
   139      def _coordinates_to_greenhouse(coordinates, greenhouses, crops):
   140        # Map the grid coordinates back to greenhouse labels
   141        arr = []
   142        for coord in coordinates:
   143          arr.append(greenhouses[coord])
   144        return dict(zip(crops, np.array(arr)))
   145  
   146      def process(self, element, records):
   147        # Generate available greenhouses and grid coordinates for each crop.
   148        grid_coordinates = []
   149        for rec in records:
   150          # Get indices for available greenhouses (w.r.t crops)
   151          filtered = [i for i, av in enumerate(rec['transport_costs']) if av[1]]
   152          grid_coordinates.append(filtered)
   153  
   154        # Generate all mappings
   155        grid = np.vstack(list(map(np.ravel, np.meshgrid(*grid_coordinates)))).T
   156        crops = [rec['crop'] for rec in records]
   157        greenhouses = [rec[0] for rec in records[0]['transport_costs']]
   158        for point in grid:
   159          # translate back to greenhouse label
   160          mapping = self._coordinates_to_greenhouse(point, greenhouses, crops)
   161          assert all(rec[0] not in mapping for rec in element)
   162          # include the incomplete mapping of 2 crops
   163          mapping.update(element)
   164          # include identifier
   165          yield (uuid.uuid4().hex, mapping)
   166  
   167    def expand(self, records):
   168      o = (
   169          records
   170          | 'pair one' >> beam.Map(lambda x: (1, x))
   171          | 'group all records' >> beam.GroupByKey()
   172          | 'split one of' >> beam.ParDo(self.PreGenerateMappings()).with_outputs(
   173              'splitted', 'combine'))
   174  
   175      # Create mappings, and prevent fusion (this limits the parallelization
   176      # in the optimization step)
   177      mappings = (
   178          o.splitted
   179          | 'create mappings' >> beam.ParDo(
   180              self.GenerateMappings(), pvalue.AsSingleton(o.combine))
   181          | 'prevent fusion' >> beam.Reshuffle())
   182  
   183      return mappings
   184  
   185  
   186  class OptimizeGrid(beam.PTransform):
   187    """A transform for optimizing all greenhouses of the mapping grid."""
   188    class CreateOptimizationTasks(beam.DoFn):
   189      """
   190      Create tasks for optimization.
   191  
   192      Input: (mapping_identifier, {crop -> greenhouse})
   193      Output: ((mapping_identifier, greenhouse), [(crop, quantity),...])
   194      """
   195      def process(self, element, quantities):
   196        mapping_identifier, mapping = element
   197  
   198        # Create (crop, quantity) lists for each greenhouse
   199        greenhouses = defaultdict(list)
   200        for crop, greenhouse in mapping.items():
   201          quantity = quantities[crop]
   202          greenhouses[greenhouse].append((crop, quantity))
   203  
   204        # Create input for OptimizeProductParameters
   205        for greenhouse, crops in greenhouses.items():
   206          key = (mapping_identifier, greenhouse)
   207          yield (key, crops)
   208  
   209    class OptimizeProductParameters(beam.DoFn):
   210      """Solve the optimization task to determine optimal production parameters.
   211      Input: ((mapping_identifier, greenhouse), [(crop, quantity),...])
   212      Two outputs:
   213        - solution: (mapping_identifier, (greenhouse, [production parameters]))
   214        - costs: (crop, greenhouse, mapping_identifier, cost)
   215      """
   216      @staticmethod
   217      def _optimize_production_parameters(sim):
   218        # setup initial starting point & bounds
   219        x0 = 0.5 * np.ones(3)
   220        bounds = list(zip(np.zeros(3), np.ones(3)))
   221  
   222        # Run L-BFGS-B optimizer
   223        result = minimize(lambda x: np.sum(sim.simulate(x)), x0, bounds=bounds)
   224        return result.x.tolist(), sim.simulate(result.x)
   225  
   226      def process(self, element):
   227        mapping_identifier, greenhouse = element[0]
   228        crops, quantities = zip(*element[1])
   229        sim = Simulator(quantities)
   230        optimum, costs = self._optimize_production_parameters(sim)
   231        solution = (mapping_identifier, (greenhouse, optimum))
   232        yield pvalue.TaggedOutput('solution', solution)
   233        for crop, cost, quantity in zip(crops, costs, quantities):
   234          costs = (crop, greenhouse, mapping_identifier, cost * quantity)
   235          yield pvalue.TaggedOutput('costs', costs)
   236  
   237    def expand(self, inputs):
   238      mappings, quantities = inputs
   239      opt = (
   240          mappings
   241          | 'optimization tasks' >> beam.ParDo(
   242              self.CreateOptimizationTasks(), pvalue.AsDict(quantities))
   243          | 'optimize' >> beam.ParDo(
   244              self.OptimizeProductParameters()).with_outputs('costs', 'solution'))
   245      return opt
   246  
   247  
   248  class CreateTransportData(beam.DoFn):
   249    """Transform records to pvalues ((crop, greenhouse), transport_cost)"""
   250    def process(self, record):
   251      crop = record['crop']
   252      for greenhouse, transport_cost in record['transport_costs']:
   253        yield ((crop, greenhouse), transport_cost)
   254  
   255  
   256  def add_transport_costs(element, transport, quantities):
   257    """Adds the transport cost for the crop to the production cost.
   258  
   259    elements are of the form (crop, greenhouse, mapping, cost), the cost only
   260    corresponds to the production cost. Return the same format, but including
   261    the transport cost.
   262    """
   263    crop = element[0]
   264    cost = element[3]
   265    # lookup & compute cost
   266    transport_key = element[:2]
   267    transport_cost = transport[transport_key] * quantities[crop]
   268    return element[:3] + (cost + transport_cost, )
   269  
   270  
   271  def parse_input(line):
   272    # Process each line of the input file to a dict representing each crop
   273    # and the transport costs
   274    columns = line.split(',')
   275  
   276    # Assign each greenhouse a character
   277    transport_costs = []
   278    for greenhouse, cost in zip(string.ascii_uppercase, columns[2:]):
   279      info = (greenhouse, int(cost) if cost else None)
   280      transport_costs.append(info)
   281  
   282    return {
   283        'crop': columns[0],
   284        'quantity': int(columns[1]),
   285        'transport_costs': transport_costs
   286    }
   287  
   288  
   289  def format_output(element):
   290    """Transforms the datastructure (unpack lists introduced by CoGroupByKey)
   291    before writing the result to file.
   292    """
   293    result = element[1]
   294    result['cost'] = result['cost'][0]
   295    result['production'] = dict(result['production'])
   296    result['mapping'] = result['mapping'][0]
   297    return result
   298  
   299  
   300  def run(argv=None, save_main_session=True):
   301    parser = argparse.ArgumentParser()
   302    parser.add_argument(
   303        '--input',
   304        dest='input',
   305        required=True,
   306        help='Input description to process.')
   307    parser.add_argument(
   308        '--output',
   309        dest='output',
   310        required=True,
   311        help='Output file to write results to.')
   312    known_args, pipeline_args = parser.parse_known_args(argv)
   313    pipeline_options = PipelineOptions(pipeline_args)
   314    pipeline_options.view_as(SetupOptions).save_main_session = save_main_session
   315  
   316    with beam.Pipeline(options=pipeline_options) as p:
   317      # Parse input file
   318      records = (
   319          p
   320          | 'read' >> beam.io.ReadFromText(known_args.input)
   321          | 'process input' >> beam.Map(parse_input))
   322  
   323      # Create two pcollections, used as side inputs
   324      transport = (
   325          records
   326          | 'create transport' >> beam.ParDo(CreateTransportData()))
   327  
   328      quantities = (
   329          records
   330          | 'create quantities' >> beam.Map(lambda r: (r['crop'], r['quantity'])))
   331  
   332      # Generate all mappings and optimize greenhouse production parameters
   333      mappings = records | CreateGrid()
   334      opt = (mappings, quantities) | OptimizeGrid()
   335  
   336      # Then add the transport costs and sum costs per crop.
   337      costs = (
   338          opt.costs
   339          | 'include transport' >> beam.Map(
   340              add_transport_costs,
   341              pvalue.AsDict(transport),
   342              pvalue.AsDict(quantities))
   343          | 'drop crop and greenhouse' >> beam.Map(lambda x: (x[2], x[3]))
   344          | 'aggregate crops' >> beam.CombinePerKey(sum))
   345  
   346      # Join cost, mapping and production settings solution on mapping identifier.
   347      # Then select best.
   348      join_operands = {
   349          'cost': costs, 'production': opt.solution, 'mapping': mappings
   350      }
   351      best = (
   352          join_operands
   353          | 'join' >> beam.CoGroupByKey()
   354          | 'select best' >> beam.CombineGlobally(
   355              min, key=lambda x: x[1]['cost']).without_defaults()
   356          | 'format output' >> beam.Map(format_output))
   357  
   358      # pylint: disable=expression-not-assigned
   359      best | 'write optimum' >> beam.io.WriteToText(known_args.output)
   360  
   361  
   362  if __name__ == '__main__':
   363    logging.getLogger().setLevel(logging.INFO)
   364    run()