Resource hints

Resource hints let pipeline authors provide information to a runner about compute resource requirements. You can use resource hints to define requirements for specific transforms or for an entire pipeline. The runner is responsible for interpreting resource hints, and runners can ignore unsupported hints.

Resource hints can be nested. For example, resource hints can be specified on subtransforms of a composite transform, and that composite transform can also have resource hints applied. By default, the innermost hint takes precedence. However, hints can define custom reconciliation behavior. For example, min_ram takes the maximum value for all min_ram values set on a given step in the pipeline.

Available hints

Currently, Beam supports the following resource hints:

The interpretaton and actuation of resource hints can vary between runners. For an example implementation, see the Dataflow resource hints.

Specifying resource hints for a pipeline

To specify resource hints for an entire pipeline, you can use pipeline options. The following command shows the basic syntax.

mvn compile exec:java -Dexec.mainClass=com.example.MyPipeline \
    -Dexec.args="... \
                 --resourceHints=min_ram=<N>GB \
                 --resourceHints=accelerator='hint'" \
    -Pdirect-runner
python my_pipeline.py \
    ... \
    --resource_hints min_ram=<N>GB \
    --resource_hints accelerator="hint"
python -m apache_beam.yaml.main
    ... \
    --resource_hints min_ram=<N>GB \
    --resource_hints accelerator="hint"

Specifying resource hints for a transform

You can set resource hints programmatically on pipeline transforms using setResourceHints.

You can set resource hints programmatically on pipeline transforms using PTransforms.with_resource_hints (also see ResourceHint).

You can set resource hints pipeline transforms using a resource_hints attribute.

pcoll.apply(MyCompositeTransform.of(...)
    .setResourceHints(
        ResourceHints.create()
            .withMinRam("15GB")
            .withAccelerator("type:nvidia-tesla-k80;count:1;install-nvidia-driver")))

pcoll.apply(ParDo.of(new BigMemFn())
    .setResourceHints(
        ResourceHints.create().withMinRam("30GB")))
pcoll | MyPTransform().with_resource_hints(
    min_ram="4GB",
    accelerator="type:nvidia-tesla-k80;count:1;install-nvidia-driver")

pcoll | beam.ParDo(BigMemFn()).with_resource_hints(
    min_ram="30GB")
  - type: RunInference
    config:
      ...
    resource_hints:
      min_ram: 4GB
      accelerator: "type:nvidia-tesla-k80;count:1;install-nvidia-driver"

  - type: MapToFields
    config:
      ...
    resource_hints:
      min_ram: 30GB

Such a resource_hints attribute can also be placed on the top-level pipeline object to apply to the entire pipeline.