@Override
  public void processElement(ProcessContext c) throws Exception {
    KeyedWorkItem<K, InputT> element = c.element();

    K key = c.element().key();
    TimerInternals timerInternals = c.windowingInternals().timerInternals();
    StateInternals<K> stateInternals = stateInternalsFactory.stateInternalsForKey(key);

    ReduceFnRunner<K, InputT, OutputT, W> reduceFnRunner =
        new ReduceFnRunner<>(
            key,
            windowingStrategy,
            ExecutableTriggerStateMachine.create(
                TriggerStateMachines.stateMachineForTrigger(windowingStrategy.getTrigger())),
            stateInternals,
            timerInternals,
            WindowingInternalsAdapters.outputWindowedValue(c.windowingInternals()),
            WindowingInternalsAdapters.sideInputReader(c.windowingInternals()),
            droppedDueToClosedWindow,
            reduceFn,
            c.getPipelineOptions());

    reduceFnRunner.processElements(element.elementsIterable());
    for (TimerData timer : element.timersIterable()) {
      reduceFnRunner.onTimer(timer);
    }
    reduceFnRunner.persist();
  }
 private void closeWindow(
     K key, W w, Map<W, AccumT> accumulators, Map<W, Instant> minTimestamps, ProcessContext c) {
   AccumT accum = accumulators.remove(w);
   Instant timestamp = minTimestamps.remove(w);
   checkState(accum != null && timestamp != null);
   c.windowingInternals()
       .outputWindowedValue(
           KV.of(key, combineFn.extractOutput(key, accum)),
           timestamp,
           Arrays.asList(w),
           PaneInfo.ON_TIME_AND_ONLY_FIRING);
 }
 @Override
 public WindowingInternals<InputT, OutputT> windowingInternals() {
   return context.windowingInternals();
 }