@Override
    public WindowedValue<T> next() throws IOException {
      Windmill.Message message =
          context.getWork().getMessageBundles(bundleIndex).getMessages(messageIndex);
      if (messageIndex >= context.getWork().getMessageBundles(bundleIndex).getMessagesCount() - 1) {
        messageIndex = 0;
        bundleIndex++;
      } else {
        messageIndex++;
      }
      Instant timestampMillis = new Instant(TimeUnit.MICROSECONDS.toMillis(message.getTimestamp()));
      InputStream data = message.getData().newInput();
      InputStream metadata = message.getMetadata().newInput();
      Collection<? extends BoundedWindow> windows =
          WindmillSink.decodeMetadataWindows(windowsCoder, message.getMetadata());
      PaneInfo pane = WindmillSink.decodeMetadataPane(message.getMetadata());
      if (valueCoder instanceof KvCoder) {
        KvCoder<?, ?> kvCoder = (KvCoder<?, ?>) valueCoder;
        InputStream key = context.getSerializedKey().newInput();
        notifyElementRead(key.available() + data.available() + metadata.available());

        @SuppressWarnings("unchecked")
        T result =
            (T) KV.of(decode(kvCoder.getKeyCoder(), key), decode(kvCoder.getValueCoder(), data));
        return WindowedValue.of(result, timestampMillis, windows, pane);
      } else {
        notifyElementRead(data.available() + metadata.available());
        return WindowedValue.of(decode(valueCoder, data), timestampMillis, windows, pane);
      }
    }
  @Override
  public void flatMap(IN value, Collector<WindowedValue<OUT>> out) throws Exception {

    @SuppressWarnings("unchecked")
    OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE;
    for (byte[] element : elements) {
      ByteArrayInputStream bai = new ByteArrayInputStream(element);
      OUT outValue = coder.decode(bai, Coder.Context.OUTER);

      if (outValue == null) {
        out.collect(
            WindowedValue.of(voidValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
      } else {
        out.collect(
            WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
      }
    }

    out.close();
  }
  @Override
  public void flatMap(WindowedValue<IN> value, Collector<WindowedValue<OUTFL>> out)
      throws Exception {
    this.initContext(doFn, out);

    // for each window the element belongs to, create a new copy here.
    Collection<? extends BoundedWindow> windows = value.getWindows();
    if (windows.size() <= 1) {
      processElement(value);
    } else {
      for (BoundedWindow window : windows) {
        processElement(
            WindowedValue.of(value.getValue(), value.getTimestamp(), window, value.getPane()));
      }
    }
  }
  protected <T> WindowedValue<T> makeWindowedValue(
      T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
    final Instant inputTimestamp = timestamp;
    final WindowFn windowFn = windowingStrategy.getWindowFn();

    if (timestamp == null) {
      timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
    }

    if (windows == null) {
      try {
        windows =
            windowFn.assignWindows(
                windowFn.new AssignContext() {
                  @Override
                  public Object element() {
                    throw new UnsupportedOperationException(
                        "WindowFn attempted to access input element when none was available"); // TODO: 12/16/15 aljoscha's comment in slack
                  }

                  @Override
                  public Instant timestamp() {
                    if (inputTimestamp == null) {
                      throw new UnsupportedOperationException(
                          "WindowFn attempted to access input timestamp when none was available");
                    }
                    return inputTimestamp;
                  }

                  @Override
                  public Collection<? extends BoundedWindow> windows() {
                    throw new UnsupportedOperationException(
                        "WindowFn attempted to access input windows when none were available");
                  }
                });
      } catch (Exception e) {
        Throwables.propagateIfInstanceOf(e, UserCodeException.class);
        throw new UserCodeException(e);
      }
    }

    return WindowedValue.of(output, timestamp, windows, pane);
  }