Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

The task starts processing at the input operators of the operator DAG. When it We introduce a class `DrainStates` which handles all the book-keeping for drain messages. When the operator encounters a draincontrol message message, it updates the internal bookkeeping tracking all the SSPs for a system stream which received a drain message. If all input streams for the operator receive a drain control message, it propagates the message to the downstream operators in the operator DAG. 

In case of a shuffle stage, `DrainMessage` is always sent to a single partition of the downstream stage (intermediate stream) by all source partitions. This is done for aggregation and `DrainStates` keep a track of it. Once the downstream aggregate partition has received drain messages from all the parent partitions, drain message is broadcasted to all the peer partitions. This stage repeats the same process for the downstream stages.

handleDrain will be added to the base OperatorImpl implementation which executes when all input streams to the operator have been received. In the case of a broadcast or partition operator, handleDrain will write Drain control messages to the intermediate stream for propagation.

Code Block
languagejava
titleDrainStates
collapsetrue
/*
 * Licensed to the Apache Software Foundation (ASF) under one
 * or more contributor license agreements.  See the NOTICE file
 * distributed with this work for additional information
 * regarding copyright ownership.  The ASF licenses this file
 * to you under the Apache License, Version 2.0 (the
 * "License"); you may not use this file except in compliance
 * with the License.  You may obtain a copy of the License at
 *
 *   http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing,
 * software distributed under the License is distributed on an
 * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 * KIND, either express or implied.  See the License for the
 * specific language governing permissions and limitations
 * under the License.
 */

package org.apache.samza.operators.impl;

import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.samza.system.DrainMessage;
import org.apache.samza.system.SystemStream;
import org.apache.samza.system.SystemStreamPartition;


/**
 * This class tracks the drain state for streams in a task. Internally it keeps track of Drain messages received
 * from upstream tasks for each system stream partition (ssp). If messages have been received from all tasks,
 * it will mark the ssp as drained. For a stream to be drained, all its partitions assigned to
 * the task need to be drained.
 *
 * This class is thread-safe.
 */
public class DrainStates {
  private static final class DrainState {
    // set of upstream tasks
    private final Set<String> tasks = new HashSet<>();
    private final int expectedTotal;
    private volatile boolean drained = false;

    DrainState(int expectedTotal) {
      this.expectedTotal = expectedTotal;
    }

    synchronized void update(String taskName) {
      if (taskName != null) {
        // aggregate the drain messages
        tasks.add(taskName);
        drained = tasks.size() == expectedTotal;
      } else {
        // drain is coming from either source or aggregator task
        drained = true;
      }
    }

    boolean isDrained() {
      return drained;
    }

    @Override
    public String toString() {
      return "DrainState: [Tasks : "
          + tasks
          + ", isDrained : "
          + drained
          + "]";
    }
  }

  private final Map<SystemStreamPartition, DrainState> drainStates;

  /**
   * Constructing the drain states for a task.
   * @param ssps all the ssps assigned to this task
   * @param producerTaskCounts mapping from a stream to the number of upstream tasks that produce to it
   */
  DrainStates(Set<SystemStreamPartition> ssps, Map<SystemStream, Integer> producerTaskCounts) {
    this.drainStates = ssps.stream()
        .collect(Collectors.toMap(
          ssp -> ssp,
          ssp -> new DrainState(producerTaskCounts.getOrDefault(ssp.getSystemStream(), 0))));
  }

  /**
   * Update the state upon receiving a drain message.
   * @param drain message of {@link DrainMessage}
   * @param ssp system stream partition
   */
  void update(DrainMessage drain, SystemStreamPartition ssp) {
    DrainState state = drainStates.get(ssp);
    state.update(drain.getTaskName());
  }

  /**
   * Checks if the system-stream is drained.
   * */
  boolean isDrained(SystemStream systemStream) {
    return drainStates.entrySet().stream()
        .filter(entry -> entry.getKey().getSystemStream().equals(systemStream))
        .allMatch(entry -> entry.getValue().isDrained());
  }

  /**
   * Checks if all streams (input SSPs) for the task has drained.
   * */
  boolean areAllStreamsDrained() {
    return drainStates.values().stream().allMatch(DrainState::isDrained);
  }

  @Override
  public String toString() {
    return drainStates.toString();
  }
}



Code Block
languagejava
titleOperatorImpl
collapsetrue
/**
* Abstract base class for all stream operator implementations.
*
* @param <M> type of the input to this operator
* @param <RM> type of the results of applying this operator
*/
public abstract class OperatorImpl<M, RM> {
 /**
   * This method is implemented when all input streams to this operation have encountered drain control message.
   * Inherited class should handle drain by overriding this function.
   * By default noop implementation is for in-memory operator to handle the drain. Output operator need to
   * override this to actually propagate drain over the wire.
   * @param collector message collector
   * @param coordinator task coordinator
   * @return results to be emitted when this operator reaches drain
   */
   protected  Collection<RM> handleDrain(MessageCollector collector, TaskCoordinator coordinator) {
     return Collections.emptyList();
   }
}

...