@Override
 public LocalProperties computeLocalProperties(LocalProperties lProps) {
   lProps.clearUniqueFieldSets();
   return lProps;
 }
  @Override
  protected void instantiate(
      OperatorDescriptorDual operator,
      Channel solutionSetIn,
      Channel worksetIn,
      List<Set<? extends NamedChannel>> broadcastPlanChannels,
      List<PlanNode> target,
      CostEstimator estimator,
      RequestedGlobalProperties globPropsReqSolutionSet,
      RequestedGlobalProperties globPropsReqWorkset,
      RequestedLocalProperties locPropsReqSolutionSet,
      RequestedLocalProperties locPropsReqWorkset) {
    // check for pipeline breaking using hash join with build on the solution set side
    placePipelineBreakersIfNecessary(
        DriverStrategy.HYBRIDHASH_BUILD_FIRST, solutionSetIn, worksetIn);

    // NOTES ON THE ENUMERATION OF THE STEP FUNCTION PLANS:
    // Whenever we instantiate the iteration, we enumerate new candidates for the step function.
    // That way, we make sure we have an appropriate plan for each candidate for the initial partial
    // solution,
    // we have a fitting candidate for the step function (often, work is pushed out of the step
    // function).
    // Among the candidates of the step function, we keep only those that meet the requested
    // properties of the
    // current candidate initial partial solution. That makes sure these properties exist at the
    // beginning of
    // every iteration.

    // 1) Because we enumerate multiple times, we may need to clean the cached plans
    //    before starting another enumeration
    this.nextWorkset.accept(PlanCacheCleaner.INSTANCE);
    this.solutionSetDelta.accept(PlanCacheCleaner.INSTANCE);

    // 2) Give the partial solution the properties of the current candidate for the initial partial
    // solution
    //    This concerns currently only the workset.
    this.worksetNode.setCandidateProperties(
        worksetIn.getGlobalProperties(), worksetIn.getLocalProperties());
    this.solutionSetNode.setCandidateProperties(this.partitionedProperties, new LocalProperties());

    final SolutionSetPlanNode sspn = this.solutionSetNode.getCurrentSolutionSetPlanNode();
    final WorksetPlanNode wspn = this.worksetNode.getCurrentWorksetPlanNode();

    // 3) Get the alternative plans
    List<PlanNode> solutionSetDeltaCandidates =
        this.solutionSetDelta.getAlternativePlans(estimator);
    List<PlanNode> worksetCandidates = this.nextWorkset.getAlternativePlans(estimator);

    // 4) Throw away all that are not compatible with the properties currently requested to the
    //    initial partial solution

    // Make sure that the workset candidates fulfill the input requirements
    for (Iterator<PlanNode> planDeleter = worksetCandidates.iterator(); planDeleter.hasNext(); ) {
      PlanNode candidate = planDeleter.next();
      if (!(globPropsReqWorkset.isMetBy(candidate.getGlobalProperties())
          && locPropsReqWorkset.isMetBy(candidate.getLocalProperties()))) {
        planDeleter.remove();
      }
    }
    if (worksetCandidates.isEmpty()) {
      return;
    }

    // sanity check the solution set delta and cancel out the delta node, if it is not needed
    for (Iterator<PlanNode> deltaPlans = solutionSetDeltaCandidates.iterator();
        deltaPlans.hasNext(); ) {
      SingleInputPlanNode candidate = (SingleInputPlanNode) deltaPlans.next();
      GlobalProperties gp = candidate.getGlobalProperties();

      if (gp.getPartitioning() != PartitioningProperty.HASH_PARTITIONED
          || gp.getPartitioningFields() == null
          || !gp.getPartitioningFields().equals(this.solutionSetKeyFields)) {
        throw new CompilerException("Bug: The solution set delta is not partitioned.");
      }
    }

    // 5) Create a candidate for the Iteration Node for every remaining plan of the step function.

    final GlobalProperties gp = new GlobalProperties();
    gp.setHashPartitioned(this.solutionSetKeyFields);
    gp.addUniqueFieldCombination(this.solutionSetKeyFields);

    final LocalProperties lp = new LocalProperties();
    lp.addUniqueFields(this.solutionSetKeyFields);

    // take all combinations of solution set delta and workset plans
    for (PlanNode solutionSetCandidate : solutionSetDeltaCandidates) {
      for (PlanNode worksetCandidate : worksetCandidates) {
        // check whether they have the same operator at their latest branching point
        if (this.singleRoot.areBranchCompatible(solutionSetCandidate, worksetCandidate)) {

          SingleInputPlanNode siSolutionDeltaCandidate = (SingleInputPlanNode) solutionSetCandidate;
          boolean immediateDeltaUpdate;

          // check whether we need a dedicated solution set delta operator, or whether we can update
          // on the fly
          if (siSolutionDeltaCandidate.getInput().getShipStrategy() == ShipStrategyType.FORWARD
              && this.solutionDeltaImmediatelyAfterSolutionJoin) {
            // we do not need this extra node. we can make the predecessor the delta
            // sanity check the node and connection
            if (siSolutionDeltaCandidate.getDriverStrategy() != DriverStrategy.UNARY_NO_OP
                || siSolutionDeltaCandidate.getInput().getLocalStrategy() != LocalStrategy.NONE) {
              throw new CompilerException("Invalid Solution set delta node.");
            }

            solutionSetCandidate = siSolutionDeltaCandidate.getInput().getSource();
            immediateDeltaUpdate = true;
          } else {
            // was not partitioned, we need to keep this node.
            // mark that we materialize the input
            siSolutionDeltaCandidate.getInput().setTempMode(TempMode.PIPELINE_BREAKER);
            immediateDeltaUpdate = false;
          }

          WorksetIterationPlanNode wsNode =
              new WorksetIterationPlanNode(
                  this,
                  "WorksetIteration (" + this.getPactContract().getName() + ")",
                  solutionSetIn,
                  worksetIn,
                  sspn,
                  wspn,
                  worksetCandidate,
                  solutionSetCandidate);
          wsNode.setImmediateSolutionSetUpdate(immediateDeltaUpdate);
          wsNode.initProperties(gp, lp);
          target.add(wsNode);
        }
      }
    }
  }